diff --git a/.baseline/checkstyle/checkstyle.xml b/.baseline/checkstyle/checkstyle.xml index c5ce13558af4..9df746fe8636 100644 --- a/.baseline/checkstyle/checkstyle.xml +++ b/.baseline/checkstyle/checkstyle.xml @@ -46,6 +46,11 @@ + + + + + diff --git a/.gitignore b/.gitignore index 3c3e72482a31..22b7a290467e 100644 --- a/.gitignore +++ b/.gitignore @@ -25,6 +25,12 @@ lib/ # web site build site/site +# benchmark output +spark2/benchmark/* +!spark2/benchmark/.gitkeep +spark3/benchmark/* +!spark3/benchmark/.gitkeep + __pycache__/ *.py[cod] .eggs/ diff --git a/README.md b/README.md index 86d32bd56bab..79514dd061a5 100644 --- a/README.md +++ b/README.md @@ -19,7 +19,8 @@ ![](site/docs/img/Iceberg-logo.png) -[![](https://travis-ci.org/apache/iceberg.svg?branch=master)](https://travis-ci.org/apache/iceberg) +[![](https://github.com/apache/iceberg/actions/workflows/java-ci.yml/badge.svg)](https://github.com/apache/iceberg/actions/workflows/java-ci.yml) +[![](https://github.com/apache/iceberg/actions/workflows/python-ci.yml/badge.svg)](https://github.com/apache/iceberg/actions/workflows/python-ci.yml) [![Slack](https://img.shields.io/badge/chat-on%20Slack-brightgreen.svg)](https://the-asf.slack.com/archives/CF01LKV9S) Apache Iceberg is a new table format for storing large, slow-moving tabular data. It is designed to improve on the de-facto standard table layout built into Hive, Trino, and Spark. diff --git a/api/src/main/java/org/apache/iceberg/ManifestFile.java b/api/src/main/java/org/apache/iceberg/ManifestFile.java index 95c492385c80..7eb89f49b5ed 100644 --- a/api/src/main/java/org/apache/iceberg/ManifestFile.java +++ b/api/src/main/java/org/apache/iceberg/ManifestFile.java @@ -62,14 +62,16 @@ public interface ManifestFile { Types.NestedField PARTITION_SUMMARIES = optional(507, "partitions", Types.ListType.ofRequired(508, PARTITION_SUMMARY_TYPE), "Summary for each partition"); - // next ID to assign: 519 + Types.NestedField KEY_METADATA = optional(519, "key_metadata", Types.BinaryType.get(), + "Encryption key metadata blob"); + // next ID to assign: 520 Schema SCHEMA = new Schema( PATH, LENGTH, SPEC_ID, MANIFEST_CONTENT, SEQUENCE_NUMBER, MIN_SEQUENCE_NUMBER, SNAPSHOT_ID, ADDED_FILES_COUNT, EXISTING_FILES_COUNT, DELETED_FILES_COUNT, ADDED_ROWS_COUNT, EXISTING_ROWS_COUNT, DELETED_ROWS_COUNT, - PARTITION_SUMMARIES); + PARTITION_SUMMARIES, KEY_METADATA); static Schema schema() { return SCHEMA; @@ -179,6 +181,13 @@ default boolean hasDeletedFiles() { */ List partitions(); + /** + * Returns metadata about how this manifest file is encrypted, or null if the file is stored in plain text. + */ + default ByteBuffer keyMetadata() { + return null; + } + /** * Copies this {@link ManifestFile manifest file}. Readers can reuse manifest file instances; use * this method to make defensive copies. diff --git a/api/src/main/java/org/apache/iceberg/Schema.java b/api/src/main/java/org/apache/iceberg/Schema.java index 06a48872edd0..dea54aab6279 100644 --- a/api/src/main/java/org/apache/iceberg/Schema.java +++ b/api/src/main/java/org/apache/iceberg/Schema.java @@ -193,11 +193,11 @@ public List columns() { * It consists of a unique set of primitive fields in the schema. * An identifier field must be at root, or nested in a chain of structs (no maps or lists). * A row should be unique in a table based on the values of the identifier fields. + * Optional, float and double columns cannot be used as identifier fields. * However, Iceberg identifier differs from primary key in the following ways: *
    *
  • Iceberg does not enforce the uniqueness of a row based on this identifier information. * It is used for operations like upsert to define the default upsert key.
  • - *
  • NULL can be used as value of an identifier field. Iceberg ensures null-safe equality check.
  • *
  • A nested field in a struct can be used as an identifier. For example, if there is a "last_name" field * inside a "user" struct in a schema, field "user.last_name" can be set as a part of the identifier field.
  • *
@@ -215,7 +215,7 @@ public Set identifierFieldIds() { public Set identifierFieldNames() { return identifierFieldIds() .stream() - .map(id -> findField(id).name()) + .map(id -> lazyIdToName().get(id)) .collect(Collectors.toSet()); } diff --git a/api/src/main/java/org/apache/iceberg/Snapshot.java b/api/src/main/java/org/apache/iceberg/Snapshot.java index 3fd8714f9840..d10a6c3a49d1 100644 --- a/api/src/main/java/org/apache/iceberg/Snapshot.java +++ b/api/src/main/java/org/apache/iceberg/Snapshot.java @@ -126,4 +126,13 @@ public interface Snapshot extends Serializable { * @return the location of the manifest list for this Snapshot */ String manifestListLocation(); + + /** + * Return the id of the schema used when this snapshot was created, or null if this information is not available. + * + * @return schema id associated with this snapshot + */ + default Integer schemaId() { + return null; + } } diff --git a/api/src/main/java/org/apache/iceberg/SortField.java b/api/src/main/java/org/apache/iceberg/SortField.java index 07131b8b9e09..f980d6f8e4de 100644 --- a/api/src/main/java/org/apache/iceberg/SortField.java +++ b/api/src/main/java/org/apache/iceberg/SortField.java @@ -80,7 +80,7 @@ public NullOrder nullOrder() { * @return true if this order satisfies the given order */ public boolean satisfies(SortField other) { - if (this == other) { + if (Objects.equals(this, other)) { return true; } else if (sourceId != other.sourceId || direction != other.direction || nullOrder != other.nullOrder) { return false; diff --git a/api/src/main/java/org/apache/iceberg/Table.java b/api/src/main/java/org/apache/iceberg/Table.java index a4557c8304de..854d68b5887f 100644 --- a/api/src/main/java/org/apache/iceberg/Table.java +++ b/api/src/main/java/org/apache/iceberg/Table.java @@ -60,6 +60,13 @@ default String name() { */ Schema schema(); + /** + * Return a map of {@link Schema schema} for this table. + * + * @return this table's schema map + */ + Map schemas(); + /** * Return the {@link PartitionSpec partition spec} for this table. * diff --git a/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java b/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java index 30adfea9ee9c..f2564ddb703b 100644 --- a/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java +++ b/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java @@ -41,10 +41,10 @@ default MigrateTable migrateTable(String tableIdent) { } /** - * Instantiates an action to remove orphan files. + * Instantiates an action to delete orphan files. */ - default RemoveOrphanFiles removeOrphanFiles(Table table) { - throw new UnsupportedOperationException(this.getClass().getName() + " does not implement removeOrphanFiles"); + default DeleteOrphanFiles deleteOrphanFiles(Table table) { + throw new UnsupportedOperationException(this.getClass().getName() + " does not implement deleteOrphanFiles"); } /** @@ -69,9 +69,9 @@ default ExpireSnapshots expireSnapshots(Table table) { } /** - * Instantiates an action to remove all the files reachable from given metadata location. + * Instantiates an action to delete all the files reachable from given metadata location. */ - default RemoveReachableFiles removeReachableFiles(String metadataLocation) { - throw new UnsupportedOperationException(this.getClass().getName() + " does not implement removeReachableFiles"); + default DeleteReachableFiles deleteReachableFiles(String metadataLocation) { + throw new UnsupportedOperationException(this.getClass().getName() + " does not implement deleteReachableFiles"); } } diff --git a/api/src/main/java/org/apache/iceberg/actions/RemoveOrphanFiles.java b/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java similarity index 89% rename from api/src/main/java/org/apache/iceberg/actions/RemoveOrphanFiles.java rename to api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java index 765eba1994f8..b841edb5cfbe 100644 --- a/api/src/main/java/org/apache/iceberg/actions/RemoveOrphanFiles.java +++ b/api/src/main/java/org/apache/iceberg/actions/DeleteOrphanFiles.java @@ -22,13 +22,13 @@ import java.util.function.Consumer; /** - * An action that removes orphan files in a table. + * An action that deletes orphan files in a table. *

* A metadata or data file is considered orphan if it is not reachable by any valid snapshot. * The set of actual files is built by listing the underlying storage which makes this operation * expensive. */ -public interface RemoveOrphanFiles extends Action { +public interface DeleteOrphanFiles extends Action { /** * Passes a location which should be scanned for orphan files. *

@@ -38,7 +38,7 @@ public interface RemoveOrphanFiles extends Action deleteFunc); + DeleteOrphanFiles deleteWith(Consumer deleteFunc); /** * The action result that contains a summary of the execution. diff --git a/api/src/main/java/org/apache/iceberg/actions/RemoveReachableFiles.java b/api/src/main/java/org/apache/iceberg/actions/DeleteReachableFiles.java similarity index 73% rename from api/src/main/java/org/apache/iceberg/actions/RemoveReachableFiles.java rename to api/src/main/java/org/apache/iceberg/actions/DeleteReachableFiles.java index c643f2e63f95..3b0eb96b55cd 100644 --- a/api/src/main/java/org/apache/iceberg/actions/RemoveReachableFiles.java +++ b/api/src/main/java/org/apache/iceberg/actions/DeleteReachableFiles.java @@ -24,7 +24,7 @@ import org.apache.iceberg.io.FileIO; /** - * An action that removes all files referenced by a table metadata file. + * An action that deletes all files referenced by a table metadata file. *

* This action will irreversibly delete all reachable files such as data files, manifests, * manifest lists and should be used to clean up the underlying storage once a table is dropped @@ -32,16 +32,16 @@ *

* Implementations may use a query engine to distribute parts of work. */ -public interface RemoveReachableFiles extends Action { +public interface DeleteReachableFiles extends Action { /** * Passes an alternative delete implementation that will be used for files. * - * @param removeFunc a function that will be called to delete files. + * @param deleteFunc a function that will be called to delete files. * The function accepts path to file as an argument. * @return this for method chaining */ - RemoveReachableFiles deleteWith(Consumer removeFunc); + DeleteReachableFiles deleteWith(Consumer deleteFunc); /** * Passes an alternative executor service that will be used for files removal. @@ -51,7 +51,7 @@ public interface RemoveReachableFiles extends Action * When grouping files, the underlying rewrite strategy will use this value as to limit the files which * will be included in a single file group. A group will be processed by a single framework "action". For example, @@ -68,7 +68,7 @@ public interface RewriteDataFiles extends SnapshotUpdate resultMap(); + List rewriteResults(); + + default int addedDataFilesCount() { + return rewriteResults().stream().mapToInt(FileGroupRewriteResult::addedDataFilesCount).sum(); + } + + default int rewrittenDataFilesCount() { + return rewriteResults().stream().mapToInt(FileGroupRewriteResult::rewrittenDataFilesCount).sum(); + } } /** @@ -114,6 +116,8 @@ interface Result { * which were formerly part of the table but have been rewritten. */ interface FileGroupRewriteResult { + FileGroupInfo info(); + int addedDataFilesCount(); int rewrittenDataFilesCount(); diff --git a/api/src/main/java/org/apache/iceberg/catalog/Namespace.java b/api/src/main/java/org/apache/iceberg/catalog/Namespace.java index e08dd6ffe537..b8d8de9c668e 100644 --- a/api/src/main/java/org/apache/iceberg/catalog/Namespace.java +++ b/api/src/main/java/org/apache/iceberg/catalog/Namespace.java @@ -21,6 +21,7 @@ import java.util.Arrays; import org.apache.iceberg.relocated.com.google.common.base.Joiner; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; /** * A namespace in a {@link Catalog}. @@ -34,6 +35,7 @@ public static Namespace empty() { } public static Namespace of(String... levels) { + Preconditions.checkArgument(null != levels, "Cannot create Namespace from null array"); if (levels.length == 0) { return empty(); } diff --git a/api/src/main/java/org/apache/iceberg/catalog/TableIdentifier.java b/api/src/main/java/org/apache/iceberg/catalog/TableIdentifier.java index 05de7fd22ec8..ecbaa0646f84 100644 --- a/api/src/main/java/org/apache/iceberg/catalog/TableIdentifier.java +++ b/api/src/main/java/org/apache/iceberg/catalog/TableIdentifier.java @@ -36,6 +36,7 @@ public class TableIdentifier { private final String name; public static TableIdentifier of(String... names) { + Preconditions.checkArgument(names != null, "Cannot create table identifier from null array"); Preconditions.checkArgument(names.length > 0, "Cannot create table identifier without a table name"); return new TableIdentifier(Namespace.of(Arrays.copyOf(names, names.length - 1)), names[names.length - 1]); } @@ -45,12 +46,14 @@ public static TableIdentifier of(Namespace namespace, String name) { } public static TableIdentifier parse(String identifier) { + Preconditions.checkArgument(identifier != null, "Cannot parse table identifier: null"); Iterable parts = DOT.split(identifier); return TableIdentifier.of(Iterables.toArray(parts, String.class)); } private TableIdentifier(Namespace namespace, String name) { - Preconditions.checkArgument(name != null && !name.isEmpty(), "Invalid table name %s", name); + Preconditions.checkArgument(name != null && !name.isEmpty(), "Invalid table name: null or empty"); + Preconditions.checkArgument(namespace != null, "Invalid Namespace: null"); this.namespace = namespace; this.name = name; } diff --git a/api/src/main/java/org/apache/iceberg/types/TypeUtil.java b/api/src/main/java/org/apache/iceberg/types/TypeUtil.java index 11c66f946e20..5185038f66fb 100644 --- a/api/src/main/java/org/apache/iceberg/types/TypeUtil.java +++ b/api/src/main/java/org/apache/iceberg/types/TypeUtil.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Predicate; @@ -45,7 +46,7 @@ public static Schema select(Schema schema, Set fieldIds) { Preconditions.checkNotNull(schema, "Schema cannot be null"); Types.StructType result = select(schema.asStruct(), fieldIds); - if (schema.asStruct() == result) { + if (Objects.equals(schema.asStruct(), result)) { return schema; } else if (result != null) { if (schema.getAliases() != null) { diff --git a/api/src/main/java/org/apache/iceberg/util/CharSequenceSet.java b/api/src/main/java/org/apache/iceberg/util/CharSequenceSet.java index 5148191f084d..80c8284bce39 100644 --- a/api/src/main/java/org/apache/iceberg/util/CharSequenceSet.java +++ b/api/src/main/java/org/apache/iceberg/util/CharSequenceSet.java @@ -32,11 +32,11 @@ public class CharSequenceSet implements Set, Serializable { private static final ThreadLocal wrappers = ThreadLocal.withInitial( () -> CharSequenceWrapper.wrap(null)); - public static Set of(Iterable charSequences) { + public static CharSequenceSet of(Iterable charSequences) { return new CharSequenceSet(charSequences); } - public static Set empty() { + public static CharSequenceSet empty() { return new CharSequenceSet(ImmutableList.of()); } @@ -116,6 +116,7 @@ public boolean remove(Object obj) { } @Override + @SuppressWarnings("CollectionUndefinedEquality") public boolean containsAll(Collection objects) { if (objects != null) { return Iterables.all(objects, this::contains); diff --git a/api/src/test/java/org/apache/iceberg/AssertHelpers.java b/api/src/test/java/org/apache/iceberg/AssertHelpers.java index c6bcc1b1de82..1874d1b75072 100644 --- a/api/src/test/java/org/apache/iceberg/AssertHelpers.java +++ b/api/src/test/java/org/apache/iceberg/AssertHelpers.java @@ -20,7 +20,10 @@ package org.apache.iceberg; import java.util.concurrent.Callable; -import org.junit.Assert; +import org.apache.avro.AvroRuntimeException; +import org.apache.avro.generic.GenericRecord; +import org.assertj.core.api.AbstractThrowableAssert; +import org.assertj.core.api.Assertions; public class AssertHelpers { @@ -39,12 +42,11 @@ public static void assertThrows(String message, Class expected, String containedInMessage, Callable callable) { - try { - callable.call(); - Assert.fail("No exception was thrown (" + message + "), expected: " + - expected.getName()); - } catch (Exception actual) { - handleException(message, expected, containedInMessage, actual); + AbstractThrowableAssert check = Assertions.assertThatThrownBy(callable::call) + .as(message) + .isInstanceOf(expected); + if (null != containedInMessage) { + check.hasMessageContaining(containedInMessage); } } @@ -60,12 +62,11 @@ public static void assertThrows(String message, Class expected, String containedInMessage, Runnable runnable) { - try { - runnable.run(); - Assert.fail("No exception was thrown (" + message + "), expected: " + - expected.getName()); - } catch (Exception actual) { - handleException(message, expected, containedInMessage, actual); + AbstractThrowableAssert check = Assertions.assertThatThrownBy(runnable::run) + .as(message) + .isInstanceOf(expected); + if (null != containedInMessage) { + check.hasMessageContaining(containedInMessage); } } @@ -105,36 +106,22 @@ public static void assertThrowsCause(String message, Class expected, String containedInMessage, Runnable runnable) { - try { - runnable.run(); - Assert.fail("No exception was thrown (" + message + "), expected: " + - expected.getName()); - } catch (Exception actual) { - Throwable cause = actual.getCause(); - if (cause instanceof Exception) { - handleException(message, expected, containedInMessage, (Exception) actual.getCause()); - } else { - Assert.fail("Occur non-exception cause: " + cause); - } - } + Assertions.assertThatThrownBy(runnable::run) + .as(message) + .getCause() + .isInstanceOf(expected) + .hasMessageContaining(containedInMessage); } - private static void handleException(String message, - Class expected, - String containedInMessage, - Exception actual) { - try { - Assert.assertEquals(message, expected, actual.getClass()); - if (containedInMessage != null) { - Assert.assertTrue( - "Expected exception message (" + containedInMessage + ") missing: " + - actual.getMessage(), - actual.getMessage().contains(containedInMessage) - ); - } - } catch (AssertionError e) { - e.addSuppressed(actual); - throw e; - } + /** + * A convenience method to check if an Avro field is empty. + * @param record The record to read from + * @param field The name of the field + */ + public static void assertEmptyAvroField(GenericRecord record, String field) { + AssertHelpers.assertThrows( + "Not a valid schema field: " + field, + AvroRuntimeException.class, + () -> record.get(field)); } } diff --git a/api/src/test/java/org/apache/iceberg/TestHelpers.java b/api/src/test/java/org/apache/iceberg/TestHelpers.java index fd4ea08036c4..401266ba42b4 100644 --- a/api/src/test/java/org/apache/iceberg/TestHelpers.java +++ b/api/src/test/java/org/apache/iceberg/TestHelpers.java @@ -34,6 +34,7 @@ import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.ExpressionVisitors; import org.apache.iceberg.expressions.UnboundPredicate; +import org.apache.iceberg.util.ByteBuffers; import org.junit.Assert; public class TestHelpers { @@ -120,6 +121,21 @@ public static void assertSerializedAndLoadedMetadata(Table expected, Table actua Assert.assertEquals("History must match", expected.history(), actual.history()); } + public static void assertSameSchemaMap(Map map1, Map map2) { + if (map1.size() != map2.size()) { + Assert.fail("Should have same number of schemas in both maps"); + } + + map1.forEach((schemaId, schema1) -> { + Schema schema2 = map2.get(schemaId); + Assert.assertNotNull(String.format("Schema ID %s does not exist in map: %s", schemaId, map2), schema2); + + Assert.assertEquals("Should have matching schema id", schema1.schemaId(), schema2.schemaId()); + Assert.assertTrue(String.format("Should be the same schema. Schema 1: %s, schema 2: %s", schema1, schema2), + schema1.sameSchema(schema2)); + }); + } + private static class CheckReferencesBound extends ExpressionVisitors.ExpressionVisitor { private final String message; @@ -240,10 +256,11 @@ public static class TestManifestFile implements ManifestFile { private final Integer deletedFiles; private final Long deletedRows; private final List partitions; + private final byte[] keyMetadata; public TestManifestFile(String path, long length, int specId, Long snapshotId, Integer addedFiles, Integer existingFiles, Integer deletedFiles, - List partitions) { + List partitions, ByteBuffer keyMetadata) { this.path = path; this.length = length; this.specId = specId; @@ -256,12 +273,13 @@ public TestManifestFile(String path, long length, int specId, Long snapshotId, this.deletedFiles = deletedFiles; this.deletedRows = null; this.partitions = partitions; + this.keyMetadata = ByteBuffers.toByteArray(keyMetadata); } public TestManifestFile(String path, long length, int specId, ManifestContent content, Long snapshotId, Integer addedFiles, Long addedRows, Integer existingFiles, Long existingRows, Integer deletedFiles, Long deletedRows, - List partitions) { + List partitions, ByteBuffer keyMetadata) { this.path = path; this.length = length; this.specId = specId; @@ -274,6 +292,7 @@ public TestManifestFile(String path, long length, int specId, ManifestContent co this.deletedFiles = deletedFiles; this.deletedRows = deletedRows; this.partitions = partitions; + this.keyMetadata = ByteBuffers.toByteArray(keyMetadata); } @Override @@ -346,6 +365,11 @@ public List partitions() { return partitions; } + @Override + public ByteBuffer keyMetadata() { + return keyMetadata == null ? null : ByteBuffer.wrap(keyMetadata); + } + @Override public ManifestFile copy() { return this; diff --git a/api/src/test/java/org/apache/iceberg/catalog/TestNamespace.java b/api/src/test/java/org/apache/iceberg/catalog/TestNamespace.java new file mode 100644 index 000000000000..81b98b411bdf --- /dev/null +++ b/api/src/test/java/org/apache/iceberg/catalog/TestNamespace.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.catalog; + +import org.assertj.core.api.Assertions; +import org.junit.Test; + +public class TestNamespace { + + @Test + public void testWithNullAndEmpty() { + Assertions.assertThatThrownBy(() -> Namespace.of((String[]) null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot create Namespace from null array"); + + Assertions.assertThat(Namespace.of()).isEqualTo(Namespace.empty()); + } + + @Test + public void testNamespace() { + String[] levels = {"a", "b", "c", "d"}; + Namespace namespace = Namespace.of(levels); + Assertions.assertThat(namespace).isNotNull(); + Assertions.assertThat(namespace.levels()).isNotNull().hasSize(4); + Assertions.assertThat(namespace.toString()).isEqualTo("a.b.c.d"); + for (int i = 0; i < levels.length; i++) { + Assertions.assertThat(namespace.level(i)).isEqualTo(levels[i]); + } + } +} diff --git a/api/src/test/java/org/apache/iceberg/catalog/TestTableIdentifier.java b/api/src/test/java/org/apache/iceberg/catalog/TestTableIdentifier.java index b67bdcd52ce5..39f05ddc1986 100644 --- a/api/src/test/java/org/apache/iceberg/catalog/TestTableIdentifier.java +++ b/api/src/test/java/org/apache/iceberg/catalog/TestTableIdentifier.java @@ -19,6 +19,7 @@ package org.apache.iceberg.catalog; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; @@ -54,4 +55,30 @@ public void testToLowerCase() { TableIdentifier.of("Catalog", "dB", "TBL").toLowerCase(), TableIdentifier.of("catalog", "db", "tbl")); } + + @Test + public void testInvalidTableName() { + Assertions.assertThatThrownBy(() -> TableIdentifier.of(Namespace.empty(), "")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid table name: null or empty"); + + Assertions.assertThatThrownBy(() -> TableIdentifier.of(Namespace.empty(), null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid table name: null or empty"); + } + + @Test + public void testNulls() { + Assertions.assertThatThrownBy(() -> TableIdentifier.of((String[]) null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot create table identifier from null array"); + + Assertions.assertThatThrownBy(() -> TableIdentifier.parse(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse table identifier: null"); + + Assertions.assertThatThrownBy(() -> TableIdentifier.of(null, "name")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid Namespace: null"); + } } diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveManifestEvaluator.java b/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveManifestEvaluator.java index 972fc8ed5871..f41b8dcd7b76 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveManifestEvaluator.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveManifestEvaluator.java @@ -91,7 +91,7 @@ public class TestInclusiveManifestEvaluator { private static final ByteBuffer STRING_MAX = toByteBuffer(Types.StringType.get(), "z"); private static final ManifestFile NO_STATS = new TestHelpers.TestManifestFile( - "manifest-list.avro", 1024, 0, System.currentTimeMillis(), null, null, null, null); + "manifest-list.avro", 1024, 0, System.currentTimeMillis(), null, null, null, null, null); private static final ManifestFile FILE = new TestHelpers.TestManifestFile("manifest-list.avro", 1024, 0, System.currentTimeMillis(), 5, 10, 0, ImmutableList.of( @@ -110,7 +110,7 @@ public class TestInclusiveManifestEvaluator { toByteBuffer(Types.FloatType.get(), 0F), toByteBuffer(Types.FloatType.get(), 20F)), new TestHelpers.TestFieldSummary(true, null, null) - )); + ), null); @Test public void testAllNulls() { diff --git a/api/src/test/java/org/apache/iceberg/util/TestCharSequenceSet.java b/api/src/test/java/org/apache/iceberg/util/TestCharSequenceSet.java new file mode 100644 index 000000000000..d208de20db0e --- /dev/null +++ b/api/src/test/java/org/apache/iceberg/util/TestCharSequenceSet.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.iceberg.util; + +import java.util.Arrays; +import java.util.Set; +import org.assertj.core.api.Assertions; +import org.junit.Test; + +public class TestCharSequenceSet { + + // This test just verifies https://errorprone.info/bugpattern/CollectionUndefinedEquality + @Test + public void testSearchingInCharSequenceCollection() { + Set set = CharSequenceSet.of(Arrays.asList("abc", new StringBuffer("def"))); + Assertions.assertThat(set).contains("abc"); + Assertions.assertThat(set.stream().anyMatch("def"::contains)).isTrue(); + + // this would fail with a normal Set + Assertions.assertThat(set.contains("def")).isTrue(); + } +} diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/ArrowSchemaUtil.java b/arrow/src/main/java/org/apache/iceberg/arrow/ArrowSchemaUtil.java index 5bef4171f042..fe7ecc9e1759 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/ArrowSchemaUtil.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/ArrowSchemaUtil.java @@ -98,6 +98,9 @@ public static Field convert(final NestedField field) { case TIME: arrowType = new ArrowType.Time(TimeUnit.MICROSECOND, Long.SIZE); break; + case UUID: + arrowType = new ArrowType.FixedSizeBinary(16); + break; case TIMESTAMP: arrowType = new ArrowType.Timestamp(TimeUnit.MICROSECOND, ((Types.TimestampType) field.type()).shouldAdjustToUTC() ? "UTC" : null); diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java index 60b79a65b26b..2386ce55253a 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java @@ -20,12 +20,11 @@ package org.apache.iceberg.arrow.vectorized; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.Collection; -import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; import java.util.stream.StreamSupport; @@ -38,6 +37,7 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.TableScan; import org.apache.iceberg.encryption.EncryptedFiles; +import org.apache.iceberg.encryption.EncryptedInputFile; import org.apache.iceberg.encryption.EncryptionManager; import org.apache.iceberg.io.CloseableGroup; import org.apache.iceberg.io.CloseableIterable; @@ -49,6 +49,7 @@ import org.apache.iceberg.parquet.TypeWithSchemaVisitor; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; import org.apache.parquet.schema.MessageType; @@ -79,8 +80,8 @@ *

  • Columns with constant values are physically encoded as a dictionary. The Arrow vector * type is int32 instead of the type as per the schema. * See https://github.com/apache/iceberg/issues/2484.
  • - *
  • Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType}, - * {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and + *
  • Data types: {@link Types.ListType}, {@link Types.MapType}, + * {@link Types.StructType}, {@link Types.FixedType} and * {@link Types.DecimalType} * See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.
  • *
  • Iceberg v2 spec is not supported. @@ -210,11 +211,21 @@ private static final class VectorizedCombinedScanIterator implements CloseableIt .flatMap(Collection::stream) .collect(Collectors.toList()); this.fileItr = fileTasks.iterator(); - this.inputFiles = Collections.unmodifiableMap(fileTasks.stream() + + Map keyMetadata = Maps.newHashMap(); + fileTasks.stream() .flatMap(fileScanTask -> Stream.concat(Stream.of(fileScanTask.file()), fileScanTask.deletes().stream())) - .map(file -> EncryptedFiles.encryptedInput(io.newInputFile(file.path().toString()), file.keyMetadata())) - .map(encryptionManager::decrypt) - .collect(Collectors.toMap(InputFile::location, Function.identity()))); + .forEach(file -> keyMetadata.put(file.path().toString(), file.keyMetadata())); + + Stream encrypted = keyMetadata.entrySet().stream() + .map(entry -> EncryptedFiles.encryptedInput(io.newInputFile(entry.getKey()), entry.getValue())); + + // decrypt with the batch call to avoid multiple RPCs to a key server, if possible + Iterable decryptedFiles = encryptionManager.decrypt(encrypted::iterator); + + Map files = Maps.newHashMapWithExpectedSize(fileTasks.size()); + decryptedFiles.forEach(decrypted -> files.putIfAbsent(decrypted.location(), decrypted)); + this.inputFiles = ImmutableMap.copyOf(files); this.currentIterator = CloseableIterator.empty(); this.expectedSchema = expectedSchema; this.nameMapping = nameMapping; diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/GenericArrowVectorAccessorFactory.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/GenericArrowVectorAccessorFactory.java index df4e2166d061..77157cea4555 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/GenericArrowVectorAccessorFactory.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/GenericArrowVectorAccessorFactory.java @@ -30,9 +30,11 @@ import org.apache.arrow.vector.DateDayVector; import org.apache.arrow.vector.DecimalVector; import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.FixedSizeBinaryVector; import org.apache.arrow.vector.Float4Vector; import org.apache.arrow.vector.Float8Vector; import org.apache.arrow.vector.IntVector; +import org.apache.arrow.vector.TimeMicroVector; import org.apache.arrow.vector.TimeStampMicroTZVector; import org.apache.arrow.vector.TimeStampMicroVector; import org.apache.arrow.vector.ValueVector; @@ -112,6 +114,7 @@ private ArrowVectorAccessor getDict case BSON: return new DictionaryStringAccessor<>((IntVector) vector, dictionary, stringFactorySupplier.get()); case INT_64: + case TIME_MICROS: case TIMESTAMP_MILLIS: case TIMESTAMP_MICROS: return new DictionaryLongAccessor<>((IntVector) vector, dictionary); @@ -189,6 +192,10 @@ private ArrowVectorAccessor getDict } else if (vector instanceof StructVector) { StructVector structVector = (StructVector) vector; return new StructAccessor<>(structVector, structChildFactorySupplier.get()); + } else if (vector instanceof TimeMicroVector) { + return new TimeMicroAccessor<>((TimeMicroVector) vector); + } else if (vector instanceof FixedSizeBinaryVector) { + return new FixedSizeBinaryAccessor<>((FixedSizeBinaryVector) vector); } throw new UnsupportedOperationException("Unsupported vector: " + vector.getClass()); } @@ -469,6 +476,38 @@ public final long getLong(int rowId) { } } + private static class TimeMicroAccessor + extends ArrowVectorAccessor { + + private final TimeMicroVector vector; + + TimeMicroAccessor(TimeMicroVector vector) { + super(vector); + this.vector = vector; + } + + @Override + public final long getLong(int rowId) { + return vector.get(rowId); + } + } + + private static class FixedSizeBinaryAccessor + extends ArrowVectorAccessor { + + private final FixedSizeBinaryVector vector; + + FixedSizeBinaryAccessor(FixedSizeBinaryVector vector) { + super(vector); + this.vector = vector; + } + + @Override + public byte[] getBinary(int rowId) { + return vector.get(rowId); + } + } + private static class ArrayAccessor extends ArrowVectorAccessor { diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java index 5ed323f76566..152433f6e2f5 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java @@ -30,6 +30,7 @@ import org.apache.arrow.vector.Float4Vector; import org.apache.arrow.vector.Float8Vector; import org.apache.arrow.vector.IntVector; +import org.apache.arrow.vector.TimeMicroVector; import org.apache.arrow.vector.TimeStampMicroTZVector; import org.apache.arrow.vector.TimeStampMicroVector; import org.apache.arrow.vector.types.FloatingPointPrecision; @@ -108,6 +109,8 @@ private enum ReadType { FLOAT, DOUBLE, TIMESTAMP_MILLIS, + TIME_MICROS, + UUID, DICTIONARY } @@ -130,44 +133,45 @@ public VectorHolder read(VectorHolder reuse, int numValsToRead) { } if (vectorizedColumnIterator.hasNext()) { if (dictEncoded) { - vectorizedColumnIterator.nextBatchDictionaryIds((IntVector) vec, nullabilityHolder); + vectorizedColumnIterator.dictionaryBatchReader().nextBatch(vec, -1, nullabilityHolder); } else { switch (readType) { case FIXED_LENGTH_DECIMAL: - vectorizedColumnIterator.nextBatchFixedLengthDecimal(vec, typeWidth, nullabilityHolder); + vectorizedColumnIterator.fixedLengthDecimalBatchReader().nextBatch(vec, typeWidth, nullabilityHolder); break; case INT_BACKED_DECIMAL: - vectorizedColumnIterator.nextBatchIntBackedDecimal(vec, nullabilityHolder); + vectorizedColumnIterator.intBackedDecimalBatchReader().nextBatch(vec, -1, nullabilityHolder); break; case LONG_BACKED_DECIMAL: - vectorizedColumnIterator.nextBatchLongBackedDecimal(vec, nullabilityHolder); + vectorizedColumnIterator.longBackedDecimalBatchReader().nextBatch(vec, -1, nullabilityHolder); break; case VARBINARY: - vectorizedColumnIterator.nextBatchVarWidthType(vec, nullabilityHolder); - break; case VARCHAR: - vectorizedColumnIterator.nextBatchVarWidthType(vec, nullabilityHolder); + vectorizedColumnIterator.varWidthTypeBatchReader().nextBatch(vec, -1, nullabilityHolder); break; case FIXED_WIDTH_BINARY: - vectorizedColumnIterator.nextBatchFixedWidthBinary(vec, typeWidth, nullabilityHolder); + vectorizedColumnIterator.fixedWidthTypeBinaryBatchReader().nextBatch(vec, typeWidth, nullabilityHolder); break; case BOOLEAN: - vectorizedColumnIterator.nextBatchBoolean(vec, nullabilityHolder); + vectorizedColumnIterator.booleanBatchReader().nextBatch(vec, -1, nullabilityHolder); break; case INT: - vectorizedColumnIterator.nextBatchIntegers(vec, typeWidth, nullabilityHolder); + vectorizedColumnIterator.integerBatchReader().nextBatch(vec, typeWidth, nullabilityHolder); break; case LONG: - vectorizedColumnIterator.nextBatchLongs(vec, typeWidth, nullabilityHolder); + vectorizedColumnIterator.longBatchReader().nextBatch(vec, typeWidth, nullabilityHolder); break; case FLOAT: - vectorizedColumnIterator.nextBatchFloats(vec, typeWidth, nullabilityHolder); + vectorizedColumnIterator.floatBatchReader().nextBatch(vec, typeWidth, nullabilityHolder); break; case DOUBLE: - vectorizedColumnIterator.nextBatchDoubles(vec, typeWidth, nullabilityHolder); + vectorizedColumnIterator.doubleBatchReader().nextBatch(vec, typeWidth, nullabilityHolder); break; case TIMESTAMP_MILLIS: - vectorizedColumnIterator.nextBatchTimestampMillis(vec, typeWidth, nullabilityHolder); + vectorizedColumnIterator.timestampMillisBatchReader().nextBatch(vec, typeWidth, nullabilityHolder); + break; + case UUID: + vectorizedColumnIterator.fixedSizeBinaryBatchReader().nextBatch(vec, typeWidth, nullabilityHolder); break; } } @@ -180,152 +184,175 @@ public VectorHolder read(VectorHolder reuse, int numValsToRead) { private void allocateFieldVector(boolean dictionaryEncodedVector) { if (dictionaryEncodedVector) { - Field field = new Field( - icebergField.name(), - new FieldType(icebergField.isOptional(), new ArrowType.Int(Integer.SIZE, true), null, null), - null); - this.vec = field.createVector(rootAlloc); - ((IntVector) vec).allocateNew(batchSize); - this.typeWidth = (int) IntVector.TYPE_WIDTH; - this.readType = ReadType.DICTIONARY; + allocateDictEncodedVector(); } else { - PrimitiveType primitive = columnDescriptor.getPrimitiveType(); Field arrowField = ArrowSchemaUtil.convert(icebergField); - if (primitive.getOriginalType() != null) { - switch (primitive.getOriginalType()) { - case ENUM: - case JSON: - case UTF8: - case BSON: - this.vec = arrowField.createVector(rootAlloc); - // TODO: Possibly use the uncompressed page size info to set the initial capacity - vec.setInitialCapacity(batchSize * AVERAGE_VARIABLE_WIDTH_RECORD_SIZE); - vec.allocateNewSafe(); - this.readType = ReadType.VARCHAR; - this.typeWidth = UNKNOWN_WIDTH; - break; - case INT_8: - case INT_16: - case INT_32: - this.vec = arrowField.createVector(rootAlloc); - ((IntVector) vec).allocateNew(batchSize); - this.readType = ReadType.INT; - this.typeWidth = (int) IntVector.TYPE_WIDTH; - break; - case DATE: - this.vec = arrowField.createVector(rootAlloc); - ((DateDayVector) vec).allocateNew(batchSize); - this.readType = ReadType.INT; - this.typeWidth = (int) IntVector.TYPE_WIDTH; - break; - case INT_64: - this.vec = arrowField.createVector(rootAlloc); - ((BigIntVector) vec).allocateNew(batchSize); - this.readType = ReadType.LONG; - this.typeWidth = (int) BigIntVector.TYPE_WIDTH; - break; - case TIMESTAMP_MILLIS: - this.vec = arrowField.createVector(rootAlloc); - ((BigIntVector) vec).allocateNew(batchSize); - this.readType = ReadType.TIMESTAMP_MILLIS; - this.typeWidth = (int) BigIntVector.TYPE_WIDTH; - break; - case TIMESTAMP_MICROS: - this.vec = arrowField.createVector(rootAlloc); - if (((Types.TimestampType) icebergField.type()).shouldAdjustToUTC()) { - ((TimeStampMicroTZVector) vec).allocateNew(batchSize); - } else { - ((TimeStampMicroVector) vec).allocateNew(batchSize); - } - this.readType = ReadType.LONG; - this.typeWidth = (int) BigIntVector.TYPE_WIDTH; - break; - case DECIMAL: - this.vec = arrowField.createVector(rootAlloc); - ((DecimalVector) vec).allocateNew(batchSize); - switch (primitive.getPrimitiveTypeName()) { - case BINARY: - case FIXED_LEN_BYTE_ARRAY: - this.readType = ReadType.FIXED_LENGTH_DECIMAL; - this.typeWidth = primitive.getTypeLength(); - break; - case INT64: - this.readType = ReadType.LONG_BACKED_DECIMAL; - this.typeWidth = (int) BigIntVector.TYPE_WIDTH; - break; - case INT32: - this.readType = ReadType.INT_BACKED_DECIMAL; - this.typeWidth = (int) IntVector.TYPE_WIDTH; - break; - default: - throw new UnsupportedOperationException( - "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName()); - } - break; - default: - throw new UnsupportedOperationException( - "Unsupported logical type: " + primitive.getOriginalType()); - } + if (columnDescriptor.getPrimitiveType().getOriginalType() != null) { + allocateVectorBasedOnOriginalType(columnDescriptor.getPrimitiveType(), arrowField); } else { + allocateVectorBasedOnTypeName(columnDescriptor.getPrimitiveType(), arrowField); + } + } + } + + private void allocateDictEncodedVector() { + Field field = new Field( + icebergField.name(), + new FieldType(icebergField.isOptional(), new ArrowType.Int(Integer.SIZE, true), null, null), + null); + this.vec = field.createVector(rootAlloc); + ((IntVector) vec).allocateNew(batchSize); + this.typeWidth = (int) IntVector.TYPE_WIDTH; + this.readType = ReadType.DICTIONARY; + } + + private void allocateVectorBasedOnOriginalType(PrimitiveType primitive, Field arrowField) { + switch (primitive.getOriginalType()) { + case ENUM: + case JSON: + case UTF8: + case BSON: + this.vec = arrowField.createVector(rootAlloc); + // TODO: Possibly use the uncompressed page size info to set the initial capacity + vec.setInitialCapacity(batchSize * AVERAGE_VARIABLE_WIDTH_RECORD_SIZE); + vec.allocateNewSafe(); + this.readType = ReadType.VARCHAR; + this.typeWidth = UNKNOWN_WIDTH; + break; + case INT_8: + case INT_16: + case INT_32: + this.vec = arrowField.createVector(rootAlloc); + ((IntVector) vec).allocateNew(batchSize); + this.readType = ReadType.INT; + this.typeWidth = (int) IntVector.TYPE_WIDTH; + break; + case DATE: + this.vec = arrowField.createVector(rootAlloc); + ((DateDayVector) vec).allocateNew(batchSize); + this.readType = ReadType.INT; + this.typeWidth = (int) IntVector.TYPE_WIDTH; + break; + case INT_64: + this.vec = arrowField.createVector(rootAlloc); + ((BigIntVector) vec).allocateNew(batchSize); + this.readType = ReadType.LONG; + this.typeWidth = (int) BigIntVector.TYPE_WIDTH; + break; + case TIMESTAMP_MILLIS: + this.vec = arrowField.createVector(rootAlloc); + ((BigIntVector) vec).allocateNew(batchSize); + this.readType = ReadType.TIMESTAMP_MILLIS; + this.typeWidth = (int) BigIntVector.TYPE_WIDTH; + break; + case TIMESTAMP_MICROS: + this.vec = arrowField.createVector(rootAlloc); + if (((Types.TimestampType) icebergField.type()).shouldAdjustToUTC()) { + ((TimeStampMicroTZVector) vec).allocateNew(batchSize); + } else { + ((TimeStampMicroVector) vec).allocateNew(batchSize); + } + this.readType = ReadType.LONG; + this.typeWidth = (int) BigIntVector.TYPE_WIDTH; + break; + case TIME_MICROS: + this.vec = arrowField.createVector(rootAlloc); + ((TimeMicroVector) vec).allocateNew(batchSize); + this.readType = ReadType.LONG; + this.typeWidth = 8; + break; + case DECIMAL: + this.vec = arrowField.createVector(rootAlloc); + ((DecimalVector) vec).allocateNew(batchSize); switch (primitive.getPrimitiveTypeName()) { - case FIXED_LEN_BYTE_ARRAY: - int len = ((Types.FixedType) icebergField.type()).length(); - this.vec = arrowField.createVector(rootAlloc); - vec.setInitialCapacity(batchSize * len); - vec.allocateNew(); - this.readType = ReadType.FIXED_WIDTH_BINARY; - this.typeWidth = len; - break; case BINARY: - this.vec = arrowField.createVector(rootAlloc); - // TODO: Possibly use the uncompressed page size info to set the initial capacity - vec.setInitialCapacity(batchSize * AVERAGE_VARIABLE_WIDTH_RECORD_SIZE); - vec.allocateNewSafe(); - this.readType = ReadType.VARBINARY; - this.typeWidth = UNKNOWN_WIDTH; - break; - case INT32: - Field intField = new Field( - icebergField.name(), - new FieldType(icebergField.isOptional(), new ArrowType.Int(Integer.SIZE, true), - null, null), null); - this.vec = intField.createVector(rootAlloc); - ((IntVector) vec).allocateNew(batchSize); - this.readType = ReadType.INT; - this.typeWidth = (int) IntVector.TYPE_WIDTH; - break; - case FLOAT: - Field floatField = new Field( - icebergField.name(), - new FieldType(icebergField.isOptional(), new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE), - null, null), null); - this.vec = floatField.createVector(rootAlloc); - ((Float4Vector) vec).allocateNew(batchSize); - this.readType = ReadType.FLOAT; - this.typeWidth = (int) Float4Vector.TYPE_WIDTH; - break; - case BOOLEAN: - this.vec = arrowField.createVector(rootAlloc); - ((BitVector) vec).allocateNew(batchSize); - this.readType = ReadType.BOOLEAN; - this.typeWidth = UNKNOWN_WIDTH; + case FIXED_LEN_BYTE_ARRAY: + this.readType = ReadType.FIXED_LENGTH_DECIMAL; + this.typeWidth = primitive.getTypeLength(); break; case INT64: - this.vec = arrowField.createVector(rootAlloc); - ((BigIntVector) vec).allocateNew(batchSize); - this.readType = ReadType.LONG; + this.readType = ReadType.LONG_BACKED_DECIMAL; this.typeWidth = (int) BigIntVector.TYPE_WIDTH; break; - case DOUBLE: - this.vec = arrowField.createVector(rootAlloc); - ((Float8Vector) vec).allocateNew(batchSize); - this.readType = ReadType.DOUBLE; - this.typeWidth = (int) Float8Vector.TYPE_WIDTH; + case INT32: + this.readType = ReadType.INT_BACKED_DECIMAL; + this.typeWidth = (int) IntVector.TYPE_WIDTH; break; default: - throw new UnsupportedOperationException("Unsupported type: " + primitive); + throw new UnsupportedOperationException( + "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName()); } - } + break; + default: + throw new UnsupportedOperationException( + "Unsupported logical type: " + primitive.getOriginalType()); + } + } + + private void allocateVectorBasedOnTypeName(PrimitiveType primitive, Field arrowField) { + switch (primitive.getPrimitiveTypeName()) { + case FIXED_LEN_BYTE_ARRAY: + int len; + if (icebergField.type() instanceof Types.UUIDType) { + len = 16; + this.readType = ReadType.UUID; + } else { + len = ((Types.FixedType) icebergField.type()).length(); + this.readType = ReadType.FIXED_WIDTH_BINARY; + } + this.vec = arrowField.createVector(rootAlloc); + vec.setInitialCapacity(batchSize * len); + vec.allocateNew(); + this.typeWidth = len; + break; + case BINARY: + this.vec = arrowField.createVector(rootAlloc); + // TODO: Possibly use the uncompressed page size info to set the initial capacity + vec.setInitialCapacity(batchSize * AVERAGE_VARIABLE_WIDTH_RECORD_SIZE); + vec.allocateNewSafe(); + this.readType = ReadType.VARBINARY; + this.typeWidth = UNKNOWN_WIDTH; + break; + case INT32: + Field intField = new Field( + icebergField.name(), + new FieldType(icebergField.isOptional(), new ArrowType.Int(Integer.SIZE, true), + null, null), null); + this.vec = intField.createVector(rootAlloc); + ((IntVector) vec).allocateNew(batchSize); + this.readType = ReadType.INT; + this.typeWidth = (int) IntVector.TYPE_WIDTH; + break; + case FLOAT: + Field floatField = new Field( + icebergField.name(), + new FieldType(icebergField.isOptional(), new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE), + null, null), null); + this.vec = floatField.createVector(rootAlloc); + ((Float4Vector) vec).allocateNew(batchSize); + this.readType = ReadType.FLOAT; + this.typeWidth = (int) Float4Vector.TYPE_WIDTH; + break; + case BOOLEAN: + this.vec = arrowField.createVector(rootAlloc); + ((BitVector) vec).allocateNew(batchSize); + this.readType = ReadType.BOOLEAN; + this.typeWidth = UNKNOWN_WIDTH; + break; + case INT64: + this.vec = arrowField.createVector(rootAlloc); + ((BigIntVector) vec).allocateNew(batchSize); + this.readType = ReadType.LONG; + this.typeWidth = (int) BigIntVector.TYPE_WIDTH; + break; + case DOUBLE: + this.vec = arrowField.createVector(rootAlloc); + ((Float8Vector) vec).allocateNew(batchSize); + this.readType = ReadType.DOUBLE; + this.typeWidth = (int) Float8Vector.TYPE_WIDTH; + break; + default: + throw new UnsupportedOperationException("Unsupported type: " + primitive); } } @@ -397,7 +424,7 @@ public VectorHolder read(VectorHolder reuse, int numValsToRead) { vec.getDataBuffer().setLong(i * Long.BYTES, rowStart + i); } for (int i = 0; i < numValsToRead; i += 1) { - BitVectorHelper.setValidityBitToOne(vec.getValidityBuffer(), i); + BitVectorHelper.setBit(vec.getValidityBuffer(), i); } nulls = new NullabilityHolder(numValsToRead); } diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedReaderBuilder.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedReaderBuilder.java index 77bd08952ea9..2bace1e5d53d 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedReaderBuilder.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedReaderBuilder.java @@ -86,6 +86,8 @@ public VectorizedReader message( reorderedFields.add(new VectorizedArrowReader.ConstantVectorReader<>(idToConstant.get(id))); } else if (id == MetadataColumns.ROW_POSITION.fieldId()) { reorderedFields.add(VectorizedArrowReader.positions()); + } else if (id == MetadataColumns.IS_DELETED.fieldId()) { + reorderedFields.add(new VectorizedArrowReader.ConstantVectorReader<>(false)); } else if (reader != null) { reorderedFields.add(reader); } else { diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/BaseVectorizedParquetValuesReader.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/BaseVectorizedParquetValuesReader.java index 60799d017ccc..5ef6efaba1c9 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/BaseVectorizedParquetValuesReader.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/BaseVectorizedParquetValuesReader.java @@ -80,12 +80,14 @@ public BaseVectorizedParquetValuesReader(int maxDefLevel, boolean setValidityVec this.setArrowValidityVector = setValidityVector; } - public BaseVectorizedParquetValuesReader( - int bitWidth, - int maxDefLevel, - boolean setValidityVector) { + public BaseVectorizedParquetValuesReader(int bitWidth, int maxDefLevel, boolean setValidityVector) { + this(bitWidth, maxDefLevel, bitWidth != 0, setValidityVector); + } + + public BaseVectorizedParquetValuesReader(int bitWidth, int maxDefLevel, boolean readLength, + boolean setValidityVector) { this.fixedWidth = true; - this.readLength = bitWidth != 0; + this.readLength = readLength; this.maxDefLevel = maxDefLevel; this.setArrowValidityVector = setValidityVector; init(bitWidth); diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedColumnIterator.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedColumnIterator.java index d963c454d35e..c87a80e13469 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedColumnIterator.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedColumnIterator.java @@ -57,168 +57,211 @@ public Dictionary setRowGroupInfo(PageReader store, boolean allPagesDictEncoded) return dictionary; } - public void nextBatchIntegers(FieldVector fieldVector, int typeWidth, NullabilityHolder holder) { - int rowsReadSoFar = 0; - while (rowsReadSoFar < batchSize && hasNext()) { - advance(); - int rowsInThisBatch = vectorizedPageIterator.nextBatchIntegers(fieldVector, batchSize - rowsReadSoFar, - rowsReadSoFar, typeWidth, holder); - rowsReadSoFar += rowsInThisBatch; - this.triplesRead += rowsInThisBatch; - fieldVector.setValueCount(rowsReadSoFar); + @Override + protected BasePageIterator pageIterator() { + return vectorizedPageIterator; + } + + public boolean producesDictionaryEncodedVector() { + return vectorizedPageIterator.producesDictionaryEncodedVector(); + } + + public abstract class BatchReader { + public void nextBatch(FieldVector fieldVector, int typeWidth, NullabilityHolder holder) { + int rowsReadSoFar = 0; + while (rowsReadSoFar < batchSize && hasNext()) { + advance(); + int rowsInThisBatch = nextBatchOf(fieldVector, batchSize - rowsReadSoFar, + rowsReadSoFar, typeWidth, holder); + rowsReadSoFar += rowsInThisBatch; + triplesRead += rowsInThisBatch; + fieldVector.setValueCount(rowsReadSoFar); + } } + + protected abstract int nextBatchOf( + FieldVector vector, int expectedBatchSize, int numValsInVector, int typeWidth, NullabilityHolder holder); } - public void nextBatchDictionaryIds(IntVector vector, NullabilityHolder holder) { - int rowsReadSoFar = 0; - while (rowsReadSoFar < batchSize && hasNext()) { - advance(); - int rowsInThisBatch = vectorizedPageIterator.nextBatchDictionaryIds(vector, batchSize - rowsReadSoFar, - rowsReadSoFar, holder); - rowsReadSoFar += rowsInThisBatch; - this.triplesRead += rowsInThisBatch; - vector.setValueCount(rowsReadSoFar); + public class IntegerBatchReader extends BatchReader { + @Override + protected int nextBatchOf( + final FieldVector vector, final int expectedBatchSize, final int numValsInVector, final int typeWidth, + NullabilityHolder holder) { + return vectorizedPageIterator.intPageReader() + .nextBatch(vector, expectedBatchSize, numValsInVector, typeWidth, holder); } } - public void nextBatchLongs(FieldVector fieldVector, int typeWidth, NullabilityHolder holder) { - int rowsReadSoFar = 0; - while (rowsReadSoFar < batchSize && hasNext()) { - advance(); - int rowsInThisBatch = vectorizedPageIterator.nextBatchLongs(fieldVector, batchSize - rowsReadSoFar, - rowsReadSoFar, typeWidth, holder); - rowsReadSoFar += rowsInThisBatch; - this.triplesRead += rowsInThisBatch; - fieldVector.setValueCount(rowsReadSoFar); + public class DictionaryBatchReader extends BatchReader { + @Override + protected int nextBatchOf( + final FieldVector vector, final int expectedBatchSize, final int numValsInVector, final int typeWidth, + NullabilityHolder holder) { + return vectorizedPageIterator.nextBatchDictionaryIds((IntVector) vector, expectedBatchSize, numValsInVector, + holder); } } - public void nextBatchTimestampMillis(FieldVector fieldVector, int typeWidth, NullabilityHolder holder) { - int rowsReadSoFar = 0; - while (rowsReadSoFar < batchSize && hasNext()) { - advance(); - int rowsInThisBatch = vectorizedPageIterator.nextBatchTimestampMillis(fieldVector, batchSize - rowsReadSoFar, - rowsReadSoFar, typeWidth, holder); - rowsReadSoFar += rowsInThisBatch; - this.triplesRead += rowsInThisBatch; - fieldVector.setValueCount(rowsReadSoFar); + public class LongBatchReader extends BatchReader { + @Override + protected int nextBatchOf( + final FieldVector vector, final int expectedBatchSize, final int numValsInVector, final int typeWidth, + NullabilityHolder holder) { + return vectorizedPageIterator.longPageReader() + .nextBatch(vector, expectedBatchSize, numValsInVector, typeWidth, holder); } } - public void nextBatchFloats(FieldVector fieldVector, int typeWidth, NullabilityHolder holder) { - int rowsReadSoFar = 0; - while (rowsReadSoFar < batchSize && hasNext()) { - advance(); - int rowsInThisBatch = vectorizedPageIterator.nextBatchFloats(fieldVector, batchSize - rowsReadSoFar, - rowsReadSoFar, typeWidth, holder); - rowsReadSoFar += rowsInThisBatch; - this.triplesRead += rowsInThisBatch; - fieldVector.setValueCount(rowsReadSoFar); + public class TimestampMillisBatchReader extends BatchReader { + @Override + protected int nextBatchOf( + final FieldVector vector, final int expectedBatchSize, final int numValsInVector, final int typeWidth, + NullabilityHolder holder) { + return vectorizedPageIterator.timestampMillisPageReader().nextBatch(vector, expectedBatchSize, numValsInVector, + typeWidth, holder); } } - public void nextBatchDoubles(FieldVector fieldVector, int typeWidth, NullabilityHolder holder) { - int rowsReadSoFar = 0; - while (rowsReadSoFar < batchSize && hasNext()) { - advance(); - int rowsInThisBatch = vectorizedPageIterator.nextBatchDoubles(fieldVector, batchSize - rowsReadSoFar, - rowsReadSoFar, typeWidth, holder); - rowsReadSoFar += rowsInThisBatch; - this.triplesRead += rowsInThisBatch; - fieldVector.setValueCount(rowsReadSoFar); + public class FloatBatchReader extends BatchReader { + @Override + protected int nextBatchOf( + final FieldVector vector, final int expectedBatchSize, final int numValsInVector, final int typeWidth, + NullabilityHolder holder) { + return vectorizedPageIterator.floatPageReader() + .nextBatch(vector, expectedBatchSize, numValsInVector, typeWidth, holder); } } - public void nextBatchIntBackedDecimal( - FieldVector fieldVector, - NullabilityHolder nullabilityHolder) { - int rowsReadSoFar = 0; - while (rowsReadSoFar < batchSize && hasNext()) { - advance(); - int rowsInThisBatch = - vectorizedPageIterator.nextBatchIntBackedDecimal(fieldVector, batchSize - rowsReadSoFar, - rowsReadSoFar, nullabilityHolder); - rowsReadSoFar += rowsInThisBatch; - this.triplesRead += rowsInThisBatch; - fieldVector.setValueCount(rowsReadSoFar); + public class DoubleBatchReader extends BatchReader { + @Override + protected int nextBatchOf( + final FieldVector vector, final int expectedBatchSize, final int numValsInVector, final int typeWidth, + NullabilityHolder holder) { + return vectorizedPageIterator.doublePageReader() + .nextBatch(vector, expectedBatchSize, numValsInVector, typeWidth, holder); } } - public void nextBatchLongBackedDecimal( - FieldVector fieldVector, - NullabilityHolder nullabilityHolder) { - int rowsReadSoFar = 0; - while (rowsReadSoFar < batchSize && hasNext()) { - advance(); - int rowsInThisBatch = - vectorizedPageIterator.nextBatchLongBackedDecimal(fieldVector, batchSize - rowsReadSoFar, - rowsReadSoFar, nullabilityHolder); - rowsReadSoFar += rowsInThisBatch; - this.triplesRead += rowsInThisBatch; - fieldVector.setValueCount(rowsReadSoFar); + public class IntBackedDecimalBatchReader extends BatchReader { + @Override + protected int nextBatchOf( + final FieldVector vector, final int expectedBatchSize, final int numValsInVector, final int typeWidth, + NullabilityHolder holder) { + return vectorizedPageIterator.intBackedDecimalPageReader() + .nextBatch(vector, expectedBatchSize, numValsInVector, typeWidth, holder); } } - public void nextBatchFixedLengthDecimal( - FieldVector fieldVector, - int typeWidth, - NullabilityHolder nullabilityHolder) { - int rowsReadSoFar = 0; - while (rowsReadSoFar < batchSize && hasNext()) { - advance(); - int rowsInThisBatch = - vectorizedPageIterator.nextBatchFixedLengthDecimal(fieldVector, batchSize - rowsReadSoFar, - rowsReadSoFar, typeWidth, nullabilityHolder); - rowsReadSoFar += rowsInThisBatch; - this.triplesRead += rowsInThisBatch; - fieldVector.setValueCount(rowsReadSoFar); + public class LongBackedDecimalBatchReader extends BatchReader { + @Override + protected int nextBatchOf( + final FieldVector vector, final int expectedBatchSize, final int numValsInVector, final int typeWidth, + NullabilityHolder holder) { + return vectorizedPageIterator.longBackedDecimalPageReader().nextBatch(vector, expectedBatchSize, numValsInVector, + typeWidth, holder); } } - public void nextBatchVarWidthType(FieldVector fieldVector, NullabilityHolder nullabilityHolder) { - int rowsReadSoFar = 0; - while (rowsReadSoFar < batchSize && hasNext()) { - advance(); - int rowsInThisBatch = vectorizedPageIterator.nextBatchVarWidthType(fieldVector, batchSize - rowsReadSoFar, - rowsReadSoFar, nullabilityHolder); - rowsReadSoFar += rowsInThisBatch; - this.triplesRead += rowsInThisBatch; - fieldVector.setValueCount(rowsReadSoFar); + public class FixedLengthDecimalBatchReader extends BatchReader { + @Override + protected int nextBatchOf( + final FieldVector vector, final int expectedBatchSize, final int numValsInVector, final int typeWidth, + NullabilityHolder holder) { + return vectorizedPageIterator.fixedLengthDecimalPageReader().nextBatch(vector, expectedBatchSize, numValsInVector, + typeWidth, holder); } } - public void nextBatchFixedWidthBinary(FieldVector fieldVector, int typeWidth, NullabilityHolder nullabilityHolder) { - int rowsReadSoFar = 0; - while (rowsReadSoFar < batchSize && hasNext()) { - advance(); - int rowsInThisBatch = - vectorizedPageIterator.nextBatchFixedWidthBinary(fieldVector, batchSize - rowsReadSoFar, - rowsReadSoFar, typeWidth, nullabilityHolder); - rowsReadSoFar += rowsInThisBatch; - this.triplesRead += rowsInThisBatch; - fieldVector.setValueCount(rowsReadSoFar); + public class FixedSizeBinaryBatchReader extends BatchReader { + @Override + protected int nextBatchOf( + final FieldVector vector, final int expectedBatchSize, final int numValsInVector, final int typeWidth, + NullabilityHolder holder) { + return vectorizedPageIterator.fixedSizeBinaryPageReader().nextBatch(vector, expectedBatchSize, numValsInVector, + typeWidth, holder); } } - public void nextBatchBoolean(FieldVector fieldVector, NullabilityHolder nullabilityHolder) { - int rowsReadSoFar = 0; - while (rowsReadSoFar < batchSize && hasNext()) { - advance(); - int rowsInThisBatch = vectorizedPageIterator.nextBatchBoolean(fieldVector, batchSize - rowsReadSoFar, - rowsReadSoFar, nullabilityHolder); - rowsReadSoFar += rowsInThisBatch; - this.triplesRead += rowsInThisBatch; - fieldVector.setValueCount(rowsReadSoFar); + public class VarWidthTypeBatchReader extends BatchReader { + @Override + protected int nextBatchOf( + final FieldVector vector, final int expectedBatchSize, final int numValsInVector, final int typeWidth, + NullabilityHolder holder) { + return vectorizedPageIterator.varWidthTypePageReader().nextBatch(vector, expectedBatchSize, numValsInVector, + typeWidth, holder); } } - @Override - protected BasePageIterator pageIterator() { - return vectorizedPageIterator; + public class FixedWidthTypeBinaryBatchReader extends BatchReader { + @Override + protected int nextBatchOf( + final FieldVector vector, final int expectedBatchSize, final int numValsInVector, final int typeWidth, + NullabilityHolder holder) { + return vectorizedPageIterator.fixedWidthBinaryPageReader().nextBatch(vector, expectedBatchSize, numValsInVector, + typeWidth, holder); + } } - public boolean producesDictionaryEncodedVector() { - return vectorizedPageIterator.producesDictionaryEncodedVector(); + public class BooleanBatchReader extends BatchReader { + @Override + protected int nextBatchOf( + final FieldVector vector, final int expectedBatchSize, final int numValsInVector, final int typeWidth, + NullabilityHolder holder) { + return vectorizedPageIterator.booleanPageReader().nextBatch(vector, expectedBatchSize, numValsInVector, + typeWidth, holder); + } + } + + public IntegerBatchReader integerBatchReader() { + return new IntegerBatchReader(); + } + + public DictionaryBatchReader dictionaryBatchReader() { + return new DictionaryBatchReader(); } + public LongBatchReader longBatchReader() { + return new LongBatchReader(); + } + + public TimestampMillisBatchReader timestampMillisBatchReader() { + return new TimestampMillisBatchReader(); + } + + public FloatBatchReader floatBatchReader() { + return new FloatBatchReader(); + } + + public DoubleBatchReader doubleBatchReader() { + return new DoubleBatchReader(); + } + + public IntBackedDecimalBatchReader intBackedDecimalBatchReader() { + return new IntBackedDecimalBatchReader(); + } + + public LongBackedDecimalBatchReader longBackedDecimalBatchReader() { + return new LongBackedDecimalBatchReader(); + } + + public FixedLengthDecimalBatchReader fixedLengthDecimalBatchReader() { + return new FixedLengthDecimalBatchReader(); + } + + public FixedSizeBinaryBatchReader fixedSizeBinaryBatchReader() { + return new FixedSizeBinaryBatchReader(); + } + + public VarWidthTypeBatchReader varWidthTypeBatchReader() { + return new VarWidthTypeBatchReader(); + } + + public FixedWidthTypeBinaryBatchReader fixedWidthTypeBinaryBatchReader() { + return new FixedWidthTypeBinaryBatchReader(); + } + + public BooleanBatchReader booleanBatchReader() { + return new BooleanBatchReader(); + } } diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedDictionaryEncodedParquetValuesReader.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedDictionaryEncodedParquetValuesReader.java index 74d9e1527985..1e694fc47246 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedDictionaryEncodedParquetValuesReader.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedDictionaryEncodedParquetValuesReader.java @@ -24,6 +24,7 @@ import org.apache.arrow.vector.BitVectorHelper; import org.apache.arrow.vector.DecimalVector; import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.FixedSizeBinaryVector; import org.apache.arrow.vector.IntVector; import org.apache.iceberg.arrow.vectorized.NullabilityHolder; import org.apache.parquet.column.Dictionary; @@ -39,375 +40,183 @@ public VectorizedDictionaryEncodedParquetValuesReader(int maxDefLevel, boolean s super(maxDefLevel, setValidityVector); } - void readBatchOfDictionaryIds(IntVector intVector, int startOffset, int numValuesToRead, - NullabilityHolder nullabilityHolder) { - int left = numValuesToRead; - int idx = startOffset; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); + abstract class BaseDictEncodedReader { + public void nextBatch( + FieldVector vector, int startOffset, int numValuesToRead, Dictionary dict, + NullabilityHolder nullabilityHolder, int typeWidth) { + int left = numValuesToRead; + int idx = startOffset; + while (left > 0) { + if (currentCount == 0) { + readNextGroup(); + } + int numValues = Math.min(left, currentCount); + for (int i = 0; i < numValues; i++) { + int index = idx * typeWidth; + if (typeWidth == -1) { + index = idx; + } + if (Mode.RLE.equals(mode)) { + nextVal(vector, dict, index, currentValue, typeWidth); + } else if (Mode.PACKED.equals(mode)) { + nextVal(vector, dict, index, packedValuesBuffer[packedValuesBufferIdx++], typeWidth); + } + nullabilityHolder.setNotNull(idx); + if (setArrowValidityVector) { + BitVectorHelper.setBit(vector.getValidityBuffer(), idx); + } + idx++; + } + left -= numValues; + currentCount -= numValues; } - int numValues = Math.min(left, this.currentCount); - switch (mode) { - case RLE: - for (int i = 0; i < numValues; i++) { - intVector.set(idx, currentValue); - setNotNull(intVector, nullabilityHolder, idx); - idx++; - } - break; - case PACKED: - for (int i = 0; i < numValues; i++) { - intVector.set(idx, packedValuesBuffer[packedValuesBufferIdx++]); - setNotNull(intVector, nullabilityHolder, idx); - idx++; - } - break; - } - left -= numValues; - currentCount -= numValues; } + + protected abstract void nextVal(FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth); } - void readBatchOfDictionaryEncodedLongs(FieldVector vector, int startOffset, int numValuesToRead, Dictionary dict, - NullabilityHolder nullabilityHolder, int typeWidth) { - int left = numValuesToRead; - int idx = startOffset; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int numValues = Math.min(left, this.currentCount); - switch (mode) { - case RLE: - for (int i = 0; i < numValues; i++) { - vector.getDataBuffer().setLong(idx * typeWidth, dict.decodeToLong(currentValue)); - setNotNull(vector, nullabilityHolder, idx); - idx++; - } - break; - case PACKED: - for (int i = 0; i < numValues; i++) { - vector.getDataBuffer() - .setLong(idx * typeWidth, dict.decodeToLong(packedValuesBuffer[packedValuesBufferIdx++])); - setNotNull(vector, nullabilityHolder, idx); - idx++; - } - break; - } - left -= numValues; - currentCount -= numValues; + class DictionaryIdReader extends BaseDictEncodedReader { + @Override + protected void nextVal(FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { + ((IntVector) vector).set(idx, currentVal); } } - void readBatchOfDictionaryEncodedTimestampMillis( - FieldVector vector, int startOffset, int numValuesToRead, - Dictionary dict, NullabilityHolder nullabilityHolder, int typeWidth) { - int left = numValuesToRead; - int idx = startOffset; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int numValues = Math.min(left, this.currentCount); - switch (mode) { - case RLE: - for (int i = 0; i < numValues; i++) { - vector.getDataBuffer().setLong(idx * typeWidth, dict.decodeToLong(currentValue) * 1000); - setNotNull(vector, nullabilityHolder, idx); - idx++; - } - break; - case PACKED: - for (int i = 0; i < numValues; i++) { - vector.getDataBuffer() - .setLong(idx * typeWidth, dict.decodeToLong(packedValuesBuffer[packedValuesBufferIdx++]) * 1000); - setNotNull(vector, nullabilityHolder, idx); - idx++; - } - break; - } - left -= numValues; - currentCount -= numValues; + class LongDictEncodedReader extends BaseDictEncodedReader { + @Override + protected void nextVal(FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { + vector.getDataBuffer().setLong(idx, dict.decodeToLong(currentVal)); } } - void readBatchOfDictionaryEncodedIntegers(FieldVector vector, int startOffset, int numValuesToRead, Dictionary dict, - NullabilityHolder nullabilityHolder, int typeWidth) { - int left = numValuesToRead; - int idx = startOffset; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int num = Math.min(left, this.currentCount); - switch (mode) { - case RLE: - for (int i = 0; i < num; i++) { - vector.getDataBuffer().setInt(idx * typeWidth, dict.decodeToInt(currentValue)); - setNotNull(vector, nullabilityHolder, idx); - idx++; - } - break; - case PACKED: - for (int i = 0; i < num; i++) { - vector.getDataBuffer() - .setInt(idx * typeWidth, dict.decodeToInt(packedValuesBuffer[packedValuesBufferIdx++])); - setNotNull(vector, nullabilityHolder, idx); - idx++; - } - break; - } - left -= num; - currentCount -= num; + class TimestampMillisDictEncodedReader extends BaseDictEncodedReader { + @Override + protected void nextVal(FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { + vector.getDataBuffer().setLong(idx, dict.decodeToLong(currentVal) * 1000); } } - void readBatchOfDictionaryEncodedFloats(FieldVector vector, int startOffset, int numValuesToRead, Dictionary dict, - NullabilityHolder nullabilityHolder, int typeWidth) { - int left = numValuesToRead; - int idx = startOffset; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int num = Math.min(left, this.currentCount); - switch (mode) { - case RLE: - for (int i = 0; i < num; i++) { - vector.getDataBuffer().setFloat(idx * typeWidth, dict.decodeToFloat(currentValue)); - setNotNull(vector, nullabilityHolder, idx); - idx++; - } - break; - case PACKED: - for (int i = 0; i < num; i++) { - vector.getDataBuffer() - .setFloat(idx * typeWidth, dict.decodeToFloat(packedValuesBuffer[packedValuesBufferIdx++])); - setNotNull(vector, nullabilityHolder, idx); - idx++; - } - break; - } - left -= num; - currentCount -= num; + class IntegerDictEncodedReader extends BaseDictEncodedReader { + @Override + protected void nextVal(FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { + vector.getDataBuffer().setInt(idx, dict.decodeToInt(currentVal)); } } - void readBatchOfDictionaryEncodedDoubles(FieldVector vector, int startOffset, int numValuesToRead, Dictionary dict, - NullabilityHolder nullabilityHolder, int typeWidth) { - int left = numValuesToRead; - int idx = startOffset; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int num = Math.min(left, this.currentCount); - switch (mode) { - case RLE: - for (int i = 0; i < num; i++) { - vector.getDataBuffer().setDouble(idx * typeWidth, dict.decodeToDouble(currentValue)); - setNotNull(vector, nullabilityHolder, idx); - idx++; - } - break; - case PACKED: - for (int i = 0; i < num; i++) { - vector.getDataBuffer() - .setDouble(idx * typeWidth, dict.decodeToDouble(packedValuesBuffer[packedValuesBufferIdx++])); - setNotNull(vector, nullabilityHolder, idx); - idx++; - } - break; - } - left -= num; - currentCount -= num; + class FloatDictEncodedReader extends BaseDictEncodedReader { + @Override + protected void nextVal(FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { + vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(currentVal)); } } - void readBatchOfDictionaryEncodedFixedWidthBinary(FieldVector vector, int typeWidth, int startOffset, - int numValuesToRead, Dictionary dict, - NullabilityHolder nullabilityHolder) { - int left = numValuesToRead; - int idx = startOffset; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int num = Math.min(left, this.currentCount); - switch (mode) { - case RLE: - for (int i = 0; i < num; i++) { - ByteBuffer buffer = dict.decodeToBinary(currentValue).toByteBuffer(); - setFixedWidthBinary(vector, typeWidth, nullabilityHolder, idx, buffer); - idx++; - } - break; - case PACKED: - for (int i = 0; i < num; i++) { - ByteBuffer buffer = dict.decodeToBinary(packedValuesBuffer[packedValuesBufferIdx++]).toByteBuffer(); - setFixedWidthBinary(vector, typeWidth, nullabilityHolder, idx, buffer); - idx++; - } - break; - } - left -= num; - currentCount -= num; + class DoubleDictEncodedReader extends BaseDictEncodedReader { + @Override + protected void nextVal(FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { + vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(currentVal)); } } - private void setFixedWidthBinary( - FieldVector vector, int typeWidth, NullabilityHolder nullabilityHolder, - int idx, ByteBuffer buffer) { - vector.getDataBuffer() - .setBytes(idx * typeWidth, buffer.array(), - buffer.position() + buffer.arrayOffset(), buffer.limit() - buffer.position()); - setNotNull(vector, nullabilityHolder, idx); + class FixedWidthBinaryDictEncodedReader extends BaseDictEncodedReader { + @Override + protected void nextVal(FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { + ByteBuffer buffer = dict.decodeToBinary(currentVal).toByteBuffer(); + vector.getDataBuffer() + .setBytes(idx, buffer.array(), buffer.position() + buffer.arrayOffset(), buffer.limit() - buffer.position()); + } } - private void setNotNull(FieldVector vector, NullabilityHolder nullabilityHolder, int idx) { - nullabilityHolder.setNotNull(idx); - if (setArrowValidityVector) { - BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx); + class FixedLengthDecimalDictEncodedReader extends BaseDictEncodedReader { + @Override + protected void nextVal(FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { + byte[] decimalBytes = dict.decodeToBinary(currentVal).getBytesUnsafe(); + byte[] vectorBytes = new byte[typeWidth]; + System.arraycopy(decimalBytes, 0, vectorBytes, 0, typeWidth); + ((DecimalVector) vector).setBigEndian(idx, vectorBytes); + ByteBuffer buffer = dict.decodeToBinary(currentVal).toByteBuffer(); + vector.getDataBuffer() + .setBytes(idx, buffer.array(), buffer.position() + buffer.arrayOffset(), buffer.limit() - buffer.position()); } } - void readBatchOfDictionaryEncodedFixedLengthDecimals(FieldVector vector, int typeWidth, int startOffset, - int numValuesToRead, Dictionary dict, - NullabilityHolder nullabilityHolder) { - int left = numValuesToRead; - int idx = startOffset; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int num = Math.min(left, this.currentCount); - switch (mode) { - case RLE: - for (int i = 0; i < num; i++) { - byte[] decimalBytes = dict.decodeToBinary(currentValue).getBytesUnsafe(); - byte[] vectorBytes = new byte[typeWidth]; - System.arraycopy(decimalBytes, 0, vectorBytes, 0, typeWidth); - ((DecimalVector) vector).setBigEndian(idx, vectorBytes); - nullabilityHolder.setNotNull(idx); - idx++; - } - break; - case PACKED: - for (int i = 0; i < num; i++) { - byte[] decimalBytes = dict.decodeToBinary(packedValuesBuffer[packedValuesBufferIdx++]).getBytesUnsafe(); - byte[] vectorBytes = new byte[typeWidth]; - System.arraycopy(decimalBytes, 0, vectorBytes, 0, typeWidth); - ((DecimalVector) vector).setBigEndian(idx, vectorBytes); - nullabilityHolder.setNotNull(idx); - idx++; - } - break; - } - left -= num; - currentCount -= num; + class VarWidthBinaryDictEncodedReader extends BaseDictEncodedReader { + @Override + protected void nextVal(FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { + ByteBuffer buffer = dict.decodeToBinary(currentVal).toByteBuffer(); + ((BaseVariableWidthVector) vector).setSafe(idx, buffer.array(), + buffer.position() + buffer.arrayOffset(), buffer.limit() - buffer.position()); } } - void readBatchOfDictionaryEncodedVarWidthBinary(FieldVector vector, int startOffset, int numValuesToRead, - Dictionary dict, NullabilityHolder nullabilityHolder) { - int left = numValuesToRead; - int idx = startOffset; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int num = Math.min(left, this.currentCount); - switch (mode) { - case RLE: - for (int i = 0; i < num; i++) { - ByteBuffer buffer = dict.decodeToBinary(currentValue).toByteBuffer(); - ((BaseVariableWidthVector) vector).setSafe(idx, buffer.array(), - buffer.position() + buffer.arrayOffset(), buffer.limit() - buffer.position()); - nullabilityHolder.setNotNull(idx); - idx++; - } - break; - case PACKED: - for (int i = 0; i < num; i++) { - ByteBuffer buffer = dict.decodeToBinary(packedValuesBuffer[packedValuesBufferIdx++]).toByteBuffer(); - ((BaseVariableWidthVector) vector).setSafe(idx, buffer.array(), - buffer.position() + buffer.arrayOffset(), buffer.limit() - buffer.position()); - nullabilityHolder.setNotNull(idx); - idx++; - } - break; - } - left -= num; - currentCount -= num; + class IntBackedDecimalDictEncodedReader extends BaseDictEncodedReader { + @Override + protected void nextVal(FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { + ((DecimalVector) vector).set(idx, dict.decodeToInt(currentVal)); } } - void readBatchOfDictionaryEncodedIntBackedDecimals(FieldVector vector, int startOffset, - int numValuesToRead, Dictionary dict, - NullabilityHolder nullabilityHolder) { - int left = numValuesToRead; - int idx = startOffset; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int num = Math.min(left, this.currentCount); - switch (mode) { - case RLE: - for (int i = 0; i < num; i++) { - ((DecimalVector) vector).set( - idx, - dict.decodeToInt(currentValue)); - nullabilityHolder.setNotNull(idx); - idx++; - } - break; - case PACKED: - for (int i = 0; i < num; i++) { - ((DecimalVector) vector).set( - idx, dict.decodeToInt(packedValuesBuffer[packedValuesBufferIdx++])); - nullabilityHolder.setNotNull(idx); - idx++; - } - break; - } - left -= num; - currentCount -= num; + class LongBackedDecimalDictEncodedReader extends BaseDictEncodedReader { + @Override + protected void nextVal(FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { + ((DecimalVector) vector).set(idx, dict.decodeToLong(currentVal)); } } - void readBatchOfDictionaryEncodedLongBackedDecimals(FieldVector vector, int startOffset, - int numValuesToRead, Dictionary dict, - NullabilityHolder nullabilityHolder) { - int left = numValuesToRead; - int idx = startOffset; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int num = Math.min(left, this.currentCount); - switch (mode) { - case RLE: - for (int i = 0; i < num; i++) { - ((DecimalVector) vector).set( - idx, - dict.decodeToLong(currentValue)); - nullabilityHolder.setNotNull(idx); - idx++; - } - break; - case PACKED: - for (int i = 0; i < num; i++) { - ((DecimalVector) vector).set( - idx, dict.decodeToLong(packedValuesBuffer[packedValuesBufferIdx++])); - nullabilityHolder.setNotNull(idx); - idx++; - } - break; - } - left -= num; - currentCount -= num; + class FixedSizeBinaryDictEncodedReader extends BaseDictEncodedReader { + @Override + protected void nextVal(FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { + byte[] bytes = dict.decodeToBinary(currentVal).getBytesUnsafe(); + byte[] vectorBytes = new byte[typeWidth]; + System.arraycopy(bytes, 0, vectorBytes, 0, typeWidth); + ((FixedSizeBinaryVector) vector).set(idx, vectorBytes); } } + + public DictionaryIdReader dictionaryIdReader() { + return new DictionaryIdReader(); + } + + public LongDictEncodedReader longDictEncodedReader() { + return new LongDictEncodedReader(); + } + + public TimestampMillisDictEncodedReader timestampMillisDictEncodedReader() { + return new TimestampMillisDictEncodedReader(); + } + + public IntegerDictEncodedReader integerDictEncodedReader() { + return new IntegerDictEncodedReader(); + } + + public FloatDictEncodedReader floatDictEncodedReader() { + return new FloatDictEncodedReader(); + } + + public DoubleDictEncodedReader doubleDictEncodedReader() { + return new DoubleDictEncodedReader(); + } + + public FixedWidthBinaryDictEncodedReader fixedWidthBinaryDictEncodedReader() { + return new FixedWidthBinaryDictEncodedReader(); + } + + public FixedLengthDecimalDictEncodedReader fixedLengthDecimalDictEncodedReader() { + return new FixedLengthDecimalDictEncodedReader(); + } + + public VarWidthBinaryDictEncodedReader varWidthBinaryDictEncodedReader() { + return new VarWidthBinaryDictEncodedReader(); + } + + public IntBackedDecimalDictEncodedReader intBackedDecimalDictEncodedReader() { + return new IntBackedDecimalDictEncodedReader(); + } + + public LongBackedDecimalDictEncodedReader longBackedDecimalDictEncodedReader() { + return new LongBackedDecimalDictEncodedReader(); + } + + public FixedSizeBinaryDictEncodedReader fixedSizeBinaryDictEncodedReader() { + return new FixedSizeBinaryDictEncodedReader(); + } } diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPageIterator.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPageIterator.java index 987696235f69..5cce5286b79f 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPageIterator.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPageIterator.java @@ -71,95 +71,141 @@ protected void reset() { this.vectorizedDefinitionLevelReader = null; } + @Override + protected void initDataReader(Encoding dataEncoding, ByteBufferInputStream in, int valueCount) { + ValuesReader previousReader = plainValuesReader; + if (dataEncoding.usesDictionary()) { + if (dictionary == null) { + throw new ParquetDecodingException( + "could not read page in col " + desc + " as the dictionary was missing for encoding " + dataEncoding); + } + try { + dictionaryEncodedValuesReader = + new VectorizedDictionaryEncodedParquetValuesReader(desc.getMaxDefinitionLevel(), setArrowValidityVector); + dictionaryEncodedValuesReader.initFromPage(valueCount, in); + if (ParquetUtil.isIntType(desc.getPrimitiveType()) || !allPagesDictEncoded) { + dictionaryDecodeMode = DictionaryDecodeMode.EAGER; + } else { + dictionaryDecodeMode = DictionaryDecodeMode.LAZY; + } + } catch (IOException e) { + throw new ParquetDecodingException("could not read page in col " + desc, e); + } + } else { + if (dataEncoding != Encoding.PLAIN) { + throw new UnsupportedOperationException("Cannot support vectorized reads for column " + desc + " with " + + "encoding " + dataEncoding + ". Disable vectorized reads to read this table/file"); + } + plainValuesReader = new ValuesAsBytesReader(); + plainValuesReader.initFromPage(valueCount, in); + dictionaryDecodeMode = DictionaryDecodeMode.NONE; + } + if (CorruptDeltaByteArrays.requiresSequentialReads(writerVersion, dataEncoding) && + previousReader instanceof RequiresPreviousReader) { + // previous reader can only be set if reading sequentially + ((RequiresPreviousReader) plainValuesReader).setPreviousReader(previousReader); + } + } + + public boolean producesDictionaryEncodedVector() { + return dictionaryDecodeMode == DictionaryDecodeMode.LAZY; + } + + @Override + protected void initDefinitionLevelsReader( + DataPageV1 dataPageV1, ColumnDescriptor desc, ByteBufferInputStream in, + int triplesCount) throws IOException { + int bitWidth = BytesUtils.getWidthFromMaxInt(desc.getMaxDefinitionLevel()); + this.vectorizedDefinitionLevelReader = new VectorizedParquetDefinitionLevelReader(bitWidth, + desc.getMaxDefinitionLevel(), setArrowValidityVector); + this.vectorizedDefinitionLevelReader.initFromPage(triplesCount, in); + } + + @Override + protected void initDefinitionLevelsReader(DataPageV2 dataPageV2, ColumnDescriptor desc) throws IOException { + int bitWidth = BytesUtils.getWidthFromMaxInt(desc.getMaxDefinitionLevel()); + // do not read the length from the stream. v2 pages handle dividing the page bytes. + this.vectorizedDefinitionLevelReader = new VectorizedParquetDefinitionLevelReader(bitWidth, + desc.getMaxDefinitionLevel(), false, setArrowValidityVector); + this.vectorizedDefinitionLevelReader.initFromPage( + dataPageV2.getValueCount(), dataPageV2.getDefinitionLevels().toInputStream()); + } + /** - * Method for reading a batch of dictionary ids from the dicitonary encoded data pages. Like definition levels, + * Method for reading a batch of dictionary ids from the dictionary encoded data pages. Like definition levels, * dictionary ids in Parquet are RLE/bin-packed encoded as well. */ public int nextBatchDictionaryIds( - final IntVector vector, final int expectedBatchSize, - final int numValsInVector, - NullabilityHolder holder) { + final IntVector vector, final int expectedBatchSize, final int numValsInVector, NullabilityHolder holder) { final int actualBatchSize = getActualBatchSize(expectedBatchSize); if (actualBatchSize <= 0) { return 0; } - vectorizedDefinitionLevelReader.readBatchOfDictionaryIds( - vector, - numValsInVector, - actualBatchSize, - holder, - dictionaryEncodedValuesReader); + vectorizedDefinitionLevelReader.dictionaryIdReader().nextDictEncodedBatch(vector, numValsInVector, -1, + actualBatchSize, holder, dictionaryEncodedValuesReader, null); triplesRead += actualBatchSize; this.hasNext = triplesRead < triplesCount; return actualBatchSize; } + abstract class BagePageReader { + public int nextBatch( + FieldVector vector, int expectedBatchSize, int numValsInVector, int typeWidth, NullabilityHolder holder) { + final int actualBatchSize = getActualBatchSize(expectedBatchSize); + if (actualBatchSize <= 0) { + return 0; + } + if (dictionaryDecodeMode == DictionaryDecodeMode.EAGER) { + nextDictEncodedVal(vector, actualBatchSize, numValsInVector, typeWidth, holder); + } else { + nextVal(vector, actualBatchSize, numValsInVector, typeWidth, holder); + } + triplesRead += actualBatchSize; + hasNext = triplesRead < triplesCount; + return actualBatchSize; + } + + protected abstract void nextVal( + FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder); + protected abstract void nextDictEncodedVal( + FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder); + } + /** * Method for reading a batch of values of INT32 data type */ - public int nextBatchIntegers( - final FieldVector vector, final int expectedBatchSize, - final int numValsInVector, - final int typeWidth, NullabilityHolder holder) { - final int actualBatchSize = getActualBatchSize(expectedBatchSize); - if (actualBatchSize <= 0) { - return 0; + class IntPageReader extends BagePageReader { + @Override + protected void nextVal(FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { + vectorizedDefinitionLevelReader.integerReader().nextBatch(vector, numVals, typeWidth, batchSize, + holder, plainValuesReader); } - if (dictionaryDecodeMode == DictionaryDecodeMode.EAGER) { - vectorizedDefinitionLevelReader.readBatchOfDictionaryEncodedIntegers( - vector, - numValsInVector, - typeWidth, - actualBatchSize, - holder, - dictionaryEncodedValuesReader, - dictionary); - } else { - vectorizedDefinitionLevelReader.readBatchOfIntegers( - vector, - numValsInVector, - typeWidth, - actualBatchSize, - holder, - plainValuesReader); + + @Override + protected void nextDictEncodedVal( + FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { + vectorizedDefinitionLevelReader.integerReader().nextDictEncodedBatch(vector, numVals, typeWidth, batchSize, + holder, dictionaryEncodedValuesReader, dictionary); } - triplesRead += actualBatchSize; - this.hasNext = triplesRead < triplesCount; - return actualBatchSize; } /** * Method for reading a batch of values of INT64 data type */ - public int nextBatchLongs( - final FieldVector vector, final int expectedBatchSize, - final int numValsInVector, - final int typeWidth, NullabilityHolder holder) { - final int actualBatchSize = getActualBatchSize(expectedBatchSize); - if (actualBatchSize <= 0) { - return 0; + class LongPageReader extends BagePageReader { + + @Override + protected void nextVal(FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { + vectorizedDefinitionLevelReader.longReader().nextBatch(vector, numVals, typeWidth, + batchSize, holder, plainValuesReader); } - if (dictionaryDecodeMode == DictionaryDecodeMode.EAGER) { - vectorizedDefinitionLevelReader.readBatchOfDictionaryEncodedLongs( - vector, - numValsInVector, - typeWidth, - actualBatchSize, - holder, - dictionaryEncodedValuesReader, - dictionary); - } else { - vectorizedDefinitionLevelReader.readBatchOfLongs( - vector, - numValsInVector, - typeWidth, - actualBatchSize, - holder, - plainValuesReader); + + @Override + protected void nextDictEncodedVal( + FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { + vectorizedDefinitionLevelReader.longReader().nextDictEncodedBatch(vector, numVals, typeWidth, + batchSize, holder, dictionaryEncodedValuesReader, dictionary); } - triplesRead += actualBatchSize; - this.hasNext = triplesRead < triplesCount; - return actualBatchSize; } /** @@ -167,103 +213,58 @@ public int nextBatchLongs( * is always represented in micro-seconds. So we multiply values stored in millis with 1000 * before writing them to the vector. */ - public int nextBatchTimestampMillis( - final FieldVector vector, final int expectedBatchSize, - final int numValsInVector, - final int typeWidth, NullabilityHolder holder) { - final int actualBatchSize = getActualBatchSize(expectedBatchSize); - if (actualBatchSize <= 0) { - return 0; + class TimestampMillisPageReader extends BagePageReader { + + @Override + protected void nextVal(FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { + vectorizedDefinitionLevelReader.timestampMillisReader().nextBatch(vector, numVals, typeWidth, + batchSize, holder, plainValuesReader); } - if (dictionaryDecodeMode == DictionaryDecodeMode.EAGER) { - vectorizedDefinitionLevelReader.readBatchOfDictionaryEncodedTimestampMillis( - vector, - numValsInVector, - typeWidth, - actualBatchSize, - holder, - dictionaryEncodedValuesReader, - dictionary); - } else { - vectorizedDefinitionLevelReader.readBatchOfTimestampMillis( - vector, - numValsInVector, - typeWidth, - actualBatchSize, - holder, - plainValuesReader); + + @Override + protected void nextDictEncodedVal( + FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { + vectorizedDefinitionLevelReader.timestampMillisReader().nextDictEncodedBatch(vector, numVals, typeWidth, + batchSize, holder, dictionaryEncodedValuesReader, dictionary); } - triplesRead += actualBatchSize; - this.hasNext = triplesRead < triplesCount; - return actualBatchSize; } /** * Method for reading a batch of values of FLOAT data type. */ - public int nextBatchFloats( - final FieldVector vector, final int expectedBatchSize, - final int numValsInVector, - final int typeWidth, NullabilityHolder holder) { - final int actualBatchSize = getActualBatchSize(expectedBatchSize); - if (actualBatchSize <= 0) { - return 0; + class FloatPageReader extends BagePageReader { + + @Override + protected void nextVal(FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { + vectorizedDefinitionLevelReader.floatReader().nextBatch(vector, numVals, typeWidth, + batchSize, holder, plainValuesReader); } - if (dictionaryDecodeMode == DictionaryDecodeMode.EAGER) { - vectorizedDefinitionLevelReader.readBatchOfDictionaryEncodedFloats( - vector, - numValsInVector, - typeWidth, - actualBatchSize, - holder, - dictionaryEncodedValuesReader, - dictionary); - } else { - vectorizedDefinitionLevelReader.readBatchOfFloats( - vector, - numValsInVector, - typeWidth, - actualBatchSize, - holder, - plainValuesReader); + + @Override + protected void nextDictEncodedVal( + FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { + vectorizedDefinitionLevelReader.floatReader().nextDictEncodedBatch(vector, numVals, typeWidth, + batchSize, holder, dictionaryEncodedValuesReader, dictionary); } - triplesRead += actualBatchSize; - this.hasNext = triplesRead < triplesCount; - return actualBatchSize; } /** * Method for reading a batch of values of DOUBLE data type */ - public int nextBatchDoubles( - final FieldVector vector, final int expectedBatchSize, - final int numValsInVector, - final int typeWidth, NullabilityHolder holder) { - final int actualBatchSize = getActualBatchSize(expectedBatchSize); - if (actualBatchSize <= 0) { - return 0; + class DoublePageReader extends BagePageReader { + + @Override + protected void nextVal(FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { + vectorizedDefinitionLevelReader.doubleReader().nextBatch(vector, numVals, typeWidth, + batchSize, holder, plainValuesReader); } - if (dictionaryDecodeMode == DictionaryDecodeMode.EAGER) { - vectorizedDefinitionLevelReader.readBatchOfDictionaryEncodedDoubles( - vector, - numValsInVector, - typeWidth, - actualBatchSize, - holder, - dictionaryEncodedValuesReader, - dictionary); - } else { - vectorizedDefinitionLevelReader.readBatchOfDoubles( - vector, - numValsInVector, - typeWidth, - actualBatchSize, - holder, - plainValuesReader); + + @Override + protected void nextDictEncodedVal( + FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { + vectorizedDefinitionLevelReader.doubleReader().nextDictEncodedBatch(vector, numVals, typeWidth, + batchSize, holder, dictionaryEncodedValuesReader, dictionary); } - triplesRead += actualBatchSize; - this.hasNext = triplesRead < triplesCount; - return actualBatchSize; } private int getActualBatchSize(int expectedBatchSize) { @@ -274,62 +275,36 @@ private int getActualBatchSize(int expectedBatchSize) { * Method for reading a batch of decimals backed by INT32 and INT64 parquet data types. Since Arrow stores all * decimals in 16 bytes, byte arrays are appropriately padded before being written to Arrow data buffers. */ - public int nextBatchIntBackedDecimal( - final FieldVector vector, final int expectedBatchSize, final int numValsInVector, - NullabilityHolder nullabilityHolder) { - final int actualBatchSize = getActualBatchSize(expectedBatchSize); - if (actualBatchSize <= 0) { - return 0; + class IntBackedDecimalPageReader extends BagePageReader { + @Override + protected void nextVal(FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { + vectorizedDefinitionLevelReader.intBackedDecimalReader().nextBatch(vector, numVals, typeWidth, batchSize, + holder, plainValuesReader); } - if (dictionaryDecodeMode == DictionaryDecodeMode.EAGER) { - vectorizedDefinitionLevelReader - .readBatchOfDictionaryEncodedIntBackedDecimals( - vector, - numValsInVector, - actualBatchSize, - nullabilityHolder, - dictionaryEncodedValuesReader, - dictionary); - } else { - vectorizedDefinitionLevelReader.readBatchOfIntBackedDecimals( - vector, - numValsInVector, - actualBatchSize, - nullabilityHolder, - plainValuesReader); + + @Override + protected void nextDictEncodedVal( + FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { + vectorizedDefinitionLevelReader.intBackedDecimalReader() + .nextDictEncodedBatch(vector, numVals, typeWidth, batchSize, + holder, dictionaryEncodedValuesReader, dictionary); } - triplesRead += actualBatchSize; - this.hasNext = triplesRead < triplesCount; - return actualBatchSize; } - public int nextBatchLongBackedDecimal( - final FieldVector vector, final int expectedBatchSize, final int numValsInVector, - NullabilityHolder nullabilityHolder) { - final int actualBatchSize = getActualBatchSize(expectedBatchSize); - if (actualBatchSize <= 0) { - return 0; + class LongBackedDecimalPageReader extends BagePageReader { + @Override + protected void nextVal(FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { + vectorizedDefinitionLevelReader.longBackedDecimalReader().nextBatch(vector, numVals, typeWidth, batchSize, + holder, plainValuesReader); } - if (dictionaryDecodeMode == DictionaryDecodeMode.EAGER) { - vectorizedDefinitionLevelReader - .readBatchOfDictionaryEncodedLongBackedDecimals( - vector, - numValsInVector, - actualBatchSize, - nullabilityHolder, - dictionaryEncodedValuesReader, - dictionary); - } else { - vectorizedDefinitionLevelReader.readBatchOfLongBackedDecimals( - vector, - numValsInVector, - actualBatchSize, - nullabilityHolder, - plainValuesReader); + + @Override + protected void nextDictEncodedVal( + FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { + vectorizedDefinitionLevelReader.longBackedDecimalReader() + .nextDictEncodedBatch(vector, numVals, typeWidth, batchSize, + holder, dictionaryEncodedValuesReader, dictionary); } - triplesRead += actualBatchSize; - this.hasNext = triplesRead < triplesCount; - return actualBatchSize; } /** @@ -339,67 +314,52 @@ public int nextBatchLongBackedDecimal( * uses {@link DecimalVector#setBigEndian(int, byte[])} method so that the data in Arrow vector is indeed little * endian. */ - public int nextBatchFixedLengthDecimal( - final FieldVector vector, final int expectedBatchSize, final int numValsInVector, - final int typeWidth, NullabilityHolder nullabilityHolder) { - final int actualBatchSize = getActualBatchSize(expectedBatchSize); - if (actualBatchSize <= 0) { - return 0; + class FixedLengthDecimalPageReader extends BagePageReader { + @Override + protected void nextVal(FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { + vectorizedDefinitionLevelReader.fixedLengthDecimalReader().nextBatch(vector, numVals, typeWidth, + batchSize, holder, plainValuesReader); } - if (dictionaryDecodeMode == DictionaryDecodeMode.EAGER) { - vectorizedDefinitionLevelReader.readBatchOfDictionaryEncodedFixedLengthDecimals( - vector, - numValsInVector, - typeWidth, - actualBatchSize, - nullabilityHolder, - dictionaryEncodedValuesReader, - dictionary); - } else { - vectorizedDefinitionLevelReader.readBatchOfFixedLengthDecimals( - vector, - numValsInVector, - typeWidth, - actualBatchSize, - nullabilityHolder, - plainValuesReader); + + @Override + protected void nextDictEncodedVal( + FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { + vectorizedDefinitionLevelReader.fixedLengthDecimalReader().nextDictEncodedBatch(vector, numVals, typeWidth, + batchSize, holder, dictionaryEncodedValuesReader, dictionary); + } + } + + class FixedSizeBinaryPageReader extends BagePageReader { + @Override + protected void nextVal(FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { + vectorizedDefinitionLevelReader.fixedSizeBinaryReader().nextBatch(vector, numVals, typeWidth, + batchSize, holder, plainValuesReader); + } + + @Override + protected void nextDictEncodedVal( + FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { + vectorizedDefinitionLevelReader.fixedSizeBinaryReader().nextDictEncodedBatch(vector, numVals, typeWidth, + batchSize, holder, dictionaryEncodedValuesReader, dictionary); } - triplesRead += actualBatchSize; - this.hasNext = triplesRead < triplesCount; - return actualBatchSize; } /** * Method for reading a batch of variable width data type (ENUM, JSON, UTF8, BSON). */ - public int nextBatchVarWidthType( - final FieldVector vector, - final int expectedBatchSize, - final int numValsInVector, - NullabilityHolder nullabilityHolder) { - final int actualBatchSize = getActualBatchSize(expectedBatchSize); - if (actualBatchSize <= 0) { - return 0; + class VarWidthTypePageReader extends BagePageReader { + @Override + protected void nextVal(FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { + vectorizedDefinitionLevelReader.varWidthReader().nextBatch(vector, numVals, typeWidth, batchSize, + holder, plainValuesReader); } - if (dictionaryDecodeMode == DictionaryDecodeMode.EAGER) { - vectorizedDefinitionLevelReader.readBatchOfDictionaryEncodedVarWidth( - vector, - numValsInVector, - actualBatchSize, - nullabilityHolder, - dictionaryEncodedValuesReader, - dictionary); - } else { - vectorizedDefinitionLevelReader.readBatchVarWidth( - vector, - numValsInVector, - actualBatchSize, - nullabilityHolder, - plainValuesReader); + + @Override + protected void nextDictEncodedVal( + FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { + vectorizedDefinitionLevelReader.varWidthReader().nextDictEncodedBatch(vector, numVals, typeWidth, batchSize, + holder, dictionaryEncodedValuesReader, dictionary); } - triplesRead += actualBatchSize; - this.hasNext = triplesRead < triplesCount; - return actualBatchSize; } /** @@ -407,107 +367,83 @@ public int nextBatchVarWidthType( * data type. To work around this limitation, the data is read as fixed width binary from parquet and stored in a * {@link VarBinaryVector} in Arrow. */ - public int nextBatchFixedWidthBinary( - final FieldVector vector, final int expectedBatchSize, final int numValsInVector, - final int typeWidth, NullabilityHolder nullabilityHolder) { - final int actualBatchSize = getActualBatchSize(expectedBatchSize); - if (actualBatchSize <= 0) { - return 0; - } - if (dictionaryDecodeMode == DictionaryDecodeMode.EAGER) { - vectorizedDefinitionLevelReader.readBatchOfDictionaryEncodedFixedWidthBinary( - vector, - numValsInVector, - typeWidth, - actualBatchSize, - nullabilityHolder, - dictionaryEncodedValuesReader, - dictionary); - } else { - vectorizedDefinitionLevelReader.readBatchOfFixedWidthBinary( - vector, - numValsInVector, - typeWidth, - actualBatchSize, - nullabilityHolder, - plainValuesReader); + class FixedWidthBinaryPageReader extends BagePageReader { + @Override + protected void nextVal(FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { + vectorizedDefinitionLevelReader.fixedWidthBinaryReader().nextBatch(vector, numVals, typeWidth, + batchSize, holder, plainValuesReader); } - triplesRead += actualBatchSize; - this.hasNext = triplesRead < triplesCount; - return actualBatchSize; - } - public boolean producesDictionaryEncodedVector() { - return dictionaryDecodeMode == DictionaryDecodeMode.LAZY; + @Override + protected void nextDictEncodedVal( + FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { + vectorizedDefinitionLevelReader.fixedWidthBinaryReader().nextDictEncodedBatch(vector, numVals, typeWidth, + batchSize, holder, dictionaryEncodedValuesReader, dictionary); + } } /** * Method for reading batches of booleans. */ - public int nextBatchBoolean( - final FieldVector vector, - final int expectedBatchSize, - final int numValsInVector, - NullabilityHolder nullabilityHolder) { - final int actualBatchSize = getActualBatchSize(expectedBatchSize); - if (actualBatchSize <= 0) { - return 0; + class BooleanPageReader extends BagePageReader { + @Override + protected void nextVal(FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { + vectorizedDefinitionLevelReader.booleanReader().nextBatch(vector, numVals, typeWidth, batchSize, holder, + plainValuesReader); } - vectorizedDefinitionLevelReader - .readBatchOfBooleans(vector, numValsInVector, actualBatchSize, - nullabilityHolder, plainValuesReader); - triplesRead += actualBatchSize; - this.hasNext = triplesRead < triplesCount; - return actualBatchSize; - } - @Override - protected void initDataReader(Encoding dataEncoding, ByteBufferInputStream in, int valueCount) { - ValuesReader previousReader = plainValuesReader; - if (dataEncoding.usesDictionary()) { - if (dictionary == null) { - throw new ParquetDecodingException( - "could not read page in col " + desc + " as the dictionary was missing for encoding " + dataEncoding); - } - try { - dictionaryEncodedValuesReader = - new VectorizedDictionaryEncodedParquetValuesReader(desc.getMaxDefinitionLevel(), setArrowValidityVector); - dictionaryEncodedValuesReader.initFromPage(valueCount, in); - if (ParquetUtil.isIntType(desc.getPrimitiveType()) || !allPagesDictEncoded) { - dictionaryDecodeMode = DictionaryDecodeMode.EAGER; - } else { - dictionaryDecodeMode = DictionaryDecodeMode.LAZY; - } - } catch (IOException e) { - throw new ParquetDecodingException("could not read page in col " + desc, e); - } - } else { - plainValuesReader = new ValuesAsBytesReader(); - plainValuesReader.initFromPage(valueCount, in); - dictionaryDecodeMode = DictionaryDecodeMode.NONE; - } - if (CorruptDeltaByteArrays.requiresSequentialReads(writerVersion, dataEncoding) && - previousReader != null && previousReader instanceof RequiresPreviousReader) { - // previous reader can only be set if reading sequentially - ((RequiresPreviousReader) plainValuesReader).setPreviousReader(previousReader); + @Override + protected void nextDictEncodedVal( + FieldVector vector, int batchSize, int numVals, int typeWidth, NullabilityHolder holder) { + throw new UnsupportedOperationException(); } } - @Override - protected void initDefinitionLevelsReader(DataPageV1 dataPageV1, ColumnDescriptor desc, ByteBufferInputStream in, - int triplesCount) throws IOException { - this.vectorizedDefinitionLevelReader = newVectorizedDefinitionLevelReader(desc); - this.vectorizedDefinitionLevelReader.initFromPage(triplesCount, in); + IntPageReader intPageReader() { + return new IntPageReader(); } - @Override - protected void initDefinitionLevelsReader(DataPageV2 dataPageV2, ColumnDescriptor desc) { - this.vectorizedDefinitionLevelReader = newVectorizedDefinitionLevelReader(desc); + LongPageReader longPageReader() { + return new LongPageReader(); + } + + TimestampMillisPageReader timestampMillisPageReader() { + return new TimestampMillisPageReader(); } - private VectorizedParquetDefinitionLevelReader newVectorizedDefinitionLevelReader(ColumnDescriptor desc) { - int bitwidth = BytesUtils.getWidthFromMaxInt(desc.getMaxDefinitionLevel()); - return new VectorizedParquetDefinitionLevelReader(bitwidth, desc.getMaxDefinitionLevel(), setArrowValidityVector); + FloatPageReader floatPageReader() { + return new FloatPageReader(); } + DoublePageReader doublePageReader() { + return new DoublePageReader(); + } + + IntBackedDecimalPageReader intBackedDecimalPageReader() { + return new IntBackedDecimalPageReader(); + } + + LongBackedDecimalPageReader longBackedDecimalPageReader() { + return new LongBackedDecimalPageReader(); + } + + FixedLengthDecimalPageReader fixedLengthDecimalPageReader() { + return new FixedLengthDecimalPageReader(); + } + + FixedSizeBinaryPageReader fixedSizeBinaryPageReader() { + return new FixedSizeBinaryPageReader(); + } + + VarWidthTypePageReader varWidthTypePageReader() { + return new VarWidthTypePageReader(); + } + + FixedWidthBinaryPageReader fixedWidthBinaryPageReader() { + return new FixedWidthBinaryPageReader(); + } + + BooleanPageReader booleanPageReader() { + return new BooleanPageReader(); + } } diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetDefinitionLevelReader.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetDefinitionLevelReader.java index d330f0977cbc..8acfd9a7556f 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetDefinitionLevelReader.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetDefinitionLevelReader.java @@ -26,6 +26,7 @@ import org.apache.arrow.vector.BitVectorHelper; import org.apache.arrow.vector.DecimalVector; import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.FixedSizeBinaryVector; import org.apache.arrow.vector.IntVector; import org.apache.arrow.vector.VarBinaryVector; import org.apache.iceberg.arrow.vectorized.NullabilityHolder; @@ -38,1000 +39,495 @@ public VectorizedParquetDefinitionLevelReader(int bitWidth, int maxDefLevel, boo super(bitWidth, maxDefLevel, setArrowValidityVector); } - public void readBatchOfDictionaryIds( - final IntVector vector, - final int startOffset, - final int numValsToRead, - NullabilityHolder nullabilityHolder, - VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader) { - int idx = startOffset; - int left = numValsToRead; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int numValues = Math.min(left, this.currentCount); - switch (mode) { - case RLE: - if (currentValue == maxDefLevel) { - dictionaryEncodedValuesReader.readBatchOfDictionaryIds(vector, idx, numValues, nullabilityHolder); - } else { - setNulls(nullabilityHolder, idx, numValues, vector.getValidityBuffer()); - } - idx += numValues; - break; - case PACKED: - for (int i = 0; i < numValues; i++) { - if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) { - vector.getDataBuffer().setInt(idx * IntVector.TYPE_WIDTH, dictionaryEncodedValuesReader.readInteger()); - nullabilityHolder.setNotNull(idx); - if (setArrowValidityVector) { - BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx); - } - } else { - setNull(nullabilityHolder, idx, vector.getValidityBuffer()); - } - idx++; - } - break; - } - left -= numValues; - currentCount -= numValues; - } + public VectorizedParquetDefinitionLevelReader(int bitWidth, int maxDefLevel, boolean readLength, + boolean setArrowValidityVector) { + super(bitWidth, maxDefLevel, readLength, setArrowValidityVector); } - public void readBatchOfLongs( - final FieldVector vector, final int startOffset, final int typeWidth, - final int numValsToRead, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) { - int bufferIdx = startOffset; - int left = numValsToRead; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int numValues = Math.min(left, this.currentCount); - switch (mode) { - case RLE: - setNextNValuesInVector( - typeWidth, - nullabilityHolder, - valuesReader, - bufferIdx, - vector, - numValues); - bufferIdx += numValues; - break; - case PACKED: - for (int i = 0; i < numValues; ++i) { - if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) { - vector.getDataBuffer().setLong(bufferIdx * typeWidth, valuesReader.readLong()); - nullabilityHolder.setNotNull(bufferIdx); - if (setArrowValidityVector) { - BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), bufferIdx); + abstract class NumericBaseReader { + public void nextBatch( + final FieldVector vector, final int startOffset, final int typeWidth, + final int numValsToRead, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) { + int bufferIdx = startOffset; + int left = numValsToRead; + while (left > 0) { + if (currentCount == 0) { + readNextGroup(); + } + int numValues = Math.min(left, currentCount); + switch (mode) { + case RLE: + setNextNValuesInVector(typeWidth, nullabilityHolder, valuesReader, bufferIdx, vector, numValues); + bufferIdx += numValues; + break; + case PACKED: + for (int i = 0; i < numValues; ++i) { + if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) { + nextVal(vector, bufferIdx * typeWidth, valuesReader, mode); + nullabilityHolder.setNotNull(bufferIdx); + if (setArrowValidityVector) { + BitVectorHelper.setBit(vector.getValidityBuffer(), bufferIdx); + } + } else { + setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer()); } - } else { - setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer()); + bufferIdx++; } - bufferIdx++; - } - break; + break; + } + left -= numValues; + currentCount -= numValues; } - left -= numValues; - currentCount -= numValues; } - } - public void readBatchOfTimestampMillis(final FieldVector vector, final int startOffset, final int typeWidth, - final int numValsToRead, NullabilityHolder nullabilityHolder, - ValuesAsBytesReader valuesReader) { - int bufferIdx = startOffset; - int left = numValsToRead; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int numValues = Math.min(left, this.currentCount); - switch (mode) { - case RLE: - ArrowBuf validityBuffer = vector.getValidityBuffer(); - if (currentValue == maxDefLevel) { - for (int i = 0; i < numValues; i++) { - vector.getDataBuffer().setLong(bufferIdx * typeWidth, valuesReader.readLong() * 1000); - } - nullabilityHolder.setNotNulls(bufferIdx, numValues); - if (setArrowValidityVector) { - for (int i = 0; i < numValues; i++) { - BitVectorHelper.setValidityBitToOne(validityBuffer, bufferIdx + i); - } + public void nextDictEncodedBatch( + final FieldVector vector, final int startOffset, final int typeWidth, + final int numValsToRead, NullabilityHolder nullabilityHolder, + VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader, Dictionary dict) { + int idx = startOffset; + int left = numValsToRead; + while (left > 0) { + if (currentCount == 0) { + readNextGroup(); + } + int numValues = Math.min(left, currentCount); + ArrowBuf validityBuffer = vector.getValidityBuffer(); + switch (mode) { + case RLE: + if (currentValue == maxDefLevel) { + nextDictEncodedVal(vector, idx, dictionaryEncodedValuesReader, dict, mode, + numValues, nullabilityHolder, typeWidth); + } else { + setNulls(nullabilityHolder, idx, numValues, validityBuffer); } - } else { - setNulls(nullabilityHolder, bufferIdx, numValues, validityBuffer); - } - bufferIdx += numValues; - break; - case PACKED: - for (int i = 0; i < numValues; i++) { - if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) { - vector.getDataBuffer().setLong(bufferIdx * typeWidth, valuesReader.readLong() * 1000); - nullabilityHolder.setNotNull(bufferIdx); - if (setArrowValidityVector) { - BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), bufferIdx); + idx += numValues; + break; + case PACKED: + for (int i = 0; i < numValues; i++) { + if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) { + nextDictEncodedVal(vector, idx, dictionaryEncodedValuesReader, dict, mode, + numValues, nullabilityHolder, typeWidth); + nullabilityHolder.setNotNull(idx); + if (setArrowValidityVector) { + BitVectorHelper.setBit(vector.getValidityBuffer(), idx); + } + } else { + setNull(nullabilityHolder, idx, validityBuffer); } - } else { - setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer()); + idx++; } - bufferIdx++; - } - break; + break; + } + left -= numValues; + currentCount -= numValues; } - left -= numValues; - currentCount -= numValues; } + + protected abstract void nextVal( + FieldVector vector, int idx, ValuesAsBytesReader valuesReader, Mode mode); + protected abstract void nextDictEncodedVal( + FieldVector vector, int idx, VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader, + Dictionary dict, Mode mode, int numValues, NullabilityHolder holder, int typeWidth); } - public void readBatchOfDictionaryEncodedLongs( - final FieldVector vector, - final int startOffset, - final int typeWidth, - final int numValsToRead, - NullabilityHolder nullabilityHolder, - VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader, - Dictionary dict) { - int idx = startOffset; - int left = numValsToRead; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int numValues = Math.min(left, this.currentCount); - ArrowBuf validityBuffer = vector.getValidityBuffer(); - switch (mode) { - case RLE: - if (currentValue == maxDefLevel) { - dictionaryEncodedValuesReader.readBatchOfDictionaryEncodedLongs(vector, - idx, numValues, dict, nullabilityHolder, typeWidth); - } else { - setNulls(nullabilityHolder, idx, numValues, validityBuffer); - } - idx += numValues; - break; - case PACKED: - for (int i = 0; i < numValues; i++) { - if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) { - vector.getDataBuffer().setLong(idx * typeWidth, - dict.decodeToLong(dictionaryEncodedValuesReader.readInteger())); - nullabilityHolder.setNotNull(idx); - if (setArrowValidityVector) { - BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx); - } - } else { - setNull(nullabilityHolder, idx, validityBuffer); - } - idx++; - } - break; - } - left -= numValues; - currentCount -= numValues; + class LongReader extends NumericBaseReader { + @Override + protected void nextVal(FieldVector vector, int idx, ValuesAsBytesReader valuesReader, Mode mode) { + vector.getDataBuffer().setLong(idx, valuesReader.readLong()); } - } - public void readBatchOfDictionaryEncodedTimestampMillis( - final FieldVector vector, - final int startOffset, - final int typeWidth, - final int numValsToRead, - NullabilityHolder nullabilityHolder, - VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader, - Dictionary dict) { - int idx = startOffset; - int left = numValsToRead; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int numValues = Math.min(left, this.currentCount); - ArrowBuf validityBuffer = vector.getValidityBuffer(); - switch (mode) { - case RLE: - if (currentValue == maxDefLevel) { - dictionaryEncodedValuesReader.readBatchOfDictionaryEncodedTimestampMillis(vector, - idx, numValues, dict, nullabilityHolder, typeWidth); - } else { - setNulls(nullabilityHolder, idx, numValues, validityBuffer); - } - idx += numValues; - break; - case PACKED: - for (int i = 0; i < numValues; i++) { - if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) { - vector.getDataBuffer().setLong(idx * typeWidth, - dict.decodeToLong(dictionaryEncodedValuesReader.readInteger()) * 1000); - nullabilityHolder.setNotNull(idx); - if (setArrowValidityVector) { - BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx); - } - } else { - setNull(nullabilityHolder, idx, validityBuffer); - } - idx++; - } - break; + @Override + protected void nextDictEncodedVal( + FieldVector vector, int idx, VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader, + Dictionary dict, Mode mode, int numValues, NullabilityHolder holder, int typeWidth) { + if (Mode.RLE.equals(mode)) { + dictionaryEncodedValuesReader.longDictEncodedReader() + .nextBatch(vector, idx, numValues, dict, holder, typeWidth); + } else if (Mode.PACKED.equals(mode)) { + vector.getDataBuffer() + .setLong((long) idx * typeWidth, dict.decodeToLong(dictionaryEncodedValuesReader.readInteger())); } - left -= numValues; - currentCount -= numValues; } } - public void readBatchOfIntegers(final FieldVector vector, final int startOffset, final int typeWidth, - final int numValsToRead, NullabilityHolder nullabilityHolder, - ValuesAsBytesReader valuesReader) { - int bufferIdx = startOffset; - int left = numValsToRead; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int num = Math.min(left, this.currentCount); - switch (mode) { - case RLE: - setNextNValuesInVector( - typeWidth, - nullabilityHolder, - valuesReader, - bufferIdx, - vector, - num); - bufferIdx += num; - break; - case PACKED: - for (int i = 0; i < num; ++i) { - if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) { - vector.getDataBuffer().setInt(bufferIdx * typeWidth, valuesReader.readInteger()); - nullabilityHolder.setNotNull(bufferIdx); - if (setArrowValidityVector) { - BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), bufferIdx); - } - } else { - setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer()); - } - bufferIdx++; - } - break; - } - left -= num; - currentCount -= num; + class DoubleReader extends NumericBaseReader { + @Override + protected void nextVal(FieldVector vector, int idx, ValuesAsBytesReader valuesReader, Mode mode) { + vector.getDataBuffer().setDouble(idx, valuesReader.readDouble()); } - } - public void readBatchOfDictionaryEncodedIntegers( - final FieldVector vector, - final int startOffset, - final int typeWidth, - final int numValsToRead, - NullabilityHolder nullabilityHolder, - VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader, - Dictionary dict) { - int idx = startOffset; - int left = numValsToRead; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); + @Override + protected void nextDictEncodedVal( + FieldVector vector, int idx, VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader, + Dictionary dict, Mode mode, int numValues, NullabilityHolder holder, int typeWidth) { + if (Mode.RLE.equals(mode)) { + dictionaryEncodedValuesReader.doubleDictEncodedReader().nextBatch(vector, + idx, numValues, dict, holder, typeWidth); + } else if (Mode.PACKED.equals(mode)) { + vector.getDataBuffer() + .setDouble((long) idx * typeWidth, dict.decodeToDouble(dictionaryEncodedValuesReader.readInteger())); } - int num = Math.min(left, this.currentCount); - switch (mode) { - case RLE: - if (currentValue == maxDefLevel) { - dictionaryEncodedValuesReader.readBatchOfDictionaryEncodedIntegers(vector, idx, - num, dict, nullabilityHolder, typeWidth); - } else { - setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer()); - } - idx += num; - break; - case PACKED: - for (int i = 0; i < num; i++) { - if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) { - vector.getDataBuffer() - .setInt(idx * typeWidth, dict.decodeToInt(dictionaryEncodedValuesReader.readInteger())); - nullabilityHolder.setNotNull(idx); - if (setArrowValidityVector) { - BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx); - } - } else { - setNull(nullabilityHolder, idx, vector.getValidityBuffer()); - } - idx++; - } - break; - } - left -= num; - currentCount -= num; } } - public void readBatchOfFloats(final FieldVector vector, final int startOffset, final int typeWidth, - final int numValsToRead, NullabilityHolder nullabilityHolder, - ValuesAsBytesReader valuesReader) { - int bufferIdx = startOffset; - int left = numValsToRead; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int num = Math.min(left, this.currentCount); - switch (mode) { - case RLE: - setNextNValuesInVector( - typeWidth, - nullabilityHolder, - valuesReader, - bufferIdx, - vector, - num); - bufferIdx += num; - break; - case PACKED: - for (int i = 0; i < num; ++i) { - if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) { - vector.getDataBuffer().setFloat(bufferIdx * typeWidth, valuesReader.readFloat()); - nullabilityHolder.setNotNull(bufferIdx); - if (setArrowValidityVector) { - BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), bufferIdx); - } - } else { - setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer()); - } - bufferIdx++; - } - break; + class FloatReader extends NumericBaseReader { + @Override + protected void nextVal(FieldVector vector, int idx, ValuesAsBytesReader valuesReader, Mode mode) { + vector.getDataBuffer().setFloat(idx, valuesReader.readFloat()); + } + + @Override + protected void nextDictEncodedVal( + FieldVector vector, int idx, + VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader, + Dictionary dict, Mode mode, int numValues, NullabilityHolder holder, int typeWidth) { + if (Mode.RLE.equals(mode)) { + dictionaryEncodedValuesReader.floatDictEncodedReader().nextBatch(vector, + idx, numValues, dict, holder, typeWidth); + } else if (Mode.PACKED.equals(mode)) { + vector.getDataBuffer() + .setFloat((long) idx * typeWidth, dict.decodeToFloat(dictionaryEncodedValuesReader.readInteger())); } - left -= num; - currentCount -= num; } } - public void readBatchOfDictionaryEncodedFloats( - final FieldVector vector, - final int startOffset, - final int typeWidth, - final int numValsToRead, - NullabilityHolder nullabilityHolder, - VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader, - Dictionary dict) { - int idx = startOffset; - int left = numValsToRead; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int num = Math.min(left, this.currentCount); - ArrowBuf validityBuffer = vector.getValidityBuffer(); - switch (mode) { - case RLE: - if (currentValue == maxDefLevel) { - dictionaryEncodedValuesReader.readBatchOfDictionaryEncodedFloats(vector, idx, - num, dict, nullabilityHolder, typeWidth); - } else { - setNulls(nullabilityHolder, idx, num, validityBuffer); - } - idx += num; - break; - case PACKED: - for (int i = 0; i < num; i++) { - if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) { - vector.getDataBuffer() - .setFloat(idx * typeWidth, dict.decodeToFloat(dictionaryEncodedValuesReader.readInteger())); - nullabilityHolder.setNotNull(idx); - if (setArrowValidityVector) { - BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx); - } - } else { - setNull(nullabilityHolder, idx, validityBuffer); - } - idx++; - } - break; + class IntegerReader extends NumericBaseReader { + @Override + protected void nextVal(FieldVector vector, int idx, ValuesAsBytesReader valuesReader, Mode mode) { + vector.getDataBuffer().setInt(idx, valuesReader.readInteger()); + } + + @Override + protected void nextDictEncodedVal( + FieldVector vector, int idx, + VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader, + Dictionary dict, Mode mode, int numValues, NullabilityHolder holder, int typeWidth) { + if (Mode.RLE.equals(mode)) { + dictionaryEncodedValuesReader.integerDictEncodedReader().nextBatch(vector, + idx, numValues, dict, holder, typeWidth); + } else if (Mode.PACKED.equals(mode)) { + vector.getDataBuffer() + .setInt((long) idx * typeWidth, dict.decodeToInt(dictionaryEncodedValuesReader.readInteger())); } - left -= num; - currentCount -= num; } } - public void readBatchOfDoubles( - final FieldVector vector, final int startOffset, - final int typeWidth, final int numValsToRead, NullabilityHolder nullabilityHolder, - ValuesAsBytesReader valuesReader) { - int bufferIdx = startOffset; - int left = numValsToRead; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int num = Math.min(left, this.currentCount); - switch (mode) { - case RLE: - setNextNValuesInVector( - typeWidth, - nullabilityHolder, - valuesReader, - bufferIdx, - vector, - num); - bufferIdx += num; - break; - case PACKED: - for (int i = 0; i < num; ++i) { - if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) { - vector.getDataBuffer().setDouble(bufferIdx * typeWidth, valuesReader.readDouble()); - nullabilityHolder.setNotNull(bufferIdx); - if (setArrowValidityVector) { - BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), bufferIdx); + abstract class BaseReader { + public void nextBatch( + final FieldVector vector, final int startOffset, final int typeWidth, + final int numValsToRead, NullabilityHolder nullabilityHolder, + ValuesAsBytesReader valuesReader) { + int bufferIdx = startOffset; + int left = numValsToRead; + while (left > 0) { + if (currentCount == 0) { + readNextGroup(); + } + int numValues = Math.min(left, currentCount); + byte[] byteArray = null; + if (typeWidth > -1) { + byteArray = new byte[typeWidth]; + } + switch (mode) { + case RLE: + if (currentValue == maxDefLevel) { + for (int i = 0; i < numValues; i++) { + nextVal(vector, bufferIdx, valuesReader, typeWidth, byteArray); + nullabilityHolder.setNotNull(bufferIdx); + bufferIdx++; } } else { - setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer()); + setNulls(nullabilityHolder, bufferIdx, numValues, vector.getValidityBuffer()); + bufferIdx += numValues; } - bufferIdx++; - } - break; + break; + case PACKED: + for (int i = 0; i < numValues; i++) { + if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) { + nextVal(vector, bufferIdx, valuesReader, typeWidth, byteArray); + nullabilityHolder.setNotNull(bufferIdx); + } else { + setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer()); + } + bufferIdx++; + } + break; + } + left -= numValues; + currentCount -= numValues; } - left -= num; - currentCount -= num; } - } - public void readBatchOfDictionaryEncodedDoubles( - final FieldVector vector, - final int startOffset, - final int typeWidth, - final int numValsToRead, - NullabilityHolder nullabilityHolder, - VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader, - Dictionary dict) { - int idx = startOffset; - int left = numValsToRead; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int num = Math.min(left, this.currentCount); - switch (mode) { - case RLE: - if (currentValue == maxDefLevel) { - dictionaryEncodedValuesReader.readBatchOfDictionaryEncodedDoubles(vector, idx, - num, dict, nullabilityHolder, typeWidth); - } else { - setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer()); - } - idx += num; - break; - case PACKED: - for (int i = 0; i < num; i++) { - if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) { - vector.getDataBuffer() - .setDouble(idx * typeWidth, dict.decodeToDouble(dictionaryEncodedValuesReader.readInteger())); - nullabilityHolder.setNotNull(idx); - if (setArrowValidityVector) { - BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx); - } + public void nextDictEncodedBatch( + final FieldVector vector, final int startOffset, final int typeWidth, + final int numValsToRead, NullabilityHolder nullabilityHolder, + VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader, + Dictionary dict) { + int idx = startOffset; + int left = numValsToRead; + while (left > 0) { + if (currentCount == 0) { + readNextGroup(); + } + int numValues = Math.min(left, currentCount); + ArrowBuf validityBuffer = vector.getValidityBuffer(); + switch (mode) { + case RLE: + if (currentValue == maxDefLevel) { + nextDictEncodedVal(vector, + idx, dictionaryEncodedValuesReader, numValues, dict, nullabilityHolder, typeWidth, mode); } else { - setNull(nullabilityHolder, idx, vector.getValidityBuffer()); + setNulls(nullabilityHolder, idx, numValues, validityBuffer); + } + idx += numValues; + break; + case PACKED: + for (int i = 0; i < numValues; i++) { + if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) { + nextDictEncodedVal(vector, + idx, dictionaryEncodedValuesReader, numValues, dict, nullabilityHolder, typeWidth, mode); + nullabilityHolder.setNotNull(idx); + if (setArrowValidityVector) { + BitVectorHelper.setBit(vector.getValidityBuffer(), idx); + } + } else { + setNull(nullabilityHolder, idx, validityBuffer); + } + idx++; } - idx++; - } - break; + break; + } + left -= numValues; + currentCount -= numValues; } - left -= num; - currentCount -= num; } + + protected abstract void nextVal( + FieldVector vector, int idx, ValuesAsBytesReader valuesReader, int typeWidth, byte[] byteArray); + protected abstract void nextDictEncodedVal( + FieldVector vector, int startOffset, VectorizedDictionaryEncodedParquetValuesReader reader, + int numValuesToRead, Dictionary dict, NullabilityHolder nullabilityHolder, int typeWidth, Mode mode); } - public void readBatchOfFixedWidthBinary( - final FieldVector vector, final int startOffset, - final int typeWidth, final int numValsToRead, NullabilityHolder nullabilityHolder, - ValuesAsBytesReader valuesReader) { - int bufferIdx = startOffset; - int left = numValsToRead; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int num = Math.min(left, this.currentCount); - switch (mode) { - case RLE: - if (currentValue == maxDefLevel) { - for (int i = 0; i < num; i++) { - setBinaryInVector((VarBinaryVector) vector, typeWidth, valuesReader, bufferIdx, nullabilityHolder); - bufferIdx++; - } - } else { - setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer()); - bufferIdx += num; - } - break; - case PACKED: - for (int i = 0; i < num; i++) { - if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) { - setBinaryInVector((VarBinaryVector) vector, typeWidth, valuesReader, bufferIdx, nullabilityHolder); - } else { - setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer()); - } - bufferIdx++; - } - break; + class TimestampMillisReader extends BaseReader { + + @Override + protected void nextVal( + FieldVector vector, int idx, ValuesAsBytesReader valuesReader, int typeWidth, byte[] byteArray) { + vector.getDataBuffer().setLong((long) idx * typeWidth, valuesReader.readLong() * 1000); + } + + @Override + protected void nextDictEncodedVal( + FieldVector vector, int idx, VectorizedDictionaryEncodedParquetValuesReader reader, + int numValues, Dictionary dict, NullabilityHolder nullabilityHolder, int typeWidth, Mode mode) { + if (Mode.RLE.equals(mode)) { + reader.timestampMillisDictEncodedReader().nextBatch(vector, + idx, numValues, dict, nullabilityHolder, typeWidth); + } else if (Mode.PACKED.equals(mode)) { + vector.getDataBuffer().setLong((long) idx * typeWidth, dict.decodeToLong(reader.readInteger()) * 1000); } - left -= num; - currentCount -= num; } } - public void readBatchOfDictionaryEncodedFixedWidthBinary( - final FieldVector vector, - final int startOffset, - final int typeWidth, - final int numValsToRead, - NullabilityHolder nullabilityHolder, - VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader, - Dictionary dict) { - int idx = startOffset; - int left = numValsToRead; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int num = Math.min(left, this.currentCount); - switch (mode) { - case RLE: - if (currentValue == maxDefLevel) { - dictionaryEncodedValuesReader.readBatchOfDictionaryEncodedFixedWidthBinary(vector, typeWidth, idx, - num, dict, nullabilityHolder); - } else { - setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer()); - } - idx += num; - break; - case PACKED: - for (int i = 0; i < num; i++) { - if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) { - ByteBuffer buffer = dict.decodeToBinary(dictionaryEncodedValuesReader.readInteger()).toByteBuffer(); - vector.getDataBuffer().setBytes(idx * typeWidth, buffer.array(), - buffer.position() + buffer.arrayOffset(), buffer.limit() - buffer.position()); - nullabilityHolder.setNotNull(idx); - if (setArrowValidityVector) { - BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx); - } - } else { - setNull(nullabilityHolder, idx, vector.getValidityBuffer()); - } - idx++; - } - break; + class FixedWidthBinaryReader extends BaseReader { + @Override + protected void nextVal( + FieldVector vector, int idx, ValuesAsBytesReader valuesReader, int typeWidth, byte[] byteArray) { + ByteBuffer buffer = valuesReader.getBuffer(typeWidth); + ((VarBinaryVector) vector).setSafe(idx, buffer.array(), buffer.position() + buffer.arrayOffset(), + buffer.limit() - buffer.position()); + } + + @Override + protected void nextDictEncodedVal( + FieldVector vector, int idx, VectorizedDictionaryEncodedParquetValuesReader reader, + int numValues, Dictionary dict, NullabilityHolder nullabilityHolder, int typeWidth, Mode mode) { + if (Mode.RLE.equals(mode)) { + reader.fixedWidthBinaryDictEncodedReader() + .nextBatch(vector, idx, numValues, dict, nullabilityHolder, typeWidth); + } else if (Mode.PACKED.equals(mode)) { + ByteBuffer buffer = dict.decodeToBinary(reader.readInteger()).toByteBuffer(); + vector.getDataBuffer().setBytes( + (long) idx * typeWidth, buffer.array(), + buffer.position() + buffer.arrayOffset(), buffer.limit() - buffer.position()); } - left -= num; - currentCount -= num; } } - public void readBatchOfFixedLengthDecimals( - final FieldVector vector, final int startOffset, - final int typeWidth, final int numValsToRead, NullabilityHolder nullabilityHolder, - ValuesAsBytesReader valuesReader) { - int bufferIdx = startOffset; - int left = numValsToRead; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int num = Math.min(left, this.currentCount); - byte[] byteArray = new byte[typeWidth]; - switch (mode) { - case RLE: - if (currentValue == maxDefLevel) { - for (int i = 0; i < num; i++) { - valuesReader.getBuffer(typeWidth).get(byteArray, 0, typeWidth); - ((DecimalVector) vector).setBigEndian(bufferIdx, byteArray); - nullabilityHolder.setNotNull(bufferIdx); - bufferIdx++; - } - } else { - setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer()); - bufferIdx += num; - } - break; - case PACKED: - for (int i = 0; i < num; ++i) { - if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) { - valuesReader.getBuffer(typeWidth).get(byteArray, 0, typeWidth); - ((DecimalVector) vector).setBigEndian(bufferIdx, byteArray); - nullabilityHolder.setNotNull(bufferIdx); - } else { - setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer()); - } - bufferIdx++; - } - break; + class FixedLengthDecimalReader extends BaseReader { + @Override + protected void nextVal( + FieldVector vector, int idx, ValuesAsBytesReader valuesReader, int typeWidth, byte[] byteArray) { + valuesReader.getBuffer(typeWidth).get(byteArray, 0, typeWidth); + ((DecimalVector) vector).setBigEndian(idx, byteArray); + } + + @Override + protected void nextDictEncodedVal( + FieldVector vector, int idx, VectorizedDictionaryEncodedParquetValuesReader reader, + int numValues, Dictionary dict, NullabilityHolder nullabilityHolder, int typeWidth, Mode mode) { + if (Mode.RLE.equals(mode)) { + reader.fixedLengthDecimalDictEncodedReader() + .nextBatch(vector, idx, numValues, dict, nullabilityHolder, typeWidth); + } else if (Mode.PACKED.equals(mode)) { + ByteBuffer decimalBytes = dict.decodeToBinary(reader.readInteger()).toByteBuffer(); + byte[] vectorBytes = new byte[typeWidth]; + System.arraycopy(decimalBytes, 0, vectorBytes, 0, typeWidth); + ((DecimalVector) vector).setBigEndian(idx, vectorBytes); } - left -= num; - currentCount -= num; } } - public void readBatchOfDictionaryEncodedFixedLengthDecimals( - final FieldVector vector, - final int startOffset, - final int typeWidth, - final int numValsToRead, - NullabilityHolder nullabilityHolder, - VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader, - Dictionary dict) { - int idx = startOffset; - int left = numValsToRead; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int num = Math.min(left, this.currentCount); - switch (mode) { - case RLE: - if (currentValue == maxDefLevel) { - dictionaryEncodedValuesReader.readBatchOfDictionaryEncodedFixedLengthDecimals(vector, typeWidth, idx, - num, dict, nullabilityHolder); - } else { - setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer()); - } - idx += num; - break; - case PACKED: - for (int i = 0; i < num; i++) { - if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) { - ByteBuffer decimalBytes = dict.decodeToBinary(dictionaryEncodedValuesReader.readInteger()).toByteBuffer(); - byte[] vectorBytes = new byte[typeWidth]; - System.arraycopy(decimalBytes, 0, vectorBytes, 0, typeWidth); - ((DecimalVector) vector).setBigEndian(idx, vectorBytes); - nullabilityHolder.setNotNull(idx); - } else { - setNull(nullabilityHolder, idx, vector.getValidityBuffer()); - } - idx++; - } - break; + class FixedSizeBinaryReader extends BaseReader { + @Override + protected void nextVal( + FieldVector vector, int idx, ValuesAsBytesReader valuesReader, int typeWidth, byte[] byteArray) { + valuesReader.getBuffer(typeWidth).get(byteArray, 0, typeWidth); + ((FixedSizeBinaryVector) vector).set(idx, byteArray); + } + + @Override + protected void nextDictEncodedVal( + FieldVector vector, int idx, VectorizedDictionaryEncodedParquetValuesReader reader, + int numValues, Dictionary dict, NullabilityHolder nullabilityHolder, int typeWidth, Mode mode) { + if (Mode.RLE.equals(mode)) { + reader.fixedSizeBinaryDictEncodedReader().nextBatch(vector, idx, + numValues, dict, nullabilityHolder, typeWidth); + } else if (Mode.PACKED.equals(mode)) { + byte[] bytes = dict.decodeToBinary(reader.readInteger()).getBytes(); + byte[] vectorBytes = new byte[typeWidth]; + System.arraycopy(bytes, 0, vectorBytes, 0, typeWidth); + ((FixedSizeBinaryVector) vector).set(idx, vectorBytes); } - left -= num; - currentCount -= num; } } - public void readBatchVarWidth( - final FieldVector vector, - final int startOffset, - final int numValsToRead, - NullabilityHolder nullabilityHolder, - ValuesAsBytesReader valuesReader) { - int bufferIdx = startOffset; - int left = numValsToRead; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); + class VarWidthReader extends BaseReader { + @Override + protected void nextVal( + FieldVector vector, int idx, ValuesAsBytesReader valuesReader, int typeWidth, byte[] byteArray) { + int len = valuesReader.readInteger(); + ByteBuffer buffer = valuesReader.getBuffer(len); + // Calling setValueLengthSafe takes care of allocating a larger buffer if + // running out of space. + ((BaseVariableWidthVector) vector).setValueLengthSafe(idx, len); + int startOffset = ((BaseVariableWidthVector) vector).getStartOffset(idx); + // It is possible that the data buffer was reallocated. So it is important to + // not cache the data buffer reference but instead use vector.getDataBuffer(). + vector.getDataBuffer().setBytes(startOffset, buffer.array(), buffer.position() + buffer.arrayOffset(), + buffer.limit() - buffer.position()); + // Similarly, we need to get the latest reference to the validity buffer as well + // since reallocation changes reference of the validity buffers as well. + if (setArrowValidityVector) { + BitVectorHelper.setBit(vector.getValidityBuffer(), idx); } - int num = Math.min(left, this.currentCount); - switch (mode) { - case RLE: - if (currentValue == maxDefLevel) { - for (int i = 0; i < num; i++) { - setVarWidthBinaryValue(vector, valuesReader, bufferIdx, nullabilityHolder); - bufferIdx++; - } - } else { - setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer()); - bufferIdx += num; - } - break; - case PACKED: - for (int i = 0; i < num; i++) { - if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) { - setVarWidthBinaryValue(vector, valuesReader, bufferIdx, nullabilityHolder); - } else { - setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer()); - } - bufferIdx++; - } - break; + } + + @Override + protected void nextDictEncodedVal( + FieldVector vector, int idx, VectorizedDictionaryEncodedParquetValuesReader reader, + int numValues, Dictionary dict, NullabilityHolder nullabilityHolder, int typeWidth, Mode mode) { + if (Mode.RLE.equals(mode)) { + reader.varWidthBinaryDictEncodedReader().nextBatch(vector, idx, + numValues, dict, nullabilityHolder, typeWidth); + } else if (Mode.PACKED.equals(mode)) { + ((BaseVariableWidthVector) vector).setSafe( + idx, dict.decodeToBinary(reader.readInteger()).getBytesUnsafe()); } - left -= num; - currentCount -= num; } } - private void setVarWidthBinaryValue(FieldVector vector, ValuesAsBytesReader valuesReader, - int bufferIdx, NullabilityHolder nullabilityHolder) { - int len = valuesReader.readInteger(); - ByteBuffer buffer = valuesReader.getBuffer(len); - // Calling setValueLengthSafe takes care of allocating a larger buffer if - // running out of space. - ((BaseVariableWidthVector) vector).setValueLengthSafe(bufferIdx, len); - int startOffset = ((BaseVariableWidthVector) vector).getStartOffset(bufferIdx); - // It is possible that the data buffer was reallocated. So it is important to - // not cache the data buffer reference but instead use vector.getDataBuffer(). - vector.getDataBuffer().setBytes(startOffset, buffer.array(), buffer.position() + buffer.arrayOffset(), - buffer.limit() - buffer.position()); - // Similarly, we need to get the latest reference to the validity buffer as well - // since reallocation changes reference of the validity buffers as well. - nullabilityHolder.setNotNull(bufferIdx); - if (setArrowValidityVector) { - BitVectorHelper.setBit(vector.getValidityBuffer(), bufferIdx); + class IntBackedDecimalReader extends BaseReader { + @Override + protected void nextVal( + FieldVector vector, int idx, ValuesAsBytesReader valuesReader, int typeWidth, byte[] byteArray) { + ((DecimalVector) vector).set(idx, valuesReader.getBuffer(Integer.BYTES).getInt()); } - } - public void readBatchOfDictionaryEncodedVarWidth( - final FieldVector vector, - final int startOffset, - final int numValsToRead, - NullabilityHolder nullabilityHolder, - VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader, - Dictionary dict) { - int idx = startOffset; - int left = numValsToRead; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int num = Math.min(left, this.currentCount); - switch (mode) { - case RLE: - if (currentValue == maxDefLevel) { - dictionaryEncodedValuesReader.readBatchOfDictionaryEncodedVarWidthBinary(vector, idx, - num, dict, nullabilityHolder); - } else { - setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer()); - } - idx += num; - break; - case PACKED: - for (int i = 0; i < num; i++) { - if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) { - ((BaseVariableWidthVector) vector).setSafe( - idx, - dict.decodeToBinary(dictionaryEncodedValuesReader.readInteger()).getBytesUnsafe()); - nullabilityHolder.setNotNull(idx); - } else { - setNull(nullabilityHolder, idx, vector.getValidityBuffer()); - } - idx++; - } - break; + @Override + protected void nextDictEncodedVal( + FieldVector vector, int idx, VectorizedDictionaryEncodedParquetValuesReader reader, + int numValues, Dictionary dict, NullabilityHolder nullabilityHolder, int typeWidth, Mode mode) { + if (Mode.RLE.equals(mode)) { + reader.intBackedDecimalDictEncodedReader() + .nextBatch(vector, idx, numValues, dict, nullabilityHolder, typeWidth); + } else if (Mode.PACKED.equals(mode)) { + ((DecimalVector) vector).set(idx, dict.decodeToInt(reader.readInteger())); } - left -= num; - currentCount -= num; } } - public void readBatchOfIntBackedDecimals( - final FieldVector vector, final int startOffset, - final int numValsToRead, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) { - int bufferIdx = startOffset; - int left = numValsToRead; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int num = Math.min(left, this.currentCount); - byte[] byteArray = new byte[Integer.BYTES]; - switch (mode) { - case RLE: - if (currentValue == maxDefLevel) { - for (int i = 0; i < num; i++) { - setIntBackedDecimal(vector, nullabilityHolder, valuesReader, bufferIdx, byteArray); - bufferIdx++; - } - } else { - setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer()); - bufferIdx += num; - } - break; - case PACKED: - for (int i = 0; i < num; ++i) { - if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) { - setIntBackedDecimal(vector, nullabilityHolder, valuesReader, bufferIdx, byteArray); - } else { - setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer()); - } - bufferIdx++; - } - break; - } - left -= num; - currentCount -= num; + class LongBackedDecimalReader extends BaseReader { + @Override + protected void nextVal( + FieldVector vector, int idx, ValuesAsBytesReader valuesReader, int typeWidth, byte[] byteArray) { + ((DecimalVector) vector).set(idx, valuesReader.getBuffer(Long.BYTES).getLong()); } - } - public void readBatchOfLongBackedDecimals( - final FieldVector vector, final int startOffset, - final int numValsToRead, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) { - int bufferIdx = startOffset; - int left = numValsToRead; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int num = Math.min(left, this.currentCount); - byte[] byteArray = new byte[Long.BYTES]; - switch (mode) { - case RLE: - if (currentValue == maxDefLevel) { - for (int i = 0; i < num; i++) { - setLongBackedDecimal(vector, nullabilityHolder, valuesReader, bufferIdx, byteArray); - bufferIdx++; - } - } else { - setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer()); - bufferIdx += num; - } - break; - case PACKED: - for (int i = 0; i < num; ++i) { - if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) { - setLongBackedDecimal(vector, nullabilityHolder, valuesReader, bufferIdx, byteArray); - } else { - setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer()); - } - bufferIdx++; - } - break; + @Override + protected void nextDictEncodedVal( + FieldVector vector, int idx, VectorizedDictionaryEncodedParquetValuesReader reader, + int numValues, Dictionary dict, NullabilityHolder nullabilityHolder, int typeWidth, Mode mode) { + if (Mode.RLE.equals(mode)) { + reader.longBackedDecimalDictEncodedReader() + .nextBatch(vector, idx, numValues, dict, nullabilityHolder, typeWidth); + } else if (Mode.PACKED.equals(mode)) { + ((DecimalVector) vector).set( + idx, dict.decodeToLong(reader.readInteger())); } - left -= num; - currentCount -= num; } } - private void setIntBackedDecimal(FieldVector vector, NullabilityHolder nullabilityHolder, - ValuesAsBytesReader valuesReader, int bufferIdx, byte[] byteArray) { - ((DecimalVector) vector).set(bufferIdx, valuesReader.getBuffer(Integer.BYTES).getInt()); - nullabilityHolder.setNotNull(bufferIdx); - } + class BooleanReader extends BaseReader { + @Override + protected void nextVal( + FieldVector vector, int idx, ValuesAsBytesReader valuesReader, int typeWidth, byte[] byteArray) { + ((BitVector) vector).setSafe(idx, valuesReader.readBooleanAsInt()); + } - private void setLongBackedDecimal(FieldVector vector, NullabilityHolder nullabilityHolder, - ValuesAsBytesReader valuesReader, int bufferIdx, byte[] byteArray) { - ((DecimalVector) vector).set(bufferIdx, valuesReader.getBuffer(Long.BYTES).getLong()); - nullabilityHolder.setNotNull(bufferIdx); + @Override + protected void nextDictEncodedVal( + FieldVector vector, int idx, VectorizedDictionaryEncodedParquetValuesReader reader, + int numValues, Dictionary dict, NullabilityHolder nullabilityHolder, int typeWidth, Mode mode) { + throw new UnsupportedOperationException(); + } } - public void readBatchOfDictionaryEncodedIntBackedDecimals( - final FieldVector vector, - final int startOffset, - final int numValsToRead, - NullabilityHolder nullabilityHolder, - VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader, - Dictionary dict) { - int idx = startOffset; - int left = numValsToRead; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int num = Math.min(left, this.currentCount); - switch (mode) { - case RLE: - if (currentValue == maxDefLevel) { - dictionaryEncodedValuesReader.readBatchOfDictionaryEncodedIntBackedDecimals(vector, idx, - num, dict, nullabilityHolder); - } else { - setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer()); - } - idx += num; - break; - case PACKED: - for (int i = 0; i < num; i++) { - if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) { - ((DecimalVector) vector).set( - idx, - dict.decodeToInt(dictionaryEncodedValuesReader.readInteger())); - nullabilityHolder.setNotNull(idx); - } else { - setNull(nullabilityHolder, idx, vector.getValidityBuffer()); - } - idx++; - } - break; - } - left -= num; - currentCount -= num; + class DictionaryIdReader extends BaseReader { + + @Override + protected void nextVal( + FieldVector vector, int idx, ValuesAsBytesReader valuesReader, int typeWidth, byte[] byteArray) { + throw new UnsupportedOperationException(); } - } - public void readBatchOfDictionaryEncodedLongBackedDecimals( - final FieldVector vector, - final int startOffset, - final int numValsToRead, - NullabilityHolder nullabilityHolder, - VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader, - Dictionary dict) { - int idx = startOffset; - int left = numValsToRead; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); + @Override + protected void nextDictEncodedVal( + FieldVector vector, int idx, VectorizedDictionaryEncodedParquetValuesReader reader, + int numValuesToRead, Dictionary dict, NullabilityHolder nullabilityHolder, int typeWidth, Mode mode) { + if (Mode.RLE.equals(mode)) { + reader.dictionaryIdReader().nextBatch(vector, idx, numValuesToRead, dict, nullabilityHolder, typeWidth); + } else if (Mode.PACKED.equals(mode)) { + vector.getDataBuffer().setInt((long) idx * IntVector.TYPE_WIDTH, reader.readInteger()); } - int num = Math.min(left, this.currentCount); - switch (mode) { - case RLE: - if (currentValue == maxDefLevel) { - dictionaryEncodedValuesReader.readBatchOfDictionaryEncodedLongBackedDecimals(vector, idx, - num, dict, nullabilityHolder); - } else { - setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer()); - } - idx += num; - break; - case PACKED: - for (int i = 0; i < num; i++) { - if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) { - ((DecimalVector) vector).set( - idx, dict.decodeToLong(dictionaryEncodedValuesReader.readInteger())); - nullabilityHolder.setNotNull(idx); - } else { - setNull(nullabilityHolder, idx, vector.getValidityBuffer()); - } - idx++; - } - break; - } - left -= num; - currentCount -= num; } } - public void readBatchOfBooleans( - final FieldVector vector, - final int startOffset, - final int numValsToRead, - NullabilityHolder nullabilityHolder, - ValuesAsBytesReader valuesReader) { - int bufferIdx = startOffset; - int left = numValsToRead; - while (left > 0) { - if (this.currentCount == 0) { - this.readNextGroup(); - } - int num = Math.min(left, this.currentCount); - switch (mode) { - case RLE: - if (currentValue == maxDefLevel) { - for (int i = 0; i < num; i++) { - ((BitVector) vector).setSafe(bufferIdx, valuesReader.readBooleanAsInt()); - nullabilityHolder.setNotNull(bufferIdx); - bufferIdx++; - } - } else { - setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer()); - bufferIdx += num; - } - break; - case PACKED: - for (int i = 0; i < num; ++i) { - if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) { - ((BitVector) vector).setSafe(bufferIdx, valuesReader.readBooleanAsInt()); - nullabilityHolder.setNotNull(bufferIdx); - } else { - setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer()); - } - bufferIdx++; - } - break; - } - left -= num; - currentCount -= num; + private void setNull(NullabilityHolder nullabilityHolder, int bufferIdx, ArrowBuf validityBuffer) { + nullabilityHolder.setNull(bufferIdx); + if (setArrowValidityVector) { + BitVectorHelper.setValidityBit(validityBuffer, bufferIdx, 0); } } - private static void setBinaryInVector( - VarBinaryVector vector, - int typeWidth, - ValuesAsBytesReader valuesReader, - int bufferIdx, NullabilityHolder nullabilityHolder) { - ByteBuffer buffer = valuesReader.getBuffer(typeWidth); - vector.setSafe(bufferIdx, buffer.array(), buffer.position() + buffer.arrayOffset(), - buffer.limit() - buffer.position()); - nullabilityHolder.setNotNull(bufferIdx); + private void setNulls(NullabilityHolder nullabilityHolder, int idx, int numValues, ArrowBuf validityBuffer) { + nullabilityHolder.setNulls(idx, numValues); + if (setArrowValidityVector) { + for (int i = 0; i < numValues; i++) { + BitVectorHelper.setValidityBit(validityBuffer, idx + i, 0); + } + } } private void setNextNValuesInVector( @@ -1040,11 +536,11 @@ private void setNextNValuesInVector( ArrowBuf validityBuffer = vector.getValidityBuffer(); if (currentValue == maxDefLevel) { ByteBuffer buffer = valuesReader.getBuffer(numValues * typeWidth); - vector.getDataBuffer().setBytes(bufferIdx * typeWidth, buffer); + vector.getDataBuffer().setBytes((long) bufferIdx * typeWidth, buffer); nullabilityHolder.setNotNulls(bufferIdx, numValues); if (setArrowValidityVector) { for (int i = 0; i < numValues; i++) { - BitVectorHelper.setValidityBitToOne(validityBuffer, bufferIdx + i); + BitVectorHelper.setBit(validityBuffer, bufferIdx + i); } } } else { @@ -1052,20 +548,55 @@ private void setNextNValuesInVector( } } - private void setNull(NullabilityHolder nullabilityHolder, int bufferIdx, ArrowBuf validityBuffer) { - nullabilityHolder.setNull(bufferIdx); - if (setArrowValidityVector) { - BitVectorHelper.setValidityBit(validityBuffer, bufferIdx, 0); - } + LongReader longReader() { + return new LongReader(); } - private void setNulls(NullabilityHolder nullabilityHolder, int idx, int numValues, ArrowBuf validityBuffer) { - nullabilityHolder.setNulls(idx, numValues); - if (setArrowValidityVector) { - for (int i = 0; i < numValues; i++) { - BitVectorHelper.setValidityBit(validityBuffer, idx + i, 0); - } - } + DoubleReader doubleReader() { + return new DoubleReader(); + } + + FloatReader floatReader() { + return new FloatReader(); + } + + IntegerReader integerReader() { + return new IntegerReader(); + } + + TimestampMillisReader timestampMillisReader() { + return new TimestampMillisReader(); + } + + FixedWidthBinaryReader fixedWidthBinaryReader() { + return new FixedWidthBinaryReader(); + } + + FixedLengthDecimalReader fixedLengthDecimalReader() { + return new FixedLengthDecimalReader(); } + FixedSizeBinaryReader fixedSizeBinaryReader() { + return new FixedSizeBinaryReader(); + } + + VarWidthReader varWidthReader() { + return new VarWidthReader(); + } + + IntBackedDecimalReader intBackedDecimalReader() { + return new IntBackedDecimalReader(); + } + + LongBackedDecimalReader longBackedDecimalReader() { + return new LongBackedDecimalReader(); + } + + BooleanReader booleanReader() { + return new BooleanReader(); + } + + DictionaryIdReader dictionaryIdReader() { + return new DictionaryIdReader(); + } } diff --git a/arrow/src/test/java/org/apache/iceberg/arrow/ArrowSchemaUtilTest.java b/arrow/src/test/java/org/apache/iceberg/arrow/ArrowSchemaUtilTest.java index 7440ed33ec4c..99d0e3ed5b4a 100644 --- a/arrow/src/test/java/org/apache/iceberg/arrow/ArrowSchemaUtilTest.java +++ b/arrow/src/test/java/org/apache/iceberg/arrow/ArrowSchemaUtilTest.java @@ -56,6 +56,7 @@ public class ArrowSchemaUtilTest { private static final String STRUCT_FIELD = "st"; private static final String LIST_FIELD = "lt"; private static final String MAP_FIELD = "mt"; + private static final String UUID_FIELD = "uu"; @Test public void convertPrimitive() { @@ -74,7 +75,8 @@ public void convertPrimitive() { Types.NestedField.optional(12, LIST_FIELD, Types.ListType.ofOptional(13, Types.IntegerType.get())), Types.NestedField.required(14, MAP_FIELD, Types.MapType.ofOptional(15, 16, StringType.get(), IntegerType.get())), - Types.NestedField.optional(17, FIXED_WIDTH_BINARY_FIELD, Types.FixedType.ofLength(10))); + Types.NestedField.optional(17, FIXED_WIDTH_BINARY_FIELD, Types.FixedType.ofLength(10)), + Types.NestedField.optional(18, UUID_FIELD, Types.UUIDType.get())); org.apache.arrow.vector.types.pojo.Schema arrow = ArrowSchemaUtil.convert(iceberg); @@ -171,6 +173,10 @@ private void validate(Type iceberg, Field field, boolean optional) { Assert.assertEquals(MAP_FIELD, field.getName()); Assert.assertEquals(ArrowType.ArrowTypeID.Map, arrowType.getTypeID()); break; + case UUID: + Assert.assertEquals(UUID_FIELD, field.getName()); + Assert.assertEquals(ArrowType.FixedSizeBinary.TYPE_TYPE, arrowType.getTypeID()); + break; default: throw new UnsupportedOperationException("Check not implemented for type: " + iceberg); } diff --git a/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/ArrowReaderTest.java b/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/ArrowReaderTest.java index 68291738f333..43f9df519833 100644 --- a/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/ArrowReaderTest.java +++ b/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/ArrowReaderTest.java @@ -26,6 +26,7 @@ import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; +import java.time.LocalTime; import java.time.OffsetDateTime; import java.time.ZoneOffset; import java.time.temporal.ChronoUnit; @@ -34,6 +35,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.function.BiFunction; import java.util.stream.Collectors; @@ -41,9 +43,11 @@ import org.apache.arrow.vector.BitVector; import org.apache.arrow.vector.DateDayVector; import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.FixedSizeBinaryVector; import org.apache.arrow.vector.Float4Vector; import org.apache.arrow.vector.Float8Vector; import org.apache.arrow.vector.IntVector; +import org.apache.arrow.vector.TimeMicroVector; import org.apache.arrow.vector.TimeStampMicroTZVector; import org.apache.arrow.vector.TimeStampMicroVector; import org.apache.arrow.vector.VarBinaryVector; @@ -74,6 +78,8 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.UUIDUtil; +import org.assertj.core.api.Assertions; import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; @@ -112,7 +118,11 @@ public class ArrowReaderTest { "bytes_nullable", "date", "date_nullable", - "int_promotion" + "int_promotion", + "time", + "time_nullable", + "uuid", + "uuid_nullable" ); @Rule @@ -343,6 +353,7 @@ private void readAndCheckArrowResult( assertEquals(expectedTotalRows, totalRows); } + @SuppressWarnings("MethodLength") private void checkColumnarBatch( int expectedNumRows, List expectedRows, @@ -364,6 +375,7 @@ private void checkColumnarBatch( (records, i) -> records.get(i).getField("timestamp"), (array, i) -> timestampFromMicros(array.getLong(i)) ); + checkColumnarArrayValues( expectedNumRows, expectedRows, batch, columnNameToIndex.get("timestamp_nullable"), columnSet, "timestamp_nullable", @@ -484,6 +496,33 @@ private void checkColumnarBatch( (records, i) -> records.get(i).getField("int_promotion"), ColumnVector::getInt ); + + checkColumnarArrayValues( + expectedNumRows, expectedRows, batch, columnNameToIndex.get("uuid"), + columnSet, "uuid", + (records, i) -> records.get(i).getField("uuid"), + ColumnVector::getBinary + + ); + checkColumnarArrayValues( + expectedNumRows, expectedRows, batch, columnNameToIndex.get("uuid_nullable"), + columnSet, "uuid_nullable", + (records, i) -> records.get(i).getField("uuid_nullable"), + ColumnVector::getBinary + ); + + checkColumnarArrayValues( + expectedNumRows, expectedRows, batch, columnNameToIndex.get("time"), + columnSet, "time", + (records, i) -> records.get(i).getField("time"), + (array, i) -> LocalTime.ofNanoOfDay(array.getLong(i) * 1000) + ); + checkColumnarArrayValues( + expectedNumRows, expectedRows, batch, columnNameToIndex.get("time_nullable"), + columnSet, "time_nullable", + (records, i) -> records.get(i).getField("time_nullable"), + (array, i) -> LocalTime.ofNanoOfDay(array.getLong(i) * 1000) + ); } private static void checkColumnarArrayValues( @@ -500,7 +539,9 @@ private static void checkColumnarArrayValues( for (int i = 0; i < expectedNumRows; i++) { Object expectedValue = expectedValueExtractor.apply(expectedRows, i); Object actualValue = vectorValueExtractor.apply(columnVector, i); - assertEquals("Row#" + i + " mismatches", expectedValue, actualValue); + // we need to use assertThat() here because it does a java.util.Objects.deepEquals() and that + // is relevant for byte[] + Assertions.assertThat(actualValue).as("Row#" + i + " mismatches").isEqualTo(expectedValue); } } } @@ -539,7 +580,11 @@ private void writeTable(boolean constantRecords) throws Exception { Types.NestedField.optional(18, "bytes_nullable", Types.BinaryType.get()), Types.NestedField.required(19, "date", Types.DateType.get()), Types.NestedField.optional(20, "date_nullable", Types.DateType.get()), - Types.NestedField.required(21, "int_promotion", Types.IntegerType.get()) + Types.NestedField.required(21, "int_promotion", Types.IntegerType.get()), + Types.NestedField.required(22, "time", Types.TimeType.get()), + Types.NestedField.optional(23, "time_nullable", Types.TimeType.get()), + Types.NestedField.required(24, "uuid", Types.UUIDType.get()), + Types.NestedField.optional(25, "uuid_nullable", Types.UUIDType.get()) ); PartitionSpec spec = PartitionSpec.builderFor(schema) @@ -617,7 +662,15 @@ private static org.apache.arrow.vector.types.pojo.Schema createExpectedArrowSche new Field( "date_nullable", new FieldType(true, MinorType.DATEDAY.getType(), null), null), new Field( - "int_promotion", new FieldType(false, MinorType.INT.getType(), null), null) + "int_promotion", new FieldType(false, MinorType.INT.getType(), null), null), + new Field( + "time", new FieldType(false, MinorType.TIMEMICRO.getType(), null), null), + new Field( + "time_nullable", new FieldType(true, MinorType.TIMEMICRO.getType(), null), null), + new Field( + "uuid", new FieldType(false, new ArrowType.FixedSizeBinary(16), null), null), + new Field( + "uuid_nullable", new FieldType(true, new ArrowType.FixedSizeBinary(16), null), null) ); List filteredFields = allFields.stream() .filter(f -> columnSet.contains(f.getName())) @@ -650,6 +703,12 @@ private List createIncrementalRecordsForDate(Schema schema, Local rec.setField("date", LocalDate.of(2020, 1, 1).plus(i, ChronoUnit.DAYS)); rec.setField("date_nullable", LocalDate.of(2020, 1, 1).plus(i, ChronoUnit.DAYS)); rec.setField("int_promotion", i); + rec.setField("time", LocalTime.of(11, i)); + rec.setField("time_nullable", LocalTime.of(11, i)); + ByteBuffer bb = UUIDUtil.convertToByteBuffer(UUID.randomUUID()); + byte[] uuid = bb.array(); + rec.setField("uuid", uuid); + rec.setField("uuid_nullable", uuid); records.add(rec); } return records; @@ -680,6 +739,12 @@ private List createConstantRecordsForDate(Schema schema, LocalDat rec.setField("date", LocalDate.of(2020, 1, 1)); rec.setField("date_nullable", LocalDate.of(2020, 1, 1)); rec.setField("int_promotion", 1); + rec.setField("time", LocalTime.of(11, 30)); + rec.setField("time_nullable", LocalTime.of(11, 30)); + ByteBuffer bb = UUIDUtil.convertToByteBuffer(UUID.fromString("abcd91cf-08d0-4223-b145-f64030b3077f")); + byte[] uuid = bb.array(); + rec.setField("uuid", uuid); + rec.setField("uuid_nullable", uuid); records.add(rec); } return records; @@ -753,6 +818,10 @@ private void checkAllVectorTypes(VectorSchemaRoot root, Set columnSet) { assertEqualsForField(root, columnSet, "bytes_nullable", VarBinaryVector.class); assertEqualsForField(root, columnSet, "date", DateDayVector.class); assertEqualsForField(root, columnSet, "date_nullable", DateDayVector.class); + assertEqualsForField(root, columnSet, "time", TimeMicroVector.class); + assertEqualsForField(root, columnSet, "time_nullable", TimeMicroVector.class); + assertEqualsForField(root, columnSet, "uuid", FixedSizeBinaryVector.class); + assertEqualsForField(root, columnSet, "uuid_nullable", FixedSizeBinaryVector.class); assertEqualsForField(root, columnSet, "int_promotion", IntVector.class); } @@ -875,6 +944,29 @@ private void checkAllVectorValues( (records, i) -> records.get(i).getField("int_promotion"), (vector, i) -> ((IntVector) vector).get(i) ); + + checkVectorValues( + expectedNumRows, expectedRows, root, columnSet, "uuid", + (records, i) -> records.get(i).getField("uuid"), + (vector, i) -> ((FixedSizeBinaryVector) vector).get(i) + ); + + checkVectorValues( + expectedNumRows, expectedRows, root, columnSet, "uuid_nullable", + (records, i) -> records.get(i).getField("uuid_nullable"), + (vector, i) -> ((FixedSizeBinaryVector) vector).get(i) + ); + + checkVectorValues( + expectedNumRows, expectedRows, root, columnSet, "time", + (records, i) -> records.get(i).getField("time"), + (vector, i) -> LocalTime.ofNanoOfDay(((TimeMicroVector) vector).get(i) * 1000) + ); + checkVectorValues( + expectedNumRows, expectedRows, root, columnSet, "time_nullable", + (records, i) -> records.get(i).getField("time_nullable"), + (vector, i) -> LocalTime.ofNanoOfDay(((TimeMicroVector) vector).get(i) * 1000) + ); } private static void checkVectorValues( @@ -891,7 +983,9 @@ private static void checkVectorValues( for (int i = 0; i < expectedNumRows; i++) { Object expectedValue = expectedValueExtractor.apply(expectedRows, i); Object actualValue = vectorValueExtractor.apply(vector, i); - assertEquals("Row#" + i + " mismatches", expectedValue, actualValue); + // we need to use assertThat() here because it does a java.util.Objects.deepEquals() and that + // is relevant for byte[] + Assertions.assertThat(actualValue).as("Row#" + i + " mismatches").isEqualTo(expectedValue); } } } diff --git a/aws/src/integration/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalogTest.java b/aws/src/integration/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalogTest.java new file mode 100644 index 000000000000..8d4273c1625f --- /dev/null +++ b/aws/src/integration/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalogTest.java @@ -0,0 +1,289 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.aws.dynamodb; + +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ForkJoinPool; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.iceberg.AssertHelpers; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.aws.AwsClientFactories; +import org.apache.iceberg.aws.AwsClientFactory; +import org.apache.iceberg.aws.AwsIntegTestUtil; +import org.apache.iceberg.aws.AwsProperties; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.Types; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; +import software.amazon.awssdk.services.dynamodb.DynamoDbClient; +import software.amazon.awssdk.services.dynamodb.model.DeleteTableRequest; +import software.amazon.awssdk.services.dynamodb.model.GetItemRequest; +import software.amazon.awssdk.services.dynamodb.model.GetItemResponse; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.NoSuchKeyException; + +public class DynamoDbCatalogTest { + + private static final ForkJoinPool POOL = new ForkJoinPool(16); + private static final Schema SCHEMA = new Schema(Types.NestedField.required(1, "id", Types.StringType.get())); + + private static String catalogTableName; + private static DynamoDbClient dynamo; + private static S3Client s3; + private static DynamoDbCatalog catalog; + private static String testBucket; + + @BeforeClass + public static void beforeClass() { + catalogTableName = genRandomName(); + AwsClientFactory clientFactory = AwsClientFactories.defaultFactory(); + dynamo = clientFactory.dynamo(); + s3 = clientFactory.s3(); + catalog = new DynamoDbCatalog(); + testBucket = AwsIntegTestUtil.testBucketName(); + catalog.initialize("test", ImmutableMap.of( + AwsProperties.DYNAMODB_TABLE_NAME, catalogTableName, + CatalogProperties.WAREHOUSE_LOCATION, "s3://" + testBucket + "/" + genRandomName())); + } + + @AfterClass + public static void afterClass() { + dynamo.deleteTable(DeleteTableRequest.builder().tableName(catalogTableName).build()); + } + + @Test + public void testCreateNamespace() { + Namespace namespace = Namespace.of(genRandomName()); + catalog.createNamespace(namespace); + GetItemResponse response = dynamo.getItem(GetItemRequest.builder() + .tableName(catalogTableName) + .key(DynamoDbCatalog.namespacePrimaryKey(namespace)) + .build()); + Assert.assertTrue("namespace must exist", response.hasItem()); + Assert.assertEquals("namespace must be stored in DynamoDB", + namespace.toString(), response.item().get("namespace").s()); + + AssertHelpers.assertThrows("should not create duplicated namespace", + AlreadyExistsException.class, + "already exists", + () -> catalog.createNamespace(namespace)); + } + + @Test + public void testCreateNamespaceBadName() { + AssertHelpers.assertThrows("should not create namespace with empty level", + ValidationException.class, + "must not be empty", + () -> catalog.createNamespace(Namespace.of("a", "", "b"))); + + AssertHelpers.assertThrows("should not create namespace with dot in level", + ValidationException.class, + "must not contain dot", + () -> catalog.createNamespace(Namespace.of("a", "b.c"))); + } + + @Test + public void testListSubNamespaces() { + Namespace parent = Namespace.of(genRandomName()); + List namespaceList = IntStream.range(0, 3) + .mapToObj(i -> Namespace.of(parent.toString(), genRandomName())) + .collect(Collectors.toList()); + catalog.createNamespace(parent); + namespaceList.forEach(ns -> catalog.createNamespace(ns)); + Assert.assertEquals(4, catalog.listNamespaces(parent).size()); + } + + @Test + public void testNamespaceProperties() { + Namespace namespace = Namespace.of(genRandomName()); + Map properties = Maps.newHashMap(); + properties.put("key1", "val1"); + properties.put("key2", "val2"); + catalog.createNamespace(namespace, properties); + Assert.assertEquals(properties, catalog.loadNamespaceMetadata(namespace)); + + properties.put("key3", "val3"); + properties.put("key2", "val2-1"); + catalog.setProperties(namespace, properties); + Assert.assertEquals(properties, catalog.loadNamespaceMetadata(namespace)); + + properties.remove("key3"); + catalog.removeProperties(namespace, Sets.newHashSet("key3")); + Assert.assertEquals(properties, catalog.loadNamespaceMetadata(namespace)); + } + + @Test + public void testCreateTable() { + Namespace namespace = Namespace.of(genRandomName()); + catalog.createNamespace(namespace); + TableIdentifier tableIdentifier = TableIdentifier.of(namespace, genRandomName()); + catalog.createTable(tableIdentifier, SCHEMA); + GetItemResponse response = dynamo.getItem(GetItemRequest.builder() + .tableName(catalogTableName) + .key(DynamoDbCatalog.tablePrimaryKey(tableIdentifier)) + .build()); + Assert.assertTrue("table must exist", response.hasItem()); + Assert.assertEquals("table must be stored in DynamoDB with table identifier as partition key", + tableIdentifier.toString(), response.item().get("identifier").s()); + Assert.assertEquals("table must be stored in DynamoDB with namespace as sort key", + namespace.toString(), response.item().get("namespace").s()); + + AssertHelpers.assertThrows("should not create duplicated table", + AlreadyExistsException.class, + "already exists", + () -> catalog.createTable(tableIdentifier, SCHEMA)); + } + + @Test + public void testCreateTableBadName() { + Namespace namespace = Namespace.of(genRandomName()); + catalog.createNamespace(namespace); + AssertHelpers.assertThrows("should not create table name with empty namespace", + ValidationException.class, + "Table namespace must not be empty", + () -> catalog.createTable(TableIdentifier.of(Namespace.empty(), "a"), SCHEMA)); + + AssertHelpers.assertThrows("should not create table name with dot", + ValidationException.class, + "must not contain dot", + () -> catalog.createTable(TableIdentifier.of(namespace, "a.b"), SCHEMA)); + } + + @Test + public void testListTable() { + Namespace namespace = Namespace.of(genRandomName()); + catalog.createNamespace(namespace); + List tableIdentifiers = IntStream.range(0, 3) + .mapToObj(i -> TableIdentifier.of(namespace, genRandomName())) + .collect(Collectors.toList()); + tableIdentifiers.forEach(id -> catalog.createTable(id, SCHEMA)); + Assert.assertEquals(3, catalog.listTables(namespace).size()); + } + + @Test + public void testDropTable() { + Namespace namespace = Namespace.of(genRandomName()); + catalog.createNamespace(namespace); + TableIdentifier tableIdentifier = TableIdentifier.of(namespace, genRandomName()); + catalog.createTable(tableIdentifier, SCHEMA); + String metadataLocation = dynamo.getItem(GetItemRequest.builder() + .tableName(catalogTableName) + .key(DynamoDbCatalog.tablePrimaryKey(tableIdentifier)).build()) + .item().get("p.metadata_location").s(); + catalog.dropTable(tableIdentifier, true); + Assert.assertFalse("table entry should not exist in dynamo", + dynamo.getItem(GetItemRequest.builder() + .tableName(catalogTableName) + .key(DynamoDbCatalog.tablePrimaryKey(tableIdentifier)).build()) + .hasItem()); + AssertHelpers.assertThrows("metadata location should be deleted", + NoSuchKeyException.class, + () -> s3.headObject(HeadObjectRequest.builder() + .bucket(testBucket) + .key(metadataLocation.substring(testBucket.length() + 6)) // s3:// + end slash + .build())); + } + + @Test + public void testRenameTable() { + Namespace namespace = Namespace.of(genRandomName()); + catalog.createNamespace(namespace); + Namespace namespace2 = Namespace.of(genRandomName()); + catalog.createNamespace(namespace2); + TableIdentifier tableIdentifier = TableIdentifier.of(namespace, genRandomName()); + catalog.createTable(tableIdentifier, SCHEMA); + TableIdentifier tableIdentifier2 = TableIdentifier.of(namespace2, genRandomName()); + + AssertHelpers.assertThrows("should not be able to rename a table not exist", + NoSuchTableException.class, + "does not exist", + () -> catalog.renameTable(TableIdentifier.of(namespace, "a"), tableIdentifier2)); + + AssertHelpers.assertThrows("should not be able to rename an existing table", + AlreadyExistsException.class, + "already exists", + () -> catalog.renameTable(tableIdentifier, tableIdentifier)); + + String metadataLocation = dynamo.getItem(GetItemRequest.builder() + .tableName(catalogTableName) + .key(DynamoDbCatalog.tablePrimaryKey(tableIdentifier)).build()) + .item().get("p.metadata_location").s(); + + catalog.renameTable(tableIdentifier, tableIdentifier2); + + String metadataLocation2 = dynamo.getItem(GetItemRequest.builder() + .tableName(catalogTableName) + .key(DynamoDbCatalog.tablePrimaryKey(tableIdentifier2)).build()) + .item().get("p.metadata_location").s(); + + Assert.assertEquals("metadata location should be copied to new table entry", + metadataLocation, metadataLocation2); + } + + @Test + public void testUpdateTable() { + Namespace namespace = Namespace.of(genRandomName()); + catalog.createNamespace(namespace); + TableIdentifier tableIdentifier = TableIdentifier.of(namespace, genRandomName()); + catalog.createTable(tableIdentifier, SCHEMA); + Table table = catalog.loadTable(tableIdentifier); + table.updateSchema().addColumn("data", Types.StringType.get()).commit(); + table.refresh(); + Assert.assertEquals(2, table.schema().columns().size()); + } + + @Test + public void testConcurrentCommits() throws Exception { + Namespace namespace = Namespace.of(genRandomName()); + catalog.createNamespace(namespace); + TableIdentifier tableIdentifier = TableIdentifier.of(namespace, genRandomName()); + catalog.createTable(tableIdentifier, SCHEMA); + Table table = catalog.loadTable(tableIdentifier); + POOL.submit(() -> IntStream.range(0, 16).parallel() + .forEach(i -> { + try { + table.updateSchema().addColumn(genRandomName(), Types.StringType.get()).commit(); + } catch (Exception e) { + // ignore + } + })).get(); + + Assert.assertEquals(2, table.schema().columns().size()); + } + + private static String genRandomName() { + return UUID.randomUUID().toString().replace("-", ""); + } +} diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueCatalogCommitFailureTest.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueCatalogCommitFailureTest.java index bcdb4c3ff952..f4530315df23 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueCatalogCommitFailureTest.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/GlueCatalogCommitFailureTest.java @@ -76,14 +76,9 @@ public void testConcurrentModificationExceptionDoesNotCheckCommitStatus() { GlueTableOperations spyOps = Mockito.spy(ops); failCommitAndThrowException(spyOps, ConcurrentModificationException.builder().build()); - try { - spyOps.commit(metadataV2, metadataV1); - } catch (CommitFailedException e) { - Assert.assertTrue("Exception message should mention concurrent exception", - e.getMessage().contains("Glue detected concurrent update")); - Assert.assertTrue("Cause should be concurrent modification exception", - e.getCause() instanceof ConcurrentModificationException); - } + AssertHelpers.assertThrows("GlueCatalog should fail on concurrent modifications", + ConcurrentModificationException.class, "Glue detected concurrent update", + () -> spyOps.commit(metadataV2, metadataV1)); Mockito.verify(spyOps, Mockito.times(0)).refresh(); ops.refresh(); diff --git a/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java b/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java index ffc30c1f7aa7..0985559b8895 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java @@ -21,6 +21,7 @@ import java.io.Serializable; import java.util.Map; +import org.apache.iceberg.aws.dynamodb.DynamoDbCatalog; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.util.PropertyUtil; import software.amazon.awssdk.services.s3.model.ObjectCannedACL; @@ -130,6 +131,12 @@ public class AwsProperties implements Serializable { */ public static final String S3FILEIO_ACL = "s3.acl"; + /** + * DynamoDB table name for {@link DynamoDbCatalog} + */ + public static final String DYNAMODB_TABLE_NAME = "dynamodb.table-name"; + public static final String DYNAMODB_TABLE_NAME_DEFAULT = "iceberg"; + /** * The implementation class of {@link AwsClientFactory} to customize AWS client configurations. * If set, all AWS clients will be initialized by the specified factory. @@ -180,6 +187,8 @@ public class AwsProperties implements Serializable { private String glueCatalogId; private boolean glueCatalogSkipArchive; + private String dynamoDbTableName; + public AwsProperties() { this.s3FileIoSseType = S3FILEIO_SSE_TYPE_NONE; this.s3FileIoSseKey = null; @@ -193,6 +202,8 @@ public AwsProperties() { this.glueCatalogId = null; this.glueCatalogSkipArchive = GLUE_CATALOG_SKIP_ARCHIVE_DEFAULT; + + this.dynamoDbTableName = DYNAMODB_TABLE_NAME_DEFAULT; } public AwsProperties(Map properties) { @@ -236,6 +247,9 @@ public AwsProperties(Map properties) { this.s3FileIoAcl = ObjectCannedACL.fromValue(aclType); Preconditions.checkArgument(s3FileIoAcl == null || !s3FileIoAcl.equals(ObjectCannedACL.UNKNOWN_TO_SDK_VERSION), "Cannot support S3 CannedACL " + aclType); + + this.dynamoDbTableName = PropertyUtil.propertyAsString(properties, DYNAMODB_TABLE_NAME, + DYNAMODB_TABLE_NAME_DEFAULT); } public String s3FileIoSseType() { @@ -317,4 +331,12 @@ public ObjectCannedACL s3FileIoAcl() { public void setS3FileIoAcl(ObjectCannedACL acl) { this.s3FileIoAcl = acl; } + + public String dynamoDbTableName() { + return dynamoDbTableName; + } + + public void setDynamoDbTableName(String name) { + this.dynamoDbTableName = name; + } } diff --git a/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java b/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java new file mode 100644 index 000000000000..f9bef1514604 --- /dev/null +++ b/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java @@ -0,0 +1,613 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.aws.dynamodb; + +import java.io.Closeable; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.BaseMetastoreCatalog; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.aws.AwsClientFactories; +import org.apache.iceberg.aws.AwsProperties; +import org.apache.iceberg.aws.s3.S3FileIO; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.NamespaceNotEmptyException; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.Tasks; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.dynamodb.DynamoDbClient; +import software.amazon.awssdk.services.dynamodb.model.AttributeDefinition; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; +import software.amazon.awssdk.services.dynamodb.model.BillingMode; +import software.amazon.awssdk.services.dynamodb.model.ConditionalCheckFailedException; +import software.amazon.awssdk.services.dynamodb.model.CreateTableRequest; +import software.amazon.awssdk.services.dynamodb.model.Delete; +import software.amazon.awssdk.services.dynamodb.model.DeleteItemRequest; +import software.amazon.awssdk.services.dynamodb.model.DescribeTableRequest; +import software.amazon.awssdk.services.dynamodb.model.DescribeTableResponse; +import software.amazon.awssdk.services.dynamodb.model.GetItemRequest; +import software.amazon.awssdk.services.dynamodb.model.GetItemResponse; +import software.amazon.awssdk.services.dynamodb.model.GlobalSecondaryIndex; +import software.amazon.awssdk.services.dynamodb.model.KeySchemaElement; +import software.amazon.awssdk.services.dynamodb.model.KeyType; +import software.amazon.awssdk.services.dynamodb.model.Projection; +import software.amazon.awssdk.services.dynamodb.model.ProjectionType; +import software.amazon.awssdk.services.dynamodb.model.Put; +import software.amazon.awssdk.services.dynamodb.model.PutItemRequest; +import software.amazon.awssdk.services.dynamodb.model.QueryRequest; +import software.amazon.awssdk.services.dynamodb.model.QueryResponse; +import software.amazon.awssdk.services.dynamodb.model.ResourceNotFoundException; +import software.amazon.awssdk.services.dynamodb.model.ScalarAttributeType; +import software.amazon.awssdk.services.dynamodb.model.TableStatus; +import software.amazon.awssdk.services.dynamodb.model.TransactWriteItem; +import software.amazon.awssdk.services.dynamodb.model.TransactWriteItemsRequest; +import software.amazon.awssdk.services.dynamodb.model.UpdateItemRequest; + +/** + * DynamoDB implementation of Iceberg catalog + */ +public class DynamoDbCatalog extends BaseMetastoreCatalog implements Closeable, SupportsNamespaces, Configurable { + + private static final Logger LOG = LoggerFactory.getLogger(DynamoDbCatalog.class); + private static final int CATALOG_TABLE_CREATION_WAIT_ATTEMPTS_MAX = 5; + static final Joiner COMMA = Joiner.on(','); + + private static final String GSI_NAMESPACE_IDENTIFIER = "namespace-identifier"; + private static final String COL_IDENTIFIER = "identifier"; + private static final String COL_IDENTIFIER_NAMESPACE = "NAMESPACE"; + private static final String COL_NAMESPACE = "namespace"; + private static final String PROPERTY_COL_PREFIX = "p."; + private static final String PROPERTY_DEFAULT_LOCATION = "default_location"; + private static final String COL_CREATED_AT = "created_at"; + private static final String COL_UPDATED_AT = "updated_at"; + + // field used for optimistic locking + static final String COL_VERSION = "v"; + + private DynamoDbClient dynamo; + private Configuration hadoopConf; + private String catalogName; + private String warehousePath; + private AwsProperties awsProperties; + private FileIO fileIO; + + public DynamoDbCatalog() { + } + + @Override + public void initialize(String name, Map properties) { + initialize( + name, + properties.get(CatalogProperties.WAREHOUSE_LOCATION), + new AwsProperties(properties), + AwsClientFactories.from(properties).dynamo(), + initializeFileIO(properties)); + } + + @VisibleForTesting + void initialize(String name, String path, AwsProperties properties, DynamoDbClient client, FileIO io) { + this.catalogName = name; + this.awsProperties = properties; + this.warehousePath = cleanWarehousePath(path); + this.dynamo = client; + this.fileIO = io; + ensureCatalogTableExistsOrCreate(); + } + + @Override + public String name() { + return catalogName; + } + + @Override + protected TableOperations newTableOps(TableIdentifier tableIdentifier) { + validateTableIdentifier(tableIdentifier); + return new DynamoDbTableOperations(dynamo, awsProperties, catalogName, fileIO, tableIdentifier); + } + + @Override + protected String defaultWarehouseLocation(TableIdentifier tableIdentifier) { + validateTableIdentifier(tableIdentifier); + GetItemResponse response = dynamo.getItem(GetItemRequest.builder() + .tableName(awsProperties.dynamoDbTableName()) + .consistentRead(true) + .key(namespacePrimaryKey(tableIdentifier.namespace())) + .build()); + + if (!response.hasItem()) { + throw new NoSuchNamespaceException("Cannot find default warehouse location: namespace %s does not exist", + tableIdentifier.namespace()); + } + + String defaultLocationCol = toPropertyCol(PROPERTY_DEFAULT_LOCATION); + if (response.item().containsKey(defaultLocationCol)) { + return String.format("%s/%s", response.item().get(defaultLocationCol), tableIdentifier.name()); + } else { + return String.format("%s/%s.db/%s", warehousePath, tableIdentifier.namespace(), tableIdentifier.name()); + } + } + + @Override + public void createNamespace(Namespace namespace, Map metadata) { + validateNamespace(namespace); + Map values = namespacePrimaryKey(namespace); + setNewCatalogEntryMetadata(values); + metadata.forEach((key, value) -> values.put(toPropertyCol(key), AttributeValue.builder().s(value).build())); + + try { + dynamo.putItem(PutItemRequest.builder() + .tableName(awsProperties.dynamoDbTableName()) + .conditionExpression("attribute_not_exists(" + DynamoDbCatalog.COL_VERSION + ")") + .item(values) + .build()); + } catch (ConditionalCheckFailedException e) { + throw new AlreadyExistsException("Cannot create namespace %s: already exists", namespace); + } + } + + @Override + public List listNamespaces(Namespace namespace) throws NoSuchNamespaceException { + validateNamespace(namespace); + List namespaces = Lists.newArrayList(); + Map lastEvaluatedKey = null; + String condition = COL_IDENTIFIER + " = :identifier"; + Map conditionValues = Maps.newHashMap(); + conditionValues.put(":identifier", AttributeValue.builder().s(COL_IDENTIFIER_NAMESPACE).build()); + if (!namespace.isEmpty()) { + condition += " AND " + "begins_with(" + COL_NAMESPACE + ",:ns)"; + conditionValues.put(":ns", AttributeValue.builder().s(namespace.toString()).build()); + } + + do { + QueryResponse response = dynamo.query(QueryRequest.builder() + .tableName(awsProperties.dynamoDbTableName()) + .consistentRead(true) + .keyConditionExpression(condition) + .expressionAttributeValues(conditionValues) + .exclusiveStartKey(lastEvaluatedKey) + .build()); + + if (response.hasItems()) { + for (Map item : response.items()) { + String ns = item.get(COL_NAMESPACE).s(); + namespaces.add(Namespace.of(ns.split("\\."))); + } + } + + lastEvaluatedKey = response.lastEvaluatedKey(); + } while (!lastEvaluatedKey.isEmpty()); + + return namespaces; + } + + @Override + public Map loadNamespaceMetadata(Namespace namespace) throws NoSuchNamespaceException { + validateNamespace(namespace); + GetItemResponse response = dynamo.getItem(GetItemRequest.builder() + .tableName(awsProperties.dynamoDbTableName()) + .consistentRead(true) + .key(namespacePrimaryKey(namespace)) + .build()); + + if (!response.hasItem()) { + throw new NoSuchNamespaceException("Cannot find namespace %s", namespace); + } + + return response.item().entrySet().stream() + .filter(e -> isProperty(e.getKey())) + .collect(Collectors.toMap(e -> toPropertyKey(e.getKey()), e -> e.getValue().s())); + } + + @Override + public boolean dropNamespace(Namespace namespace) throws NamespaceNotEmptyException { + validateNamespace(namespace); + if (!listTables(namespace).isEmpty()) { + throw new NamespaceNotEmptyException("Cannot delete non-empty namespace %s", namespace); + } + + try { + dynamo.deleteItem(DeleteItemRequest.builder() + .tableName(awsProperties.dynamoDbTableName()) + .key(namespacePrimaryKey(namespace)) + .conditionExpression("attribute_exists(" + namespace + ")") + .build()); + return true; + } catch (ConditionalCheckFailedException e) { + return false; + } + } + + @Override + public boolean setProperties(Namespace namespace, Map properties) throws NoSuchNamespaceException { + List updateParts = Lists.newArrayList(); + Map attributeNames = Maps.newHashMap(); + Map attributeValues = Maps.newHashMap(); + int idx = 0; + for (Map.Entry property : properties.entrySet()) { + String attributeValue = ":v" + idx; + String attributeKey = "#k" + idx; + idx++; + updateParts.add(attributeKey + " = " + attributeValue); + attributeNames.put(attributeKey, toPropertyCol(property.getKey())); + attributeValues.put(attributeValue, AttributeValue.builder().s(property.getValue()).build()); + } + + updateCatalogEntryMetadata(updateParts, attributeValues); + String updateExpression = "SET " + COMMA.join(updateParts); + return updateProperties(namespace, updateExpression, attributeValues, attributeNames); + } + + @Override + public boolean removeProperties(Namespace namespace, Set properties) throws NoSuchNamespaceException { + List removeParts = Lists.newArrayList(properties.iterator()); + Map attributeNames = Maps.newHashMap(); + Map attributeValues = Maps.newHashMap(); + int idx = 0; + for (String property : properties) { + String attributeKey = "#k" + idx; + idx++; + removeParts.add(attributeKey); + attributeNames.put(attributeKey, toPropertyCol(property)); + } + + List updateParts = Lists.newArrayList(); + updateCatalogEntryMetadata(updateParts, attributeValues); + String updateExpression = "REMOVE " + COMMA.join(removeParts) + " SET " + COMMA.join(updateParts); + return updateProperties(namespace, updateExpression, attributeValues, attributeNames); + } + + @Override + public List listTables(Namespace namespace) { + List identifiers = Lists.newArrayList(); + Map lastEvaluatedKey; + String condition = COL_NAMESPACE + " = :ns"; + Map conditionValues = ImmutableMap.of( + ":ns", AttributeValue.builder().s(namespace.toString()).build()); + do { + QueryResponse response = dynamo.query(QueryRequest.builder() + .tableName(awsProperties.dynamoDbTableName()) + .indexName(GSI_NAMESPACE_IDENTIFIER) + .keyConditionExpression(condition) + .expressionAttributeValues(conditionValues) + .build()); + + if (response.hasItems()) { + for (Map item : response.items()) { + String identifier = item.get(COL_IDENTIFIER).s(); + if (!COL_IDENTIFIER_NAMESPACE.equals(identifier)) { + identifiers.add(TableIdentifier.of(identifier.split("\\."))); + } + } + } + + lastEvaluatedKey = response.lastEvaluatedKey(); + } while (!lastEvaluatedKey.isEmpty()); + return identifiers; + } + + @Override + public boolean dropTable(TableIdentifier identifier, boolean purge) { + Map key = tablePrimaryKey(identifier); + try { + GetItemResponse response = dynamo.getItem(GetItemRequest.builder() + .tableName(awsProperties.dynamoDbTableName()) + .consistentRead(true) + .key(key) + .build()); + + if (!response.hasItem()) { + throw new NoSuchTableException("Cannot find table %s to drop", identifier); + } + + TableOperations ops = newTableOps(identifier); + TableMetadata lastMetadata = ops.current(); + dynamo.deleteItem(DeleteItemRequest.builder() + .tableName(awsProperties.dynamoDbTableName()) + .key(tablePrimaryKey(identifier)) + .conditionExpression(COL_VERSION + " = :v") + .expressionAttributeValues(ImmutableMap.of(":v", response.item().get(COL_VERSION))) + .build()); + LOG.info("Successfully dropped table {} from DynamoDb catalog", identifier); + + if (purge && lastMetadata != null) { + CatalogUtil.dropTableData(ops.io(), lastMetadata); + LOG.info("Table {} data purged", identifier); + } + + LOG.info("Dropped table: {}", identifier); + return true; + } catch (ConditionalCheckFailedException e) { + LOG.error("Cannot complete drop table operation for {}: commit conflict", identifier, e); + return false; + } catch (Exception e) { + LOG.error("Cannot complete drop table operation for {}: unexpected exception", identifier, e); + throw e; + } + } + + @Override + public void renameTable(TableIdentifier from, TableIdentifier to) { + Map fromKey = tablePrimaryKey(from); + Map toKey = tablePrimaryKey(to); + + GetItemResponse fromResponse = dynamo.getItem(GetItemRequest.builder() + .tableName(awsProperties.dynamoDbTableName()) + .consistentRead(true) + .key(fromKey) + .build()); + + if (!fromResponse.hasItem()) { + throw new NoSuchTableException("Cannot rename table %s to %s: %s does not exist", from, to, from); + } + + GetItemResponse toResponse = dynamo.getItem(GetItemRequest.builder() + .tableName(awsProperties.dynamoDbTableName()) + .consistentRead(true) + .key(toKey) + .build()); + + if (toResponse.hasItem()) { + throw new AlreadyExistsException("Cannot rename table %s to %s: %s already exists", from, to, to); + } + + fromResponse.item().entrySet().stream() + .filter(e -> isProperty(e.getKey())) + .forEach(e -> toKey.put(e.getKey(), e.getValue())); + + setNewCatalogEntryMetadata(toKey); + + dynamo.transactWriteItems(TransactWriteItemsRequest.builder() + .transactItems( + TransactWriteItem.builder() + .delete(Delete.builder() + .tableName(awsProperties.dynamoDbTableName()) + .key(fromKey) + .conditionExpression(COL_VERSION + " = :v") + .expressionAttributeValues(ImmutableMap.of(":v", fromResponse.item().get(COL_VERSION))) + .build()) + .build(), + TransactWriteItem.builder() + .put(Put.builder() + .tableName(awsProperties.dynamoDbTableName()) + .item(toKey) + .conditionExpression("attribute_not_exists(" + COL_VERSION + ")") + .build()) + .build()) + .build()); + + LOG.info("Successfully renamed table from {} to {}", from, to); + } + + @Override + public void setConf(Configuration conf) { + hadoopConf = conf; + } + + @Override + public Configuration getConf() { + return hadoopConf; + } + + @Override + public void close() throws IOException { + dynamo.close(); + } + + /** + * The property used to set a default location for tables in a namespace. + * Call {@link #setProperties(Namespace, Map)} to set a path value using this property for a namespace, + * then all tables in the namespace will have default table root path under that given path. + * @return default location property key + */ + public static String defaultLocationProperty() { + return PROPERTY_DEFAULT_LOCATION; + } + + static String toPropertyCol(String propertyKey) { + return PROPERTY_COL_PREFIX + propertyKey; + } + + static boolean isProperty(String dynamoCol) { + return dynamoCol.startsWith(PROPERTY_COL_PREFIX); + } + + static String toPropertyKey(String propertyCol) { + return propertyCol.substring(PROPERTY_COL_PREFIX.length()); + } + + static Map namespacePrimaryKey(Namespace namespace) { + Map key = Maps.newHashMap(); + key.put(COL_IDENTIFIER, AttributeValue.builder().s(COL_IDENTIFIER_NAMESPACE).build()); + key.put(COL_NAMESPACE, AttributeValue.builder().s(namespace.toString()).build()); + return key; + } + + static Map tablePrimaryKey(TableIdentifier identifier) { + Map key = Maps.newHashMap(); + key.put(COL_IDENTIFIER, AttributeValue.builder().s(identifier.toString()).build()); + key.put(COL_NAMESPACE, AttributeValue.builder().s(identifier.namespace().toString()).build()); + return key; + } + + static void setNewCatalogEntryMetadata(Map values) { + String current = Long.toString(System.currentTimeMillis()); + values.put(COL_CREATED_AT, AttributeValue.builder().n(current).build()); + values.put(COL_UPDATED_AT, AttributeValue.builder().n(current).build()); + values.put(COL_VERSION, AttributeValue.builder().s(UUID.randomUUID().toString()).build()); + } + + static void updateCatalogEntryMetadata(List updateParts, Map attributeValues) { + updateParts.add(COL_UPDATED_AT + " = :uat"); + attributeValues.put(":uat", AttributeValue.builder().n(Long.toString(System.currentTimeMillis())).build()); + updateParts.add(COL_VERSION + " = :uv"); + attributeValues.put(":uv", AttributeValue.builder().s(UUID.randomUUID().toString()).build()); + } + + private FileIO initializeFileIO(Map properties) { + String fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL); + if (fileIOImpl == null) { + FileIO io = new S3FileIO(); + io.initialize(properties); + return io; + } else { + return CatalogUtil.loadFileIO(fileIOImpl, properties, hadoopConf); + } + } + + private String cleanWarehousePath(String path) { + Preconditions.checkArgument(path != null && path.length() > 0, + "Cannot initialize DynamoDbCatalog because warehousePath must not be null"); + int len = path.length(); + if (path.charAt(len - 1) == '/') { + return path.substring(0, len - 1); + } else { + return path; + } + } + + private void validateNamespace(Namespace namespace) { + for (String level : namespace.levels()) { + ValidationException.check(level != null && !level.isEmpty(), + "Namespace level must not be empty: %s", namespace); + ValidationException.check(!level.contains("."), + "Namespace level must not contain dot, but found %s in %s", level, namespace); + } + } + + private void validateTableIdentifier(TableIdentifier identifier) { + validateNamespace(identifier.namespace()); + ValidationException.check(identifier.hasNamespace(), + "Table namespace must not be empty: %s", identifier); + String tableName = identifier.name(); + ValidationException.check(!tableName.contains("."), + "Table name must not contain dot: %s", tableName); + } + + private boolean dynamoDbTableExists(String tableName) { + try { + dynamo.describeTable(DescribeTableRequest.builder() + .tableName(tableName) + .build()); + return true; + } catch (ResourceNotFoundException e) { + return false; + } + } + + private void ensureCatalogTableExistsOrCreate() { + if (dynamoDbTableExists(awsProperties.dynamoDbTableName())) { + return; + } + + LOG.info("DynamoDb catalog table {} not found, trying to create", awsProperties.dynamoDbTableName()); + dynamo.createTable(CreateTableRequest.builder() + .tableName(awsProperties.dynamoDbTableName()) + .keySchema( + KeySchemaElement.builder().attributeName(COL_IDENTIFIER).keyType(KeyType.HASH).build(), + KeySchemaElement.builder().attributeName(COL_NAMESPACE).keyType(KeyType.RANGE).build()) + .attributeDefinitions( + AttributeDefinition.builder().attributeName(COL_IDENTIFIER).attributeType(ScalarAttributeType.S).build(), + AttributeDefinition.builder().attributeName(COL_NAMESPACE).attributeType(ScalarAttributeType.S).build()) + .globalSecondaryIndexes(GlobalSecondaryIndex.builder() + .indexName(GSI_NAMESPACE_IDENTIFIER) + .keySchema( + KeySchemaElement.builder().attributeName(COL_NAMESPACE).keyType(KeyType.HASH).build(), + KeySchemaElement.builder().attributeName(COL_IDENTIFIER).keyType(KeyType.RANGE).build()) + .projection(Projection.builder().projectionType(ProjectionType.KEYS_ONLY).build()) + .build()) + .billingMode(BillingMode.PAY_PER_REQUEST) + .build()); + + // wait for the dynamo table to complete provisioning, which takes around 10 seconds + Tasks.foreach(awsProperties.dynamoDbTableName()) + .retry(CATALOG_TABLE_CREATION_WAIT_ATTEMPTS_MAX) + .throwFailureWhenFinished() + .onlyRetryOn(IllegalStateException.class) + .run(this::checkTableActive); + } + + private void checkTableActive(String tableName) { + try { + DescribeTableResponse response = dynamo.describeTable(DescribeTableRequest.builder() + .tableName(tableName) + .build()); + TableStatus currentStatus = response.table().tableStatus(); + if (!currentStatus.equals(TableStatus.ACTIVE)) { + throw new IllegalStateException(String.format("Dynamo catalog table %s is not active, current status: %s", + tableName, currentStatus)); + } + } catch (ResourceNotFoundException e) { + throw new IllegalStateException(String.format("Cannot find Dynamo catalog table %s", tableName)); + } + } + + private boolean updateProperties(Namespace namespace, String updateExpression, + Map attributeValues, + Map attributeNames) { + validateNamespace(namespace); + Map key = namespacePrimaryKey(namespace); + try { + GetItemResponse response = dynamo.getItem(GetItemRequest.builder() + .tableName(awsProperties.dynamoDbTableName()) + .consistentRead(true) + .key(key) + .build()); + + if (!response.hasItem()) { + throw new NoSuchNamespaceException("Cannot find namespace %s", namespace); + } + + attributeValues.put(":v", response.item().get(COL_VERSION)); + dynamo.updateItem(UpdateItemRequest.builder() + .tableName(awsProperties.dynamoDbTableName()) + .key(key) + .conditionExpression(COL_VERSION + " = :v") + .updateExpression(updateExpression) + .expressionAttributeValues(attributeValues) + .expressionAttributeNames(attributeNames) + .build()); + return true; + } catch (ConditionalCheckFailedException e) { + return false; + } + } +} diff --git a/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbTableOperations.java b/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbTableOperations.java new file mode 100644 index 000000000000..81157e91b3b2 --- /dev/null +++ b/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbTableOperations.java @@ -0,0 +1,215 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.aws.dynamodb; + +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; +import org.apache.iceberg.BaseMetastoreTableOperations; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.aws.AwsProperties; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.CommitStateUnknownException; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.dynamodb.DynamoDbClient; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; +import software.amazon.awssdk.services.dynamodb.model.ConditionalCheckFailedException; +import software.amazon.awssdk.services.dynamodb.model.GetItemRequest; +import software.amazon.awssdk.services.dynamodb.model.GetItemResponse; +import software.amazon.awssdk.services.dynamodb.model.PutItemRequest; +import software.amazon.awssdk.services.dynamodb.model.UpdateItemRequest; + +class DynamoDbTableOperations extends BaseMetastoreTableOperations { + + private static final Logger LOG = LoggerFactory.getLogger(DynamoDbTableOperations.class); + + private final DynamoDbClient dynamo; + private final AwsProperties awsProperties; + private final TableIdentifier tableIdentifier; + private final String fullTableName; + private final FileIO fileIO; + + DynamoDbTableOperations( + DynamoDbClient dynamo, + AwsProperties awsProperties, + String catalogName, + FileIO fileIO, + TableIdentifier tableIdentifier) { + this.dynamo = dynamo; + this.awsProperties = awsProperties; + this.fullTableName = String.format("%s.%s", catalogName, tableIdentifier); + this.tableIdentifier = tableIdentifier; + this.fileIO = fileIO; + } + + @Override + protected String tableName() { + return fullTableName; + } + + @Override + public FileIO io() { + return fileIO; + } + + @Override + protected void doRefresh() { + String metadataLocation = null; + GetItemResponse table = dynamo.getItem(GetItemRequest.builder() + .tableName(awsProperties.dynamoDbTableName()) + .consistentRead(true) + .key(DynamoDbCatalog.tablePrimaryKey(tableIdentifier)) + .build()); + if (table.hasItem()) { + metadataLocation = getMetadataLocation(table); + } else { + if (currentMetadataLocation() != null) { + throw new NoSuchTableException("Cannot find table %s after refresh, " + + "maybe another process deleted it or revoked your access permission", tableName()); + } + } + + refreshFromMetadataLocation(metadataLocation); + } + + @Override + protected void doCommit(TableMetadata base, TableMetadata metadata) { + String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1); + CommitStatus commitStatus = CommitStatus.FAILURE; + Map tableKey = DynamoDbCatalog.tablePrimaryKey(tableIdentifier); + try { + GetItemResponse table = dynamo.getItem(GetItemRequest.builder() + .tableName(awsProperties.dynamoDbTableName()) + .consistentRead(true) + .key(tableKey) + .build()); + checkMetadataLocation(table, base); + Map properties = prepareProperties(table, newMetadataLocation); + persistTable(tableKey, table, properties); + commitStatus = CommitStatus.SUCCESS; + } catch (ConditionalCheckFailedException e) { + throw new CommitFailedException(e, "Cannot commit %s: concurrent update detected", tableName()); + } catch (RuntimeException persistFailure) { + LOG.error("Confirming if commit to {} indeed failed to persist, attempting to reconnect and check.", + fullTableName, persistFailure); + commitStatus = checkCommitStatus(newMetadataLocation, metadata); + + switch (commitStatus) { + case SUCCESS: + break; + case FAILURE: + throw new CommitFailedException(persistFailure, + "Cannot commit %s due to unexpected exception", tableName()); + case UNKNOWN: + throw new CommitStateUnknownException(persistFailure); + } + } finally { + try { + if (commitStatus == CommitStatus.FAILURE) { + // if anything went wrong, clean up the uncommitted metadata file + io().deleteFile(newMetadataLocation); + } + } catch (RuntimeException e) { + LOG.error("Fail to cleanup metadata file at {}", newMetadataLocation, e); + throw e; + } + } + } + + private void checkMetadataLocation(GetItemResponse table, TableMetadata base) { + String dynamoMetadataLocation = table.hasItem() ? getMetadataLocation(table) : null; + String baseMetadataLocation = base != null ? base.metadataFileLocation() : null; + if (!Objects.equals(baseMetadataLocation, dynamoMetadataLocation)) { + throw new CommitFailedException( + "Cannot commit %s because base metadata location '%s' is not same as the current DynamoDb location '%s'", + tableName(), baseMetadataLocation, dynamoMetadataLocation); + } + } + + private String getMetadataLocation(GetItemResponse table) { + return table.item().get(DynamoDbCatalog.toPropertyCol(METADATA_LOCATION_PROP)).s(); + } + + private Map prepareProperties(GetItemResponse response, String newMetadataLocation) { + Map properties = response.hasItem() ? getProperties(response) : Maps.newHashMap(); + properties.put(TABLE_TYPE_PROP, ICEBERG_TABLE_TYPE_VALUE.toUpperCase(Locale.ENGLISH)); + properties.put(METADATA_LOCATION_PROP, newMetadataLocation); + if (currentMetadataLocation() != null && !currentMetadataLocation().isEmpty()) { + properties.put(PREVIOUS_METADATA_LOCATION_PROP, currentMetadataLocation()); + } + + return properties; + } + + private Map getProperties(GetItemResponse table) { + return table.item().entrySet().stream() + .filter(e -> DynamoDbCatalog.isProperty(e.getKey())) + .collect(Collectors.toMap(e -> DynamoDbCatalog.toPropertyKey(e.getKey()), e -> e.getValue().s())); + } + + void persistTable(Map tableKey, GetItemResponse table, Map parameters) { + if (table.hasItem()) { + LOG.debug("Committing existing DynamoDb catalog table: {}", tableName()); + List updateParts = Lists.newArrayList(); + Map attributeNames = Maps.newHashMap(); + Map attributeValues = Maps.newHashMap(); + int idx = 0; + for (Map.Entry property : parameters.entrySet()) { + String attributeValue = ":v" + idx; + String attributeKey = "#k" + idx; + idx++; + updateParts.add(attributeKey + " = " + attributeValue); + attributeNames.put(attributeKey, DynamoDbCatalog.toPropertyCol(property.getKey())); + attributeValues.put(attributeValue, AttributeValue.builder().s(property.getValue()).build()); + } + DynamoDbCatalog.updateCatalogEntryMetadata(updateParts, attributeValues); + String updateExpression = "SET " + DynamoDbCatalog.COMMA.join(updateParts); + attributeValues.put(":v", table.item().get(DynamoDbCatalog.COL_VERSION)); + dynamo.updateItem(UpdateItemRequest.builder() + .tableName(awsProperties.dynamoDbTableName()) + .key(tableKey) + .conditionExpression(DynamoDbCatalog.COL_VERSION + " = :v") + .updateExpression(updateExpression) + .expressionAttributeValues(attributeValues) + .expressionAttributeNames(attributeNames) + .build()); + } else { + LOG.debug("Committing new DynamoDb catalog table: {}", tableName()); + Map values = Maps.newHashMap(tableKey); + parameters.forEach((k, v) -> values.put(DynamoDbCatalog.toPropertyCol(k), + AttributeValue.builder().s(v).build())); + DynamoDbCatalog.setNewCatalogEntryMetadata(values); + + dynamo.putItem(PutItemRequest.builder() + .tableName(awsProperties.dynamoDbTableName()) + .item(values) + .conditionExpression("attribute_not_exists(" + DynamoDbCatalog.COL_VERSION + ")") + .build()); + } + } +} diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputFile.java b/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputFile.java index 418484927bb7..f4bfe0c02c57 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputFile.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputFile.java @@ -61,7 +61,7 @@ public PositionOutputStream createOrOverwrite() { try { return new S3OutputStream(client(), uri(), awsProperties()); } catch (IOException e) { - throw new UncheckedIOException("Filed to create output stream for location: " + uri(), e); + throw new UncheckedIOException("Failed to create output stream for location: " + uri(), e); } } diff --git a/build.gradle b/build.gradle index bacb2f1a249e..31f776b6f157 100644 --- a/build.gradle +++ b/build.gradle @@ -27,7 +27,7 @@ buildscript { classpath 'com.github.jengelman.gradle.plugins:shadow:5.0.0' classpath 'com.palantir.baseline:gradle-baseline-java:3.36.2' classpath 'com.palantir.gradle.gitversion:gradle-git-version:0.12.3' - classpath 'com.diffplug.spotless:spotless-plugin-gradle:3.14.0' + classpath 'com.diffplug.spotless:spotless-plugin-gradle:5.14.0' classpath 'gradle.plugin.org.inferred:gradle-processors:2.1.0' classpath 'me.champeau.gradle:jmh-gradle-plugin:0.4.8' } @@ -64,6 +64,10 @@ allprojects { mavenCentral() mavenLocal() } + project.ext { + Spark30Version = '3.0.3' + Spark31Version = '3.1.1' + } } subprojects { @@ -122,10 +126,11 @@ subprojects { compile 'org.slf4j:slf4j-api' compile 'com.github.stephenc.findbugs:findbugs-annotations' - testCompile 'junit:junit' + testCompile 'org.junit.vintage:junit-vintage-engine' + testCompile 'org.junit.jupiter:junit-jupiter' testCompile 'org.slf4j:slf4j-simple' - testCompile 'org.mockito:mockito-core' + testCompile 'org.assertj:assertj-core' } test { @@ -225,6 +230,7 @@ project(':iceberg-core') { exclude group: 'org.slf4j', module: 'slf4j-log4j12' } + testCompile "org.xerial:sqlite-jdbc" testCompile project(path: ':iceberg-api', configuration: 'testArtifacts') } } @@ -280,6 +286,7 @@ project(':iceberg-aws') { testCompile("com.adobe.testing:s3mock-junit4") { exclude module: "spring-boot-starter-logging" exclude module: "logback-classic" + exclude group: 'junit' } } @@ -327,9 +334,12 @@ project(':iceberg-flink') { testCompile "org.apache.flink:flink-core" testCompile "org.apache.flink:flink-runtime_2.12" testCompile "org.apache.flink:flink-table-planner-blink_2.12" - testCompile "org.apache.flink:flink-test-utils-junit" + testCompile ("org.apache.flink:flink-test-utils-junit") { + exclude group: 'junit' + } testCompile("org.apache.flink:flink-test-utils_2.12") { exclude group: "org.apache.curator", module: 'curator-test' + exclude group: 'junit' } testCompile project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') @@ -665,8 +675,8 @@ if (jdkVersion == '8') { exclude group: 'org.apache.hive', module: 'hive-exec' exclude group: 'org.apache.orc' } - testCompile("org.apache.tez:tez-dag:0.9.1") - testCompile("org.apache.tez:tez-mapreduce:0.9.1") + testCompile("org.apache.tez:tez-dag:0.10.1") + testCompile("org.apache.tez:tez-mapreduce:0.10.1") } test { @@ -857,7 +867,6 @@ project(':iceberg-spark') { } if (jdkVersion == '8') { - apply from: 'jmh.gradle' project(':iceberg-spark2') { configurations.all { @@ -977,6 +986,21 @@ if (jdkVersion == '8') { } project(':iceberg-spark3') { + sourceSets { + // Compile test source against Spark 3.1 and main classes compiled against Spark 3.0 + spark31 { + java.srcDir "$projectDir/src/test/java" + resources.srcDir "$projectDir/src/test/resources" + compileClasspath += sourceSets.test.output + sourceSets.main.output + runtimeClasspath += sourceSets.test.output + } + } + + configurations { + spark31Implementation.extendsFrom testImplementation + spark31RuntimeOnly.extendsFrom testRuntimeOnly + } + dependencies { compile project(':iceberg-api') compile project(':iceberg-common') @@ -989,7 +1013,7 @@ project(':iceberg-spark3') { compile project(':iceberg-spark') compileOnly "org.apache.avro:avro" - compileOnly("org.apache.spark:spark-hive_2.12") { + compileOnly("org.apache.spark:spark-hive_2.12:${project.ext.Spark30Version}") { exclude group: 'org.apache.avro', module: 'avro' exclude group: 'org.apache.arrow' } @@ -1002,9 +1026,16 @@ project(':iceberg-spark3') { testCompile project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') testCompile project(path: ':iceberg-api', configuration: 'testArtifacts') testCompile project(path: ':iceberg-data', configuration: 'testArtifacts') + testCompile "org.xerial:sqlite-jdbc" + + spark31Implementation("org.apache.spark:spark-hive_2.12:${project.ext.Spark31Version}") { + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'org.apache.arrow' + exclude group: 'junit' + } } - test { + tasks.withType(Test) { // For vectorized reads // Allow unsafe memory access to avoid the costly check arrow does to check if index is within bounds systemProperty("arrow.enable_unsafe_memory_access", "true") @@ -1013,8 +1044,17 @@ project(':iceberg-spark3') { systemProperty("arrow.enable_null_check_for_get", "false") // Vectorized reads need more memory - maxHeapSize '2500m' + maxHeapSize '2560m' } + + task testSpark31(type: Test) { + dependsOn classes + description = "Test against Spark 3.1" + testClassesDirs = sourceSets.spark31.output.classesDirs + classpath = sourceSets.spark31.runtimeClasspath + sourceSets.main.output + } + + test.dependsOn testSpark31 } project(":iceberg-spark3-extensions") { @@ -1022,7 +1062,21 @@ project(":iceberg-spark3-extensions") { apply plugin: 'scala' apply plugin: 'antlr' + sourceSets { + // Compile test source against Spark 3.1 and main classes compiled against Spark 3.0 + spark31 { + // Main source is in scala, but test source is only in java + java.srcDir "$projectDir/src/test/java" + resources.srcDir "$projectDir/src/test/resources" + compileClasspath += sourceSets.test.output + sourceSets.main.output + runtimeClasspath += sourceSets.test.output + } + } + configurations { + spark31Implementation.extendsFrom testImplementation + spark31RuntimeOnly.extendsFrom testRuntimeOnly + /* The Gradle Antlr plugin erroneously adds both antlr-build and runtime dependencies to the runtime path. This bug https://github.com/gradle/gradle/issues/820 exists because older versions of Antlr do not have separate @@ -1036,10 +1090,9 @@ project(":iceberg-spark3-extensions") { } dependencies { - compileOnly project(':iceberg-spark3') - compileOnly "org.scala-lang:scala-library" - compileOnly("org.apache.spark:spark-hive_2.12") { + compileOnly project(':iceberg-spark3') + compileOnly("org.apache.spark:spark-hive_2.12:${project.ext.Spark30Version}") { exclude group: 'org.apache.avro', module: 'avro' exclude group: 'org.apache.arrow' } @@ -1049,6 +1102,11 @@ project(":iceberg-spark3-extensions") { testCompile project(path: ':iceberg-spark', configuration: 'testArtifacts') testCompile project(path: ':iceberg-spark3', configuration: 'testArtifacts') + spark31Implementation("org.apache.spark:spark-hive_2.12:${project.ext.Spark31Version}") { + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'org.apache.arrow' + } + // Required because we remove antlr plugin dependencies from the compile configuration, see note above // We shade this in Spark3 Runtime to avoid issues with Spark's Antlr Runtime runtime "org.antlr:antlr4-runtime:4.7.1" @@ -1059,6 +1117,15 @@ project(":iceberg-spark3-extensions") { maxHeapSize = "64m" arguments += ['-visitor', '-package', 'org.apache.spark.sql.catalyst.parser.extensions'] } + + task testSpark31(type: Test) { + dependsOn classes + description = "Test against Spark 3.1" + testClassesDirs = sourceSets.spark31.output.classesDirs + classpath = sourceSets.spark31.runtimeClasspath + sourceSets.main.output + } + + test.dependsOn testSpark31 } project(':iceberg-spark3-runtime') { @@ -1071,6 +1138,12 @@ project(':iceberg-spark3-runtime') { java.srcDir "$projectDir/src/integration/java" resources.srcDir "$projectDir/src/integration/resources" } + spark31 { + java.srcDir "$projectDir/src/integration/java" + resources.srcDir "$projectDir/src/integration/resources" + compileClasspath += sourceSets.integration.output + runtimeClasspath += sourceSets.integration.output + } } configurations { @@ -1085,6 +1158,8 @@ project(':iceberg-spark3-runtime') { exclude group: 'javax.xml.bind' exclude group: 'javax.annotation' } + spark31Implementation.extendsFrom integrationImplementation + spark31CompileOnly.extendsFrom integrationCompileOnly } dependencies { @@ -1095,8 +1170,8 @@ project(':iceberg-spark3-runtime') { exclude group: 'com.google.code.findbugs', module: 'jsr305' } - integrationImplementation 'org.apache.spark:spark-hive_2.12' - integrationImplementation 'junit:junit' + integrationImplementation "org.apache.spark:spark-hive_2.12:${project.ext.Spark30Version}" + integrationImplementation 'org.junit.vintage:junit-vintage-engine' integrationImplementation 'org.slf4j:slf4j-simple' integrationImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') @@ -1106,6 +1181,8 @@ project(':iceberg-spark3-runtime') { // Not allowed on our classpath, only the runtime jar is allowed integrationCompileOnly project(':iceberg-spark3-extensions') integrationCompileOnly project(':iceberg-spark3') + + spark31Implementation "org.apache.spark:spark-hive_2.12:${project.ext.Spark31Version}" } shadowJar { @@ -1143,14 +1220,24 @@ project(':iceberg-spark3-runtime') { } task integrationTest(type: Test) { - description = "Test Spark3 Runtime Jar" + description = "Test Spark3 Runtime Jar against Spark 3.0" group = "verification" testClassesDirs = sourceSets.integration.output.classesDirs classpath = sourceSets.integration.runtimeClasspath + files(shadowJar.archiveFile.get().asFile.path) inputs.file(shadowJar.archiveFile.get().asFile.path) } integrationTest.dependsOn shadowJar - check.dependsOn integrationTest + + task spark31IntegrationTest(type: Test) { + dependsOn classes + description = "Test Spark3 Runtime Jar against Spark 3.1" + group = "verification" + testClassesDirs = sourceSets.spark31.output.classesDirs + classpath = sourceSets.spark31.runtimeClasspath + files(shadowJar.archiveFile.get().asFile.path) + } + spark31IntegrationTest.dependsOn shadowJar + + check.dependsOn integrationTest, spark31IntegrationTest jar { enabled = false @@ -1234,6 +1321,7 @@ String getJavadocVersion() { } } +apply from: 'jmh.gradle' apply from: 'baseline.gradle' apply from: 'deploy.gradle' apply from: 'tasks.gradle' diff --git a/core/src/main/java/org/apache/iceberg/AllDataFilesTable.java b/core/src/main/java/org/apache/iceberg/AllDataFilesTable.java index 13963b8e0437..6b34204ccc1e 100644 --- a/core/src/main/java/org/apache/iceberg/AllDataFilesTable.java +++ b/core/src/main/java/org/apache/iceberg/AllDataFilesTable.java @@ -84,6 +84,11 @@ private AllDataFilesTableScan(TableOperations ops, Table table, Schema schema, S this.fileSchema = fileSchema; } + @Override + protected String tableType() { + return MetadataTableType.ALL_DATA_FILES.name(); + } + @Override protected TableScan newRefinedScan(TableOperations ops, Table table, Schema schema, TableScanContext context) { return new AllDataFilesTableScan(ops, table, schema, fileSchema, context); diff --git a/core/src/main/java/org/apache/iceberg/AllEntriesTable.java b/core/src/main/java/org/apache/iceberg/AllEntriesTable.java index 66f8b4927135..c69429d80576 100644 --- a/core/src/main/java/org/apache/iceberg/AllEntriesTable.java +++ b/core/src/main/java/org/apache/iceberg/AllEntriesTable.java @@ -86,6 +86,11 @@ protected TableScan newRefinedScan(TableOperations ops, Table table, Schema sche return new Scan(ops, table, schema, context); } + @Override + protected String tableType() { + return MetadataTableType.ALL_ENTRIES.name(); + } + @Override public long targetSplitSize() { return tableOps().current().propertyAsLong( diff --git a/core/src/main/java/org/apache/iceberg/AllManifestsTable.java b/core/src/main/java/org/apache/iceberg/AllManifestsTable.java index dde78493e7c4..d6719343cbc0 100644 --- a/core/src/main/java/org/apache/iceberg/AllManifestsTable.java +++ b/core/src/main/java/org/apache/iceberg/AllManifestsTable.java @@ -106,6 +106,11 @@ public TableScan asOfTime(long timestampMillis) { throw new UnsupportedOperationException("Cannot select snapshot: all_manifests is for all snapshots"); } + @Override + protected String tableType() { + return MetadataTableType.ALL_MANIFESTS.name(); + } + @Override public long targetSplitSize() { return tableOps().current().propertyAsLong( diff --git a/core/src/main/java/org/apache/iceberg/BaseAllMetadataTableScan.java b/core/src/main/java/org/apache/iceberg/BaseAllMetadataTableScan.java index 5c0af71437da..a142a3d0092d 100644 --- a/core/src/main/java/org/apache/iceberg/BaseAllMetadataTableScan.java +++ b/core/src/main/java/org/apache/iceberg/BaseAllMetadataTableScan.java @@ -36,6 +36,26 @@ abstract class BaseAllMetadataTableScan extends BaseTableScan { super(ops, table, schema, context); } + /** + * Type of scan being performed, such as {@link MetadataTableType#ALL_DATA_FILES} when scanning + * a table's {@link org.apache.iceberg.AllDataFilesTable}. + *

    + * Used for logging and error messages. + */ + protected abstract String tableType(); + + @Override + public TableScan appendsBetween(long fromSnapshotId, long toSnapshotId) { + throw new UnsupportedOperationException( + String.format("Cannot incrementally scan table of type %s", tableType())); + } + + @Override + public TableScan appendsAfter(long fromSnapshotId) { + throw new UnsupportedOperationException( + String.format("Cannot incrementally scan table of type %s", tableType())); + } + @Override public CloseableIterable planFiles() { LOG.info("Scanning metadata table {} with filter {}.", table(), filter()); diff --git a/core/src/main/java/org/apache/iceberg/BaseMetadataTable.java b/core/src/main/java/org/apache/iceberg/BaseMetadataTable.java index 28a5a59089da..81ecaefb599a 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMetadataTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetadataTable.java @@ -88,6 +88,11 @@ public void refresh() { table().refresh(); } + @Override + public Map schemas() { + return ImmutableMap.of(TableMetadata.INITIAL_SCHEMA_ID, schema()); + } + @Override public PartitionSpec spec() { return spec; diff --git a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java index e81a148627ac..10b570c8bcf7 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java @@ -140,6 +140,10 @@ protected void requestRefresh() { this.shouldRefresh = true; } + protected void disableRefresh() { + this.shouldRefresh = false; + } + protected String writeNewMetadata(TableMetadata metadata, int newVersion) { String newTableMetadataFilePath = newTableMetadataFilePath(metadata, newVersion); OutputFile newMetadataLocation = io().newOutputFile(newTableMetadataFilePath); diff --git a/core/src/main/java/org/apache/iceberg/BaseRowDelta.java b/core/src/main/java/org/apache/iceberg/BaseRowDelta.java index 7a86df2adbbe..485609f8a46c 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRowDelta.java +++ b/core/src/main/java/org/apache/iceberg/BaseRowDelta.java @@ -19,14 +19,13 @@ package org.apache.iceberg; -import java.util.Set; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.util.CharSequenceSet; class BaseRowDelta extends MergingSnapshotProducer implements RowDelta { private Long startingSnapshotId = null; // check all versions by default - private final Set referencedDataFiles = CharSequenceSet.empty(); + private final CharSequenceSet referencedDataFiles = CharSequenceSet.empty(); private boolean validateDeletes = false; private Expression conflictDetectionFilter = null; private boolean caseSensitive = true; diff --git a/core/src/main/java/org/apache/iceberg/BaseSnapshot.java b/core/src/main/java/org/apache/iceberg/BaseSnapshot.java index 95e6d621f238..2641e3775af4 100644 --- a/core/src/main/java/org/apache/iceberg/BaseSnapshot.java +++ b/core/src/main/java/org/apache/iceberg/BaseSnapshot.java @@ -43,6 +43,7 @@ class BaseSnapshot implements Snapshot { private final String manifestListLocation; private final String operation; private final Map summary; + private final Integer schemaId; // lazily initialized private transient List allManifests = null; @@ -56,9 +57,10 @@ class BaseSnapshot implements Snapshot { */ BaseSnapshot(FileIO io, long snapshotId, + Integer schemaId, String... manifestFiles) { this(io, snapshotId, null, System.currentTimeMillis(), null, null, - Lists.transform(Arrays.asList(manifestFiles), + schemaId, Lists.transform(Arrays.asList(manifestFiles), path -> new GenericManifestFile(io.newInputFile(path), 0))); } @@ -69,6 +71,7 @@ class BaseSnapshot implements Snapshot { long timestampMillis, String operation, Map summary, + Integer schemaId, String manifestList) { this.io = io; this.sequenceNumber = sequenceNumber; @@ -77,6 +80,7 @@ class BaseSnapshot implements Snapshot { this.timestampMillis = timestampMillis; this.operation = operation; this.summary = summary; + this.schemaId = schemaId; this.manifestListLocation = manifestList; } @@ -86,8 +90,9 @@ class BaseSnapshot implements Snapshot { long timestampMillis, String operation, Map summary, + Integer schemaId, List dataManifests) { - this(io, INITIAL_SEQUENCE_NUMBER, snapshotId, parentId, timestampMillis, operation, summary, null); + this(io, INITIAL_SEQUENCE_NUMBER, snapshotId, parentId, timestampMillis, operation, summary, schemaId, null); this.allManifests = dataManifests; } @@ -121,6 +126,11 @@ public Map summary() { return summary; } + @Override + public Integer schemaId() { + return schemaId; + } + private void cacheManifests() { if (allManifests == null) { // if manifests isn't set, then the snapshotFile is set and should be read to get the list @@ -222,7 +232,8 @@ public boolean equals(Object o) { return this.snapshotId == other.snapshotId() && Objects.equal(this.parentId, other.parentId()) && this.sequenceNumber == other.sequenceNumber() && - this.timestampMillis == other.timestampMillis(); + this.timestampMillis == other.timestampMillis() && + Objects.equal(this.schemaId, other.schemaId()); } return false; @@ -234,7 +245,8 @@ public int hashCode() { this.snapshotId, this.parentId, this.sequenceNumber, - this.timestampMillis + this.timestampMillis, + this.schemaId ); } @@ -246,6 +258,7 @@ public String toString() { .add("operation", operation) .add("summary", summary) .add("manifest-list", manifestListLocation) + .add("schema-id", schemaId) .toString(); } } diff --git a/core/src/main/java/org/apache/iceberg/BaseTable.java b/core/src/main/java/org/apache/iceberg/BaseTable.java index f7e7540a365a..b9886f0a1e68 100644 --- a/core/src/main/java/org/apache/iceberg/BaseTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseTable.java @@ -69,6 +69,11 @@ public Schema schema() { return ops.current().schema(); } + @Override + public Map schemas() { + return ops.current().schemasById(); + } + @Override public PartitionSpec spec() { return ops.current().spec(); diff --git a/core/src/main/java/org/apache/iceberg/BaseTransaction.java b/core/src/main/java/org/apache/iceberg/BaseTransaction.java index 36cf53bd0dbd..8c5d498b92e2 100644 --- a/core/src/main/java/org/apache/iceberg/BaseTransaction.java +++ b/core/src/main/java/org/apache/iceberg/BaseTransaction.java @@ -526,6 +526,11 @@ public Schema schema() { return current.schema(); } + @Override + public Map schemas() { + return current.schemasById(); + } + @Override public PartitionSpec spec() { return current.spec(); diff --git a/core/src/main/java/org/apache/iceberg/BaseUpdatePartitionSpec.java b/core/src/main/java/org/apache/iceberg/BaseUpdatePartitionSpec.java index e1fe84be1dc3..80f8188002d5 100644 --- a/core/src/main/java/org/apache/iceberg/BaseUpdatePartitionSpec.java +++ b/core/src/main/java/org/apache/iceberg/BaseUpdatePartitionSpec.java @@ -142,13 +142,26 @@ public BaseUpdatePartitionSpec addField(String name, Term term) { PartitionField newField = new PartitionField( sourceTransform.first(), assignFieldId(), name, sourceTransform.second()); - checkForRedundantAddedPartitions(newField); + if (newField.name() == null) { + String partitionName = PartitionSpecVisitor.visit(schema, newField, PartitionNameGenerator.INSTANCE); + newField = new PartitionField(newField.sourceId(), newField.fieldId(), partitionName, newField.transform()); + } + checkForRedundantAddedPartitions(newField); transformToAddedField.put(validationKey, newField); - if (name != null) { - nameToAddedField.put(name, newField); + + PartitionField existingField = nameToField.get(newField.name()); + if (existingField != null) { + if (isVoidTransform(existingField)) { + // rename the old deleted field that is being replaced by the new field + renameField(existingField.name(), existingField.name() + "_" + existingField.fieldId()); + } else { + throw new IllegalArgumentException(String.format("Cannot add duplicate partition field name: %s", name)); + } } + nameToAddedField.put(newField.name(), newField); + adds.add(newField); return this; @@ -192,6 +205,12 @@ public BaseUpdatePartitionSpec removeField(Term term) { @Override public BaseUpdatePartitionSpec renameField(String name, String newName) { + PartitionField existingField = nameToField.get(newName); + if (existingField != null && isVoidTransform(existingField)) { + // rename the old deleted field that is being replaced by the new field + renameField(existingField.name(), existingField.name() + "_" + existingField.fieldId()); + } + PartitionField added = nameToAddedField.get(name); Preconditions.checkArgument(added == null, "Cannot rename newly added partition field: %s", name); @@ -228,14 +247,7 @@ public PartitionSpec apply() { } for (PartitionField newField : adds) { - String partitionName; - if (newField.name() != null) { - partitionName = newField.name(); - } else { - partitionName = PartitionSpecVisitor.visit(schema, newField, PartitionNameGenerator.INSTANCE); - } - - builder.add(newField.sourceId(), newField.fieldId(), partitionName, newField.transform()); + builder.add(newField.sourceId(), newField.fieldId(), newField.name(), newField.transform()); } return builder.build(); @@ -287,13 +299,13 @@ private static Map indexSpecByName(PartitionSpec spec) { } private static Map, PartitionField> indexSpecByTransform(PartitionSpec spec) { - ImmutableMap.Builder, PartitionField> builder = ImmutableMap.builder(); + Map, PartitionField> indexSpecs = Maps.newHashMap(); List fields = spec.fields(); for (PartitionField field : fields) { - builder.put(Pair.of(field.sourceId(), field.transform().toString()), field); + indexSpecs.put(Pair.of(field.sourceId(), field.transform().toString()), field); } - return builder.build(); + return indexSpecs; } private boolean isTimeTransform(PartitionField field) { @@ -352,6 +364,62 @@ public Boolean unknown(int fieldId, String sourceName, int sourceId, String tran } } + private boolean isVoidTransform(PartitionField field) { + return PartitionSpecVisitor.visit(schema, field, IsVoidTransform.INSTANCE); + } + + private static class IsVoidTransform implements PartitionSpecVisitor { + private static final IsVoidTransform INSTANCE = new IsVoidTransform(); + + private IsVoidTransform() { + } + + @Override + public Boolean identity(int fieldId, String sourceName, int sourceId) { + return false; + } + + @Override + public Boolean bucket(int fieldId, String sourceName, int sourceId, int numBuckets) { + return false; + } + + @Override + public Boolean truncate(int fieldId, String sourceName, int sourceId, int width) { + return false; + } + + @Override + public Boolean year(int fieldId, String sourceName, int sourceId) { + return false; + } + + @Override + public Boolean month(int fieldId, String sourceName, int sourceId) { + return false; + } + + @Override + public Boolean day(int fieldId, String sourceName, int sourceId) { + return false; + } + + @Override + public Boolean hour(int fieldId, String sourceName, int sourceId) { + return false; + } + + @Override + public Boolean alwaysNull(int fieldId, String sourceName, int sourceId) { + return true; + } + + @Override + public Boolean unknown(int fieldId, String sourceName, int sourceId, String transform) { + return false; + } + } + private static class PartitionNameGenerator implements PartitionSpecVisitor { private static final PartitionNameGenerator INSTANCE = new PartitionNameGenerator(); diff --git a/core/src/main/java/org/apache/iceberg/DataFilesTable.java b/core/src/main/java/org/apache/iceberg/DataFilesTable.java index ded26c07bdbf..13d28ad52b13 100644 --- a/core/src/main/java/org/apache/iceberg/DataFilesTable.java +++ b/core/src/main/java/org/apache/iceberg/DataFilesTable.java @@ -76,6 +76,18 @@ private FilesTableScan(TableOperations ops, Table table, Schema schema, Schema f this.fileSchema = fileSchema; } + @Override + public TableScan appendsBetween(long fromSnapshotId, long toSnapshotId) { + throw new UnsupportedOperationException( + String.format("Cannot incrementally scan table of type %s", MetadataTableType.FILES.name())); + } + + @Override + public TableScan appendsAfter(long fromSnapshotId) { + throw new UnsupportedOperationException( + String.format("Cannot incrementally scan table of type %s", MetadataTableType.FILES.name())); + } + @Override protected TableScan newRefinedScan(TableOperations ops, Table table, Schema schema, TableScanContext context) { return new FilesTableScan(ops, table, schema, fileSchema, context); diff --git a/core/src/main/java/org/apache/iceberg/DoubleFieldMetrics.java b/core/src/main/java/org/apache/iceberg/DoubleFieldMetrics.java new file mode 100644 index 000000000000..8185f3a9f2a5 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/DoubleFieldMetrics.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg; + +/** + * Iceberg internally tracked field level metrics, used by Parquet and ORC writers only. + *

    + * Parquet/ORC keeps track of most metrics in file statistics, and only NaN counter is actually tracked by writers. + * This wrapper ensures that metrics not being updated by those writers will not be incorrectly used, by throwing + * exceptions when they are accessed. + */ +public class DoubleFieldMetrics extends FieldMetrics { + + private DoubleFieldMetrics(int id, long valueCount, long nanValueCount, Double lowerBound, Double upperBound) { + super(id, valueCount, 0L, nanValueCount, lowerBound, upperBound); + } + + public static class Builder { + private final int id; + private long valueCount = 0; + private long nanValueCount = 0; + private double lowerBound = Double.POSITIVE_INFINITY; + private double upperBound = Double.NEGATIVE_INFINITY; + + public Builder(int id) { + this.id = id; + } + + public void addValue(double value) { + this.valueCount++; + if (Double.isNaN(value)) { + this.nanValueCount++; + } else { + if (Double.compare(value, lowerBound) < 0) { + this.lowerBound = value; + } + if (Double.compare(value, upperBound) > 0) { + this.upperBound = value; + } + } + } + + public DoubleFieldMetrics build() { + boolean hasBound = valueCount - nanValueCount > 0; + return new DoubleFieldMetrics(id, valueCount, nanValueCount, + hasBound ? lowerBound : null, hasBound ? upperBound : null); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/FieldMetrics.java b/core/src/main/java/org/apache/iceberg/FieldMetrics.java index d67faa94f1ab..effcb78cdf58 100644 --- a/core/src/main/java/org/apache/iceberg/FieldMetrics.java +++ b/core/src/main/java/org/apache/iceberg/FieldMetrics.java @@ -20,25 +20,23 @@ package org.apache.iceberg; -import java.nio.ByteBuffer; - /** * Iceberg internally tracked field level metrics. */ -public class FieldMetrics { +public class FieldMetrics { private final int id; private final long valueCount; private final long nullValueCount; private final long nanValueCount; - private final ByteBuffer lowerBound; - private final ByteBuffer upperBound; + private final T lowerBound; + private final T upperBound; public FieldMetrics(int id, long valueCount, long nullValueCount, long nanValueCount, - ByteBuffer lowerBound, - ByteBuffer upperBound) { + T lowerBound, + T upperBound) { this.id = id; this.valueCount = valueCount; this.nullValueCount = nullValueCount; @@ -78,14 +76,21 @@ public long nanValueCount() { /** * Returns the lower bound value of this field. */ - public ByteBuffer lowerBound() { + public T lowerBound() { return lowerBound; } /** * Returns the upper bound value of this field. */ - public ByteBuffer upperBound() { + public T upperBound() { return upperBound; } + + /** + * Returns if the metrics has bounds (i.e. there is at least non-null value for this field) + */ + public boolean hasBounds() { + return upperBound != null; + } } diff --git a/core/src/main/java/org/apache/iceberg/FloatFieldMetrics.java b/core/src/main/java/org/apache/iceberg/FloatFieldMetrics.java index 873138750685..5854b0c57393 100644 --- a/core/src/main/java/org/apache/iceberg/FloatFieldMetrics.java +++ b/core/src/main/java/org/apache/iceberg/FloatFieldMetrics.java @@ -19,8 +19,6 @@ package org.apache.iceberg; -import java.nio.ByteBuffer; - /** * Iceberg internally tracked field level metrics, used by Parquet and ORC writers only. *

    @@ -28,35 +26,45 @@ * This wrapper ensures that metrics not being updated by those writers will not be incorrectly used, by throwing * exceptions when they are accessed. */ -public class FloatFieldMetrics extends FieldMetrics { - - /** - * Constructor for creating a FieldMetrics with only NaN counter. - * @param id field id being tracked by the writer - * @param nanValueCount number of NaN values, will only be non-0 for double or float field. - */ - public FloatFieldMetrics(int id, - long nanValueCount) { - super(id, 0L, 0L, nanValueCount, null, null); - } +public class FloatFieldMetrics extends FieldMetrics { - @Override - public long valueCount() { - throw new IllegalStateException("Shouldn't access this method, as this metric is tracked in file statistics. "); + private FloatFieldMetrics(int id, long valueCount, long nanValueCount, Float lowerBound, Float upperBound) { + super(id, valueCount, 0L, nanValueCount, lowerBound, upperBound); } - @Override - public long nullValueCount() { - throw new IllegalStateException("Shouldn't access this method, as this metric is tracked in file statistics. "); + public Builder builderFor(int id) { + return new Builder(id); } - @Override - public ByteBuffer lowerBound() { - throw new IllegalStateException("Shouldn't access this method, as this metric is tracked in file statistics. "); - } + public static class Builder { + private final int id; + private long valueCount = 0; + private long nanValueCount = 0; + private float lowerBound = Float.POSITIVE_INFINITY; + private float upperBound = Float.NEGATIVE_INFINITY; + + public Builder(int id) { + this.id = id; + } + + public void addValue(float value) { + this.valueCount++; + if (Float.isNaN(value)) { + this.nanValueCount++; + } else { + if (Float.compare(value, lowerBound) < 0) { + this.lowerBound = value; + } + if (Float.compare(value, upperBound) > 0) { + this.upperBound = value; + } + } + } - @Override - public ByteBuffer upperBound() { - throw new IllegalStateException("Shouldn't access this method, as this metric is tracked in file statistics. "); + public FloatFieldMetrics build() { + boolean hasBound = valueCount - nanValueCount > 0; + return new FloatFieldMetrics(id, valueCount, nanValueCount, + hasBound ? lowerBound : null, hasBound ? upperBound : null); + } } } diff --git a/core/src/main/java/org/apache/iceberg/GenericManifestFile.java b/core/src/main/java/org/apache/iceberg/GenericManifestFile.java index 716f41b3938f..1a95149a8af3 100644 --- a/core/src/main/java/org/apache/iceberg/GenericManifestFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericManifestFile.java @@ -20,6 +20,7 @@ package org.apache.iceberg; import java.io.Serializable; +import java.nio.ByteBuffer; import java.util.Arrays; import java.util.List; import java.util.function.Function; @@ -33,6 +34,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Objects; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ByteBuffers; public class GenericManifestFile implements ManifestFile, StructLike, IndexedRecord, SchemaConstructable, Serializable { @@ -58,6 +60,7 @@ public class GenericManifestFile private Long existingRowsCount = null; private Long deletedRowsCount = null; private PartitionFieldSummary[] partitions = null; + private byte[] keyMetadata = null; /** * Used by Avro reflection to instantiate this class when reading manifest files. @@ -101,13 +104,14 @@ public GenericManifestFile(org.apache.avro.Schema avroSchema) { this.deletedRowsCount = null; this.partitions = null; this.fromProjectionPos = null; + this.keyMetadata = null; } public GenericManifestFile(String path, long length, int specId, ManifestContent content, long sequenceNumber, long minSequenceNumber, Long snapshotId, int addedFilesCount, long addedRowsCount, int existingFilesCount, long existingRowsCount, int deletedFilesCount, long deletedRowsCount, - List partitions) { + List partitions, ByteBuffer keyMetadata) { this.avroSchema = AVRO_SCHEMA; this.manifestPath = path; this.length = length; @@ -124,6 +128,7 @@ public GenericManifestFile(String path, long length, int specId, ManifestContent this.deletedRowsCount = deletedRowsCount; this.partitions = partitions == null ? null : partitions.toArray(new PartitionFieldSummary[0]); this.fromProjectionPos = null; + this.keyMetadata = ByteBuffers.toByteArray(keyMetadata); } /** @@ -154,6 +159,7 @@ private GenericManifestFile(GenericManifestFile toCopy) { this.partitions = null; } this.fromProjectionPos = toCopy.fromProjectionPos; + this.keyMetadata = toCopy.keyMetadata == null ? null : Arrays.copyOf(toCopy.keyMetadata, toCopy.keyMetadata.length); } /** @@ -245,6 +251,11 @@ public List partitions() { return partitions == null ? null : Arrays.asList(partitions); } + @Override + public ByteBuffer keyMetadata() { + return keyMetadata == null ? null : ByteBuffer.wrap(keyMetadata); + } + @Override public int size() { return ManifestFile.schema().columns().size(); @@ -291,6 +302,8 @@ public Object get(int i) { return deletedRowsCount; case 13: return partitions(); + case 14: + return keyMetadata(); default: throw new UnsupportedOperationException("Unknown field ordinal: " + pos); } @@ -349,6 +362,9 @@ public void set(int i, T value) { this.partitions = value == null ? null : ((List) value).toArray(new PartitionFieldSummary[0]); return; + case 14: + this.keyMetadata = ByteBuffers.toByteArray((ByteBuffer) value); + return; default: // ignore the object, it must be from a newer version of the format } @@ -399,6 +415,7 @@ public String toString() { .add("deleted_data_files_count", deletedFilesCount) .add("deleted_rows_count", deletedRowsCount) .add("partitions", partitions) + .add("key_metadata", keyMetadata == null ? "null" : "(redacted)") .toString(); } @@ -418,7 +435,7 @@ private CopyBuilder(ManifestFile toCopy) { toCopy.sequenceNumber(), toCopy.minSequenceNumber(), toCopy.snapshotId(), toCopy.addedFilesCount(), toCopy.addedRowsCount(), toCopy.existingFilesCount(), toCopy.existingRowsCount(), toCopy.deletedFilesCount(), toCopy.deletedRowsCount(), - copyList(toCopy.partitions(), PartitionFieldSummary::copy)); + copyList(toCopy.partitions(), PartitionFieldSummary::copy), toCopy.keyMetadata()); } } diff --git a/core/src/main/java/org/apache/iceberg/HistoryTable.java b/core/src/main/java/org/apache/iceberg/HistoryTable.java index b731ebb2e3ec..9b607ccbd6be 100644 --- a/core/src/main/java/org/apache/iceberg/HistoryTable.java +++ b/core/src/main/java/org/apache/iceberg/HistoryTable.java @@ -74,7 +74,7 @@ private DataTask task(TableScan scan) { private class HistoryScan extends StaticTableScan { HistoryScan(TableOperations ops, Table table) { - super(ops, table, HISTORY_SCHEMA, HistoryTable.this::task); + super(ops, table, HISTORY_SCHEMA, HistoryTable.this.metadataTableType().name(), HistoryTable.this::task); } @Override diff --git a/core/src/main/java/org/apache/iceberg/ManifestEntriesTable.java b/core/src/main/java/org/apache/iceberg/ManifestEntriesTable.java index 5e006e0f031a..6b434fa1ed14 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestEntriesTable.java +++ b/core/src/main/java/org/apache/iceberg/ManifestEntriesTable.java @@ -77,6 +77,18 @@ private EntriesTableScan(TableOperations ops, Table table, Schema schema, TableS super(ops, table, schema, context); } + @Override + public TableScan appendsBetween(long fromSnapshotId, long toSnapshotId) { + throw new UnsupportedOperationException( + String.format("Cannot incrementally scan table of type %s", MetadataTableType.ENTRIES.name())); + } + + @Override + public TableScan appendsAfter(long fromSnapshotId) { + throw new UnsupportedOperationException( + String.format("Cannot incrementally scan table of type %s", MetadataTableType.ENTRIES.name())); + } + @Override protected TableScan newRefinedScan(TableOperations ops, Table table, Schema schema, TableScanContext context) { diff --git a/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java b/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java index 506d2579a7f0..a81fd28d8310 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java +++ b/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java @@ -68,7 +68,7 @@ public String partition() { private final Map specsById; private final PartitionSet deleteFilePartitions; private final PartitionSet dropPartitions; - private final Set deletePaths = CharSequenceSet.empty(); + private final CharSequenceSet deletePaths = CharSequenceSet.empty(); private Expression deleteExpression = Expressions.alwaysFalse(); private long minSequenceNumber = 0; private boolean hasPathOnlyDeletes = false; @@ -216,17 +216,18 @@ SnapshotSummary.Builder buildSummary(Iterable manifests) { * * @param manifests a set of filtered manifests */ + @SuppressWarnings("CollectionUndefinedEquality") private void validateRequiredDeletes(ManifestFile... manifests) { if (failMissingDeletePaths) { - Set deletedFiles = deletedFiles(manifests); + CharSequenceSet deletedFiles = deletedFiles(manifests); ValidationException.check(deletedFiles.containsAll(deletePaths), "Missing required files to delete: %s", COMMA.join(Iterables.filter(deletePaths, path -> !deletedFiles.contains(path)))); } } - private Set deletedFiles(ManifestFile[] manifests) { - Set deletedFiles = CharSequenceSet.empty(); + private CharSequenceSet deletedFiles(ManifestFile[] manifests) { + CharSequenceSet deletedFiles = CharSequenceSet.empty(); if (manifests != null) { for (ManifestFile manifest : manifests) { @@ -337,6 +338,7 @@ private boolean canContainDeletedFiles(ManifestFile manifest) { return canContainExpressionDeletes || canContainDroppedPartitions || canContainDroppedFiles || canContainDropBySeq; } + @SuppressWarnings("CollectionUndefinedEquality") private boolean manifestHasDeletedFiles( StrictMetricsEvaluator metricsEvaluator, ManifestReader reader) { boolean isDelete = reader.isDeleteManifestReader(); @@ -364,6 +366,7 @@ private boolean manifestHasDeletedFiles( return hasDeletedFiles; } + @SuppressWarnings("CollectionUndefinedEquality") private ManifestFile filterManifestWithDeletedFiles( StrictMetricsEvaluator metricsEvaluator, ManifestFile manifest, ManifestReader reader) { boolean isDelete = reader.isDeleteManifestReader(); diff --git a/core/src/main/java/org/apache/iceberg/ManifestWriter.java b/core/src/main/java/org/apache/iceberg/ManifestWriter.java index b03d68ed217e..a03d18a8a1a0 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestWriter.java +++ b/core/src/main/java/org/apache/iceberg/ManifestWriter.java @@ -156,7 +156,7 @@ public ManifestFile toManifestFile() { long minSeqNumber = minSequenceNumber != null ? minSequenceNumber : UNASSIGNED_SEQ; return new GenericManifestFile(file.location(), writer.length(), specId, content(), UNASSIGNED_SEQ, minSeqNumber, snapshotId, - addedFiles, addedRows, existingFiles, existingRows, deletedFiles, deletedRows, stats.summaries()); + addedFiles, addedRows, existingFiles, existingRows, deletedFiles, deletedRows, stats.summaries(), null); } @Override diff --git a/core/src/main/java/org/apache/iceberg/ManifestsTable.java b/core/src/main/java/org/apache/iceberg/ManifestsTable.java index c75daf62a2bc..d67b8330d21d 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestsTable.java +++ b/core/src/main/java/org/apache/iceberg/ManifestsTable.java @@ -81,7 +81,7 @@ protected DataTask task(TableScan scan) { private class ManifestsTableScan extends StaticTableScan { ManifestsTableScan(TableOperations ops, Table table) { - super(ops, table, SNAPSHOT_SCHEMA, ManifestsTable.this::task); + super(ops, table, SNAPSHOT_SCHEMA, ManifestsTable.this.metadataTableType().name(), ManifestsTable.this::task); } } diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index 45e4326bea9b..f5c61f550afd 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -40,6 +40,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Iterators; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.CharSequenceSet; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -296,8 +297,9 @@ protected void validateAddedDataFiles(TableMetadata base, Long startingSnapshotI } } + @SuppressWarnings("CollectionUndefinedEquality") protected void validateDataFilesExist(TableMetadata base, Long startingSnapshotId, - Set requiredDataFiles, boolean skipDeletes) { + CharSequenceSet requiredDataFiles, boolean skipDeletes) { // if there is no current table state, no files have been removed if (base.currentSnapshot() == null) { return; diff --git a/core/src/main/java/org/apache/iceberg/MetadataColumns.java b/core/src/main/java/org/apache/iceberg/MetadataColumns.java index 0761084dc215..a8eb2eb37821 100644 --- a/core/src/main/java/org/apache/iceberg/MetadataColumns.java +++ b/core/src/main/java/org/apache/iceberg/MetadataColumns.java @@ -36,6 +36,8 @@ private MetadataColumns() { Integer.MAX_VALUE - 1, "_file", Types.StringType.get(), "Path of the file in which a row is stored"); public static final NestedField ROW_POSITION = NestedField.required( Integer.MAX_VALUE - 2, "_pos", Types.LongType.get(), "Ordinal position of a row in the source data file"); + public static final NestedField IS_DELETED = NestedField.required( + Integer.MAX_VALUE - 3, "_deleted", Types.BooleanType.get(), "Whether the row has been deleted"); // IDs Integer.MAX_VALUE - (101-200) are used for reserved columns public static final NestedField DELETE_FILE_PATH = NestedField.required( @@ -47,7 +49,8 @@ private MetadataColumns() { private static final Map META_COLUMNS = ImmutableMap.of( FILE_PATH.name(), FILE_PATH, - ROW_POSITION.name(), ROW_POSITION); + ROW_POSITION.name(), ROW_POSITION, + IS_DELETED.name(), IS_DELETED); private static final Set META_IDS = META_COLUMNS.values().stream().map(NestedField::fieldId) .collect(ImmutableSet.toImmutableSet()); diff --git a/core/src/main/java/org/apache/iceberg/MetricsUtil.java b/core/src/main/java/org/apache/iceberg/MetricsUtil.java index 5f5b1c659920..8cddcab0902a 100644 --- a/core/src/main/java/org/apache/iceberg/MetricsUtil.java +++ b/core/src/main/java/org/apache/iceberg/MetricsUtil.java @@ -34,7 +34,7 @@ private MetricsUtil() { * Construct mapping relationship between column id to NaN value counts from input metrics and metrics config. */ public static Map createNanValueCounts( - Stream fieldMetrics, MetricsConfig metricsConfig, Schema inputSchema) { + Stream> fieldMetrics, MetricsConfig metricsConfig, Schema inputSchema) { Preconditions.checkNotNull(metricsConfig, "metricsConfig is required"); if (fieldMetrics == null || inputSchema == null) { diff --git a/core/src/main/java/org/apache/iceberg/MicroBatches.java b/core/src/main/java/org/apache/iceberg/MicroBatches.java index d97dc7fd811c..798c3379b1fd 100644 --- a/core/src/main/java/org/apache/iceberg/MicroBatches.java +++ b/core/src/main/java/org/apache/iceberg/MicroBatches.java @@ -242,7 +242,7 @@ private CloseableIterable open(ManifestFile manifestFile, boolean ManifestGroup manifestGroup = new ManifestGroup(io, ImmutableList.of(manifestFile)) .specsById(specsById) .caseSensitive(caseSensitive); - if (scanAllFiles) { + if (!scanAllFiles) { manifestGroup = manifestGroup .filterManifestEntries(entry -> entry.snapshotId() == snapshot.snapshotId() && entry.status() == ManifestEntry.Status.ADDED) diff --git a/core/src/main/java/org/apache/iceberg/PartitionsTable.java b/core/src/main/java/org/apache/iceberg/PartitionsTable.java index 1410924c31a2..e190ca44560a 100644 --- a/core/src/main/java/org/apache/iceberg/PartitionsTable.java +++ b/core/src/main/java/org/apache/iceberg/PartitionsTable.java @@ -20,9 +20,14 @@ package org.apache.iceberg; import java.util.Map; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Projections; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.StructLikeWrapper; +import org.apache.iceberg.util.ThreadPools; /** * A {@link Table} implementation that exposes a table's partitions as rows. @@ -30,6 +35,9 @@ public class PartitionsTable extends BaseMetadataTable { private final Schema schema; + static final boolean PLAN_SCANS_WITH_WORKER_POOL = + SystemProperties.getBoolean(SystemProperties.SCAN_THREAD_POOL_ENABLED, true); + private static final String PARTITION_FIELD_PREFIX = "partition."; PartitionsTable(TableOperations ops, Table table) { this(ops, table, table.name() + ".partitions"); @@ -63,9 +71,9 @@ MetadataTableType metadataTableType() { return MetadataTableType.PARTITIONS; } - private DataTask task(TableScan scan) { + private DataTask task(StaticTableScan scan) { TableOperations ops = operations(); - Iterable partitions = partitions(table(), scan.snapshot().snapshotId()); + Iterable partitions = partitions(scan); if (table().spec().fields().size() < 1) { // the table is unpartitioned, partitions contains only the root partition return StaticDataTask.of(io().newInputFile(ops.current().metadataFileLocation()), partitions, @@ -80,24 +88,66 @@ private static StaticDataTask.Row convertPartition(Partition partition) { return StaticDataTask.Row.of(partition.key, partition.recordCount, partition.fileCount); } - private static Iterable partitions(Table table, Long snapshotId) { - PartitionMap partitions = new PartitionMap(table.spec().partitionType()); - TableScan scan = table.newScan(); + private static Iterable partitions(StaticTableScan scan) { + CloseableIterable tasks = planFiles(scan); - if (snapshotId != null) { - scan = scan.useSnapshot(snapshotId); + PartitionMap partitions = new PartitionMap(scan.table().spec().partitionType()); + for (FileScanTask task : tasks) { + partitions.get(task.file().partition()).update(task.file()); } + return partitions.all(); + } - for (FileScanTask task : scan.planFiles()) { - partitions.get(task.file().partition()).update(task.file()); + @VisibleForTesting + static CloseableIterable planFiles(StaticTableScan scan) { + Table table = scan.table(); + Snapshot snapshot = table.snapshot(scan.snapshot().snapshotId()); + boolean caseSensitive = scan.isCaseSensitive(); + + // use an inclusive projection to remove the partition name prefix and filter out any non-partition expressions + Expression partitionFilter = Projections + .inclusive(transformSpec(scan.schema(), table.spec()), caseSensitive) + .project(scan.filter()); + + ManifestGroup manifestGroup = new ManifestGroup(table.io(), snapshot.dataManifests(), snapshot.deleteManifests()) + .caseSensitive(caseSensitive) + .filterPartitions(partitionFilter) + .specsById(scan.table().specs()) + .ignoreDeleted(); + + if (scan.shouldIgnoreResiduals()) { + manifestGroup = manifestGroup.ignoreResiduals(); } - return partitions.all(); + if (PLAN_SCANS_WITH_WORKER_POOL && scan.snapshot().dataManifests().size() > 1) { + manifestGroup = manifestGroup.planWith(ThreadPools.getWorkerPool()); + } + + return manifestGroup.planFiles(); + } + + /** + * This method transforms the table's partition spec to a spec that is used to rewrite the user-provided filter + * expression against the partitions table. + *

    + * The resulting partition spec maps partition.X fields to partition X using an identity partition transform. When + * this spec is used to project an expression for the partitions table, the projection will remove predicates for + * non-partition fields (not in the spec) and will remove the "partition." prefix from fields. + * + * @param partitionTableSchema schema of the partition table + * @param spec spec on which the partition table schema is based + * @return a spec used to rewrite partition table filters to partition filters using an inclusive projection + */ + private static PartitionSpec transformSpec(Schema partitionTableSchema, PartitionSpec spec) { + PartitionSpec.Builder identitySpecBuilder = PartitionSpec.builderFor(partitionTableSchema); + spec.fields().forEach(pf -> identitySpecBuilder.identity(PARTITION_FIELD_PREFIX + pf.name(), pf.name())); + return identitySpecBuilder.build(); } private class PartitionsScan extends StaticTableScan { PartitionsScan(TableOperations ops, Table table) { - super(ops, table, PartitionsTable.this.schema(), PartitionsTable.this::task); + super(ops, table, PartitionsTable.this.schema(), PartitionsTable.this.metadataTableType().name(), + PartitionsTable.this::task); } } diff --git a/core/src/main/java/org/apache/iceberg/SchemaUpdate.java b/core/src/main/java/org/apache/iceberg/SchemaUpdate.java index c14398d17e1c..e5f9206fb3b9 100644 --- a/core/src/main/java/org/apache/iceberg/SchemaUpdate.java +++ b/core/src/main/java/org/apache/iceberg/SchemaUpdate.java @@ -458,7 +458,7 @@ private static Schema applyChanges(Schema schema, List deletes, Set freshIdentifierFieldIds = Sets.newHashSet(); for (String name : identifierFieldNames) { Preconditions.checkArgument(nameToId.containsKey(name), - "Cannot add field %s as an identifier field: not found in current schema or added columns"); + "Cannot add field %s as an identifier field: not found in current schema or added columns", name); freshIdentifierFieldIds.add(nameToId.get(name)); } @@ -474,6 +474,11 @@ private static void validateIdentifierField(int fieldId, Map addFields(List fields, return newFields; } - @SuppressWarnings("checkstyle:IllegalType") + @SuppressWarnings({"checkstyle:IllegalType", "JdkObsolete"}) private static List moveFields(List fields, Collection moves) { LinkedList reordered = Lists.newLinkedList(fields); diff --git a/core/src/main/java/org/apache/iceberg/SerializableTable.java b/core/src/main/java/org/apache/iceberg/SerializableTable.java index 5b649f6eb1fa..d8f623de4024 100644 --- a/core/src/main/java/org/apache/iceberg/SerializableTable.java +++ b/core/src/main/java/org/apache/iceberg/SerializableTable.java @@ -24,7 +24,7 @@ import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.hadoop.HadoopConfigurable; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -102,13 +102,11 @@ private String metadataFileLocation(Table table) { } private FileIO fileIO(Table table) { - if (table.io() instanceof HadoopFileIO) { - HadoopFileIO hadoopFileIO = (HadoopFileIO) table.io(); - SerializableConfiguration serializedConf = new SerializableConfiguration(hadoopFileIO.getConf()); - return new HadoopFileIO(serializedConf::get); - } else { - return table.io(); + if (table.io() instanceof HadoopConfigurable) { + ((HadoopConfigurable) table.io()).serializeConfWith(conf -> new SerializableConfiguration(conf)::get); } + + return table.io(); } private Table lazyTable() { @@ -163,6 +161,11 @@ public Schema schema() { return lazySchema; } + @Override + public Map schemas() { + return lazyTable().schemas(); + } + @Override public PartitionSpec spec() { if (lazySpec == null) { diff --git a/core/src/main/java/org/apache/iceberg/SnapshotParser.java b/core/src/main/java/org/apache/iceberg/SnapshotParser.java index cf13c35b02ff..2675e966056f 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotParser.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotParser.java @@ -46,6 +46,7 @@ private SnapshotParser() { private static final String OPERATION = "operation"; private static final String MANIFESTS = "manifests"; private static final String MANIFEST_LIST = "manifest-list"; + private static final String SCHEMA_ID = "schema-id"; static void toJson(Snapshot snapshot, JsonGenerator generator) throws IOException { @@ -88,6 +89,11 @@ static void toJson(Snapshot snapshot, JsonGenerator generator) generator.writeEndArray(); } + // schema ID might be null for snapshots written by old writers + if (snapshot.schemaId() != null) { + generator.writeNumberField(SCHEMA_ID, snapshot.schemaId()); + } + generator.writeEndObject(); } @@ -139,17 +145,20 @@ static Snapshot fromJson(FileIO io, JsonNode node) { summary = builder.build(); } + Integer schemaId = JsonUtil.getIntOrNull(SCHEMA_ID, node); + if (node.has(MANIFEST_LIST)) { // the manifest list is stored in a manifest list file String manifestList = JsonUtil.getString(MANIFEST_LIST, node); - return new BaseSnapshot(io, sequenceNumber, snapshotId, parentId, timestamp, operation, summary, manifestList); + return new BaseSnapshot( + io, sequenceNumber, snapshotId, parentId, timestamp, operation, summary, schemaId, manifestList); } else { // fall back to an embedded manifest list. pass in the manifest's InputFile so length can be // loaded lazily, if it is needed List manifests = Lists.transform(JsonUtil.getStringList(MANIFESTS, node), location -> new GenericManifestFile(io.newInputFile(location), 0)); - return new BaseSnapshot(io, snapshotId, parentId, timestamp, operation, summary, manifests); + return new BaseSnapshot(io, snapshotId, parentId, timestamp, operation, summary, schemaId, manifests); } } diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 3a179f99fbe4..7825500bc7cb 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -188,12 +188,12 @@ public Snapshot apply() { return new BaseSnapshot(ops.io(), sequenceNumber, snapshotId(), parentSnapshotId, System.currentTimeMillis(), operation(), summary(base), - manifestList.location()); + base.currentSchemaId(), manifestList.location()); } else { return new BaseSnapshot(ops.io(), snapshotId(), parentSnapshotId, System.currentTimeMillis(), operation(), summary(base), - manifests); + base.currentSchemaId(), manifests); } } @@ -441,7 +441,7 @@ private static ManifestFile addMetadata(TableOperations ops, ManifestFile manife return new GenericManifestFile(manifest.path(), manifest.length(), manifest.partitionSpecId(), ManifestContent.DATA, manifest.sequenceNumber(), manifest.minSequenceNumber(), snapshotId, - addedFiles, addedRows, existingFiles, existingRows, deletedFiles, deletedRows, stats.summaries()); + addedFiles, addedRows, existingFiles, existingRows, deletedFiles, deletedRows, stats.summaries(), null); } catch (IOException e) { throw new RuntimeIOException(e, "Failed to read manifest: %s", manifest.path()); diff --git a/core/src/main/java/org/apache/iceberg/SnapshotsTable.java b/core/src/main/java/org/apache/iceberg/SnapshotsTable.java index 338b7aacc15d..3501662bc46a 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotsTable.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotsTable.java @@ -71,7 +71,7 @@ MetadataTableType metadataTableType() { private class SnapshotsTableScan extends StaticTableScan { SnapshotsTableScan(TableOperations ops, Table table) { - super(ops, table, SNAPSHOT_SCHEMA, SnapshotsTable.this::task); + super(ops, table, SNAPSHOT_SCHEMA, SnapshotsTable.this.metadataTableType().name(), SnapshotsTable.this::task); } @Override diff --git a/core/src/main/java/org/apache/iceberg/StaticTableScan.java b/core/src/main/java/org/apache/iceberg/StaticTableScan.java index 3e1918566a65..08a73dd8d2bb 100644 --- a/core/src/main/java/org/apache/iceberg/StaticTableScan.java +++ b/core/src/main/java/org/apache/iceberg/StaticTableScan.java @@ -25,16 +25,32 @@ class StaticTableScan extends BaseTableScan { private final Function buildTask; + private final String tableType; - StaticTableScan(TableOperations ops, Table table, Schema schema, Function buildTask) { + StaticTableScan(TableOperations ops, Table table, Schema schema, String tableType, + Function buildTask) { super(ops, table, schema); this.buildTask = buildTask; + this.tableType = tableType; } - private StaticTableScan(TableOperations ops, Table table, Schema schema, + private StaticTableScan(TableOperations ops, Table table, Schema schema, String tableType, Function buildTask, TableScanContext context) { super(ops, table, schema, context); this.buildTask = buildTask; + this.tableType = tableType; + } + + @Override + public TableScan appendsBetween(long fromSnapshotId, long toSnapshotId) { + throw new UnsupportedOperationException( + String.format("Cannot incrementally scan table of type %s", tableType)); + } + + @Override + public TableScan appendsAfter(long fromSnapshotId) { + throw new UnsupportedOperationException( + String.format("Cannot incrementally scan table of type %s", tableType)); } @Override @@ -46,7 +62,7 @@ public long targetSplitSize() { @Override protected TableScan newRefinedScan(TableOperations ops, Table table, Schema schema, TableScanContext context) { return new StaticTableScan( - ops, table, schema, buildTask, context); + ops, table, schema, tableType, buildTask, context); } @Override diff --git a/core/src/main/java/org/apache/iceberg/V2Metadata.java b/core/src/main/java/org/apache/iceberg/V2Metadata.java index 4b064ffeebcb..40b8624baa3b 100644 --- a/core/src/main/java/org/apache/iceberg/V2Metadata.java +++ b/core/src/main/java/org/apache/iceberg/V2Metadata.java @@ -132,6 +132,8 @@ public Object get(int pos) { return wrapped.deletedRowsCount(); case 13: return wrapped.partitions(); + case 14: + return wrapped.keyMetadata(); default: throw new UnsupportedOperationException("Unknown field ordinal: " + pos); } @@ -222,6 +224,11 @@ public List partitions() { return wrapped.partitions(); } + @Override + public ByteBuffer keyMetadata() { + return wrapped.keyMetadata(); + } + @Override public ManifestFile copy() { return wrapped.copy(); diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseRemoveOrphanFilesActionResult.java b/core/src/main/java/org/apache/iceberg/actions/BaseDeleteOrphanFilesActionResult.java similarity index 89% rename from core/src/main/java/org/apache/iceberg/actions/BaseRemoveOrphanFilesActionResult.java rename to core/src/main/java/org/apache/iceberg/actions/BaseDeleteOrphanFilesActionResult.java index 7122943633ff..881e908706ea 100644 --- a/core/src/main/java/org/apache/iceberg/actions/BaseRemoveOrphanFilesActionResult.java +++ b/core/src/main/java/org/apache/iceberg/actions/BaseDeleteOrphanFilesActionResult.java @@ -19,11 +19,11 @@ package org.apache.iceberg.actions; -public class BaseRemoveOrphanFilesActionResult implements RemoveOrphanFiles.Result { +public class BaseDeleteOrphanFilesActionResult implements DeleteOrphanFiles.Result { private final Iterable orphanFileLocations; - public BaseRemoveOrphanFilesActionResult(Iterable orphanFileLocations) { + public BaseDeleteOrphanFilesActionResult(Iterable orphanFileLocations) { this.orphanFileLocations = orphanFileLocations; } diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseRemoveFilesActionResult.java b/core/src/main/java/org/apache/iceberg/actions/BaseDeleteReachableFilesActionResult.java similarity index 72% rename from core/src/main/java/org/apache/iceberg/actions/BaseRemoveFilesActionResult.java rename to core/src/main/java/org/apache/iceberg/actions/BaseDeleteReachableFilesActionResult.java index 8707975e0a11..61abd43141e4 100644 --- a/core/src/main/java/org/apache/iceberg/actions/BaseRemoveFilesActionResult.java +++ b/core/src/main/java/org/apache/iceberg/actions/BaseDeleteReachableFilesActionResult.java @@ -19,17 +19,17 @@ package org.apache.iceberg.actions; -public class BaseRemoveFilesActionResult implements RemoveReachableFiles.Result { +public class BaseDeleteReachableFilesActionResult implements DeleteReachableFiles.Result { private final long deletedDataFilesCount; private final long deletedManifestsCount; private final long deletedManifestListsCount; private final long deletedOtherFilesCount; - public BaseRemoveFilesActionResult(long deletedDataFilesCount, - long deletedManifestsCount, - long deletedManifestListsCount, - long otherDeletedFilesCount) { + public BaseDeleteReachableFilesActionResult(long deletedDataFilesCount, + long deletedManifestsCount, + long deletedManifestListsCount, + long otherDeletedFilesCount) { this.deletedDataFilesCount = deletedDataFilesCount; this.deletedManifestsCount = deletedManifestsCount; this.deletedManifestListsCount = deletedManifestListsCount; @@ -37,22 +37,22 @@ public BaseRemoveFilesActionResult(long deletedDataFilesCount, } @Override - public long removedDataFilesCount() { + public long deletedDataFilesCount() { return deletedDataFilesCount; } @Override - public long removedManifestsCount() { + public long deletedManifestsCount() { return deletedManifestsCount; } @Override - public long removedManifestListsCount() { + public long deletedManifestListsCount() { return deletedManifestListsCount; } @Override - public long otherRemovedFilesCount() { + public long deletedOtherFilesCount() { return deletedOtherFilesCount; } } diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseFileGroupRewriteResult.java b/core/src/main/java/org/apache/iceberg/actions/BaseFileGroupRewriteResult.java new file mode 100644 index 000000000000..d61162096832 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/BaseFileGroupRewriteResult.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.actions; + +import org.apache.iceberg.actions.RewriteDataFiles.FileGroupInfo; +import org.apache.iceberg.actions.RewriteDataFiles.FileGroupRewriteResult; + +public class BaseFileGroupRewriteResult implements FileGroupRewriteResult { + private final int addedDataFilesCount; + private final int rewrittenDataFilesCount; + private final FileGroupInfo info; + + public BaseFileGroupRewriteResult(FileGroupInfo info, int addedFilesCount, int rewrittenFilesCount) { + this.info = info; + this.addedDataFilesCount = addedFilesCount; + this.rewrittenDataFilesCount = rewrittenFilesCount; + } + + @Override + public FileGroupInfo info() { + return info; + } + + @Override + public int addedDataFilesCount() { + return addedDataFilesCount; + } + + @Override + public int rewrittenDataFilesCount() { + return rewrittenDataFilesCount; + } +} diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFilesFileGroupInfo.java b/core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFilesFileGroupInfo.java new file mode 100644 index 000000000000..480799b80ba2 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFilesFileGroupInfo.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.actions; + +import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +public class BaseRewriteDataFilesFileGroupInfo implements RewriteDataFiles.FileGroupInfo { + private final int globalIndex; + private final int partitionIndex; + private final StructLike partition; + + public BaseRewriteDataFilesFileGroupInfo(int globalIndex, int partitionIndex, StructLike partition) { + this.globalIndex = globalIndex; + this.partitionIndex = partitionIndex; + this.partition = partition; + } + + @Override + public int globalIndex() { + return globalIndex; + } + + @Override + public int partitionIndex() { + return partitionIndex; + } + + @Override + public StructLike partition() { + return partition; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("globalIndex", globalIndex) + .add("partitionIndex", partitionIndex) + .add("partition", partition) + .toString(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFilesResult.java b/core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFilesResult.java new file mode 100644 index 000000000000..af9236fb16a3 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFilesResult.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.actions; + +import java.util.List; +import org.apache.iceberg.actions.RewriteDataFiles.FileGroupRewriteResult; +import org.apache.iceberg.actions.RewriteDataFiles.Result; + +public class BaseRewriteDataFilesResult implements Result { + private final List rewriteResults; + + public BaseRewriteDataFilesResult(List rewriteResults) { + this.rewriteResults = rewriteResults; + } + + @Override + public List rewriteResults() { + return rewriteResults; + } +} diff --git a/core/src/main/java/org/apache/iceberg/actions/BinPackStrategy.java b/core/src/main/java/org/apache/iceberg/actions/BinPackStrategy.java index b42001cd76d5..c3d1f7a67cb5 100644 --- a/core/src/main/java/org/apache/iceberg/actions/BinPackStrategy.java +++ b/core/src/main/java/org/apache/iceberg/actions/BinPackStrategy.java @@ -19,6 +19,7 @@ package org.apache.iceberg.actions; +import java.math.RoundingMode; import java.util.List; import java.util.Map; import java.util.Set; @@ -28,21 +29,22 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.FluentIterable; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.math.LongMath; import org.apache.iceberg.util.BinPacking; import org.apache.iceberg.util.BinPacking.ListPacker; import org.apache.iceberg.util.PropertyUtil; /** * A rewrite strategy for data files which determines which files to rewrite - * based on their size. If files are either smaller than the {@link MIN_FILE_SIZE_BYTES} threshold or - * larger than the {@link MAX_FILE_SIZE_BYTES} threshold, they are considered targets for being rewritten. + * based on their size. If files are either smaller than the {@link #MIN_FILE_SIZE_BYTES} threshold or + * larger than the {@link #MAX_FILE_SIZE_BYTES} threshold, they are considered targets for being rewritten. *

    * Once selected files are grouped based on a {@link BinPacking} into groups defined * by {@link RewriteDataFiles#MAX_FILE_GROUP_SIZE_BYTES}. Groups will be considered for rewriting if they contain - * more files than {@link MIN_INPUT_FILES} or would produce at least one file of + * more files than {@link #MIN_INPUT_FILES} or would produce at least one file of * {@link RewriteDataFiles#TARGET_FILE_SIZE_BYTES}. */ -abstract class BinPackStrategy implements RewriteStrategy { +public abstract class BinPackStrategy implements RewriteStrategy { /** * The minimum number of files that need to be in a file group for it to be considered for @@ -55,8 +57,8 @@ abstract class BinPackStrategy implements RewriteStrategy { /** * Adjusts files which will be considered for rewriting. Files smaller than - * {@link MIN_FILE_SIZE_BYTES} will be considered for rewriting. This functions independently - * of {@link MAX_FILE_SIZE_BYTES}. + * {@link #MIN_FILE_SIZE_BYTES} will be considered for rewriting. This functions independently + * of {@link #MAX_FILE_SIZE_BYTES}. *

    * Defaults to 75% of the target file size */ @@ -65,8 +67,8 @@ abstract class BinPackStrategy implements RewriteStrategy { /** * Adjusts files which will be considered for rewriting. Files larger than - * {@link MAX_FILE_SIZE_BYTES} will be considered for rewriting. This functions independently - * of {@link MIN_FILE_SIZE_BYTES}. + * {@link #MAX_FILE_SIZE_BYTES} will be considered for rewriting. This functions independently + * of {@link #MIN_FILE_SIZE_BYTES}. *

    * Defaults to 180% of the target file size */ @@ -137,6 +139,77 @@ public Iterable> planFileGroups(Iterable dataFi ).collect(Collectors.toList()); } + protected long targetFileSize() { + return this.targetFileSize; + } + + /** + * Determine how many output files to create when rewriting. We use this to determine the split-size + * we want to use when actually writing files to avoid the following situation. + *

    + * If we are writing 10.1 G of data with a target file size of 1G we would end up with + * 11 files, one of which would only have 0.1g. This would most likely be less preferable to + * 10 files each of which was 1.01g. So here we decide whether to round up or round down + * based on what the estimated average file size will be if we ignore the remainder (0.1g). If + * the new file size is less than 10% greater than the target file size then we will round down + * when determining the number of output files. + * @param totalSizeInBytes total data size for a file group + * @return the number of files this strategy should create + */ + protected long numOutputFiles(long totalSizeInBytes) { + if (totalSizeInBytes < targetFileSize) { + return 1; + } + + long fileCountWithRemainder = LongMath.divide(totalSizeInBytes, targetFileSize, RoundingMode.CEILING); + if (LongMath.mod(totalSizeInBytes, targetFileSize) > minFileSize) { + // Our Remainder file is of valid size for this compaction so keep it + return fileCountWithRemainder; + } + + long fileCountWithoutRemainder = LongMath.divide(totalSizeInBytes, targetFileSize, RoundingMode.FLOOR); + long avgFileSizeWithoutRemainder = totalSizeInBytes / fileCountWithoutRemainder; + if (avgFileSizeWithoutRemainder < Math.min(1.1 * targetFileSize, writeMaxFileSize())) { + // Round down and distribute remainder amongst other files + return fileCountWithoutRemainder; + } else { + // Keep the remainder file + return fileCountWithRemainder; + } + } + + /** + * Returns the smaller of our max write file threshold, and our estimated split size based on + * the number of output files we want to generate. + */ + protected long splitSize(long totalSizeInBytes) { + long estimatedSplitSize = totalSizeInBytes / numOutputFiles(totalSizeInBytes); + return Math.min(estimatedSplitSize, writeMaxFileSize()); + } + + protected long inputFileSize(List fileToRewrite) { + return fileToRewrite.stream().mapToLong(FileScanTask::length).sum(); + } + + /** + * Estimates a larger max target file size than our target size used in task creation to avoid + * tasks which are predicted to have a certain size, but exceed that target size when serde is complete creating + * tiny remainder files. + *

    + * While we create tasks that should all be smaller than our target size there is a chance that the actual + * data will end up being larger than our target size due to various factors of compression, serialization and + * other factors outside our control. If this occurs, instead of making a single file that is close in size to + * our target we would end up producing one file of the target size, and then a small extra file with the remaining + * data. For example, if our target is 512 MB we may generate a rewrite task that should be 500 MB. When we write + * the data we may find we actually have to write out 530 MB. If we use the target size while writing we would + * produced a 512 MB file and a 18 MB file. If instead we use a larger size estimated by this method, + * then we end up writing a single file. + * @return the target size plus one half of the distance between max and target + */ + protected long writeMaxFileSize() { + return (long) (targetFileSize + ((maxFileSize - targetFileSize) * 0.5)); + } + private long sizeOfInputFiles(List group) { return group.stream().mapToLong(FileScanTask::length).sum(); } diff --git a/core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitManager.java b/core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitManager.java new file mode 100644 index 000000000000..28c3748ac2e5 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitManager.java @@ -0,0 +1,214 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.actions; + +import java.io.Closeable; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.exceptions.CommitStateUnknownException; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Queues; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.iceberg.util.Tasks; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Functionality used by RewriteDataFile Actions from different platforms to handle commits. + */ +public class RewriteDataFilesCommitManager { + private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesCommitManager.class); + + private final Table table; + + public RewriteDataFilesCommitManager(Table table) { + this.table = table; + } + + /** + * Perform a commit operation on the table adding and removing files as + * required for this set of file groups + * @param fileGroups fileSets to commit + */ + public void commitFileGroups(Set fileGroups) { + Set rewrittenDataFiles = Sets.newHashSet(); + Set addedDataFiles = Sets.newHashSet(); + for (RewriteFileGroup group : fileGroups) { + rewrittenDataFiles = Sets.union(rewrittenDataFiles, group.rewrittenFiles()); + addedDataFiles = Sets.union(addedDataFiles, group.addedFiles()); + } + + table.newRewrite() + .rewriteFiles(rewrittenDataFiles, addedDataFiles) + .commit(); + } + + /** + * Clean up a specified file set by removing any files created for that operation, should + * not throw any exceptions + * @param fileGroup group of files which has already been rewritten + */ + public void abortFileGroup(RewriteFileGroup fileGroup) { + Preconditions.checkState(fileGroup.addedFiles() != null, + "Cannot abort a fileGroup that was not rewritten"); + + Tasks.foreach(fileGroup.addedFiles()) + .noRetry() + .suppressFailureWhenFinished() + .onFailure((dataFile, exc) -> LOG.warn("Failed to delete: {}", dataFile.path(), exc)) + .run(dataFile -> table.io().deleteFile(dataFile.path().toString())); + } + + public void commitOrClean(Set rewriteGroups) { + try { + commitFileGroups(rewriteGroups); + } catch (CommitStateUnknownException e) { + LOG.error("Commit state unknown for {}, cannot clean up files because they may have been committed successfully.", + rewriteGroups, e); + throw e; + } catch (Exception e) { + LOG.error("Cannot commit groups {}, attempting to clean up written files", rewriteGroups, e); + rewriteGroups.forEach(this::abortFileGroup); + throw e; + } + } + + /** + * An async service which allows for committing multiple file groups as their rewrites complete. The service also + * allows for partial-progress since commits can fail. Once the service has been closed no new file groups should not + * be offered. + * @param rewritesPerCommit number of file groups to include in a commit + * @return the service for handling commits + */ + public CommitService service(int rewritesPerCommit) { + return new CommitService(rewritesPerCommit); + } + + public class CommitService implements Closeable { + private final ExecutorService committerService; + private final ConcurrentLinkedQueue completedRewrites; + private final List committedRewrites; + private final int rewritesPerCommit; + private final AtomicBoolean running = new AtomicBoolean(false); + + CommitService(int rewritesPerCommit) { + LOG.info("Creating commit service for table {} with {} groups per commit", table, rewritesPerCommit); + this.rewritesPerCommit = rewritesPerCommit; + + committerService = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder() + .setNameFormat("Committer-Service") + .build()); + + completedRewrites = Queues.newConcurrentLinkedQueue(); + committedRewrites = Lists.newArrayList(); + } + + /** + * Starts a single threaded executor service for handling file group commits. + */ + public void start() { + Preconditions.checkState(running.compareAndSet(false, true), "Rewrite Commit service already started"); + LOG.info("Starting commit service for {}", table); + // Partial progress commit service + committerService.execute(() -> { + while (running.get() || completedRewrites.size() > 0) { + try { + if (completedRewrites.size() == 0) { + // Give other threads a chance to make progress + Thread.sleep(100); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted while processing commits", e); + } + + // Either we have a full commit group, or we have completed writing and need to commit what is left over + if (completedRewrites.size() >= rewritesPerCommit || (!running.get() && completedRewrites.size() > 0)) { + Set batch = Sets.newHashSetWithExpectedSize(rewritesPerCommit); + for (int i = 0; i < rewritesPerCommit && !completedRewrites.isEmpty(); i++) { + batch.add(completedRewrites.poll()); + } + + try { + commitOrClean(batch); + committedRewrites.addAll(batch); + } catch (Exception e) { + LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e); + } + } + } + }); + } + + /** + * Places a file group in the queue to be asynchronously committed either when the queue has enough elements + * to do a batch of size {@link #rewritesPerCommit} or the service has been closed. + * @param group file group to eventually be committed + */ + public void offer(RewriteFileGroup group) { + LOG.debug("Offered to commit service: {}", group); + Preconditions.checkState(running.get(), "Cannot add rewrites to a service which has already been closed"); + completedRewrites.add(group); + } + + /** + * Returns all File groups which have been committed + */ + public List results() { + Preconditions.checkState(committerService.isShutdown(), + "Cannot get results from a service which has not been closed"); + return committedRewrites; + } + + @Override + public void close() { + Preconditions.checkState(running.compareAndSet(true, false), + "Cannot close already closed RewriteService"); + LOG.info("Closing commit service for {}", table); + committerService.shutdown(); + + try { + // All rewrites have completed and all new files have been created, we are now waiting for the commit + // pool to finish doing it's commits to Iceberg State. In the case of partial progress this should + // have been occurring simultaneously with rewrites, if not there should be only a single commit operation. + // In either case this should take much less than 10 minutes to actually complete. + if (!committerService.awaitTermination(10, TimeUnit.MINUTES)) { + LOG.warn("Commit operation did not complete within 10 minutes of the files being written. This may mean " + + "that changes were not successfully committed to the the Iceberg table."); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Cannot complete commit for rewrite, commit service interrupted", e); + } + + Preconditions.checkState(completedRewrites.isEmpty(), "File groups offered after service was closed, " + + "they were not successfully committed."); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/actions/RewriteFileGroup.java b/core/src/main/java/org/apache/iceberg/actions/RewriteFileGroup.java new file mode 100644 index 000000000000..1dd1476ba62b --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/RewriteFileGroup.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.actions; + +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.actions.RewriteDataFiles.FileGroupInfo; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * Container class representing a set of files to be rewritten by a RewriteAction and the new files which have been + * written by the action. + */ +public class RewriteFileGroup { + private final FileGroupInfo info; + private final List fileScanTasks; + + private Set addedFiles = Collections.emptySet(); + + public RewriteFileGroup(FileGroupInfo info, List fileScanTasks) { + this.info = info; + this.fileScanTasks = fileScanTasks; + } + + public FileGroupInfo info() { + return info; + } + + public List fileScans() { + return fileScanTasks; + } + + public void setOutputFiles(Set files) { + addedFiles = files; + } + + public Set rewrittenFiles() { + return fileScans().stream().map(FileScanTask::file).collect(Collectors.toSet()); + } + + public Set addedFiles() { + return addedFiles; + } + + public RewriteDataFiles.FileGroupRewriteResult asResult() { + Preconditions.checkState(addedFiles != null, "Cannot get result, Group was never rewritten"); + return new BaseFileGroupRewriteResult(info, addedFiles.size(), fileScanTasks.size()); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("info", info) + .add("numRewrittenFiles", fileScanTasks.size()) + .add("numAddedFiles", addedFiles == null ? "Rewrite Incomplete" : Integer.toString(addedFiles.size())) + .toString(); + } +} diff --git a/api/src/main/java/org/apache/iceberg/actions/RewriteStrategy.java b/core/src/main/java/org/apache/iceberg/actions/RewriteStrategy.java similarity index 94% rename from api/src/main/java/org/apache/iceberg/actions/RewriteStrategy.java rename to core/src/main/java/org/apache/iceberg/actions/RewriteStrategy.java index 13f1386c0d1e..72a10680caf2 100644 --- a/api/src/main/java/org/apache/iceberg/actions/RewriteStrategy.java +++ b/core/src/main/java/org/apache/iceberg/actions/RewriteStrategy.java @@ -27,7 +27,7 @@ import org.apache.iceberg.FileScanTask; import org.apache.iceberg.Table; -interface RewriteStrategy extends Serializable { +public interface RewriteStrategy extends Serializable { /** * Returns the name of this rewrite strategy */ @@ -72,7 +72,7 @@ interface RewriteStrategy extends Serializable { * This will most likely be Action framework specific (Spark/Presto/Flink ....). * * @param filesToRewrite a group of files to be rewritten together - * @return a list of newly written files + * @return a set of newly written files */ - List rewriteFiles(List filesToRewrite); + Set rewriteFiles(List filesToRewrite); } diff --git a/core/src/main/java/org/apache/iceberg/avro/BuildAvroProjection.java b/core/src/main/java/org/apache/iceberg/avro/BuildAvroProjection.java index 5a954103aef0..cd934dc4b89d 100644 --- a/core/src/main/java/org/apache/iceberg/avro/BuildAvroProjection.java +++ b/core/src/main/java/org/apache/iceberg/avro/BuildAvroProjection.java @@ -21,6 +21,7 @@ import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.function.Supplier; import org.apache.avro.JsonProperties; import org.apache.avro.Schema; @@ -133,7 +134,7 @@ public Schema.Field field(Schema.Field field, Supplier fieldResult) { try { Schema schema = fieldResult.get(); - if (schema != field.schema() || !expectedName.equals(field.name())) { + if (!Objects.equals(schema, field.schema()) || !expectedName.equals(field.name())) { // add an alias for the field return AvroSchemaUtil.copyField(field, schema, AvroSchemaUtil.makeCompatibleName(expectedName)); } else { @@ -153,7 +154,7 @@ public Schema union(Schema union, Iterable options) { Schema nonNullOriginal = AvroSchemaUtil.fromOption(union); Schema nonNullResult = AvroSchemaUtil.fromOptions(Lists.newArrayList(options)); - if (nonNullOriginal != nonNullResult) { + if (!Objects.equals(nonNullOriginal, nonNullResult)) { return AvroSchemaUtil.toOption(nonNullResult); } @@ -174,7 +175,7 @@ public Schema array(Schema array, Supplier element) { Schema.Field valueProjection = element.get().getField("value"); // element was changed, create a new array - if (valueProjection.schema() != valueField.schema()) { + if (!Objects.equals(valueProjection.schema(), valueField.schema())) { return AvroSchemaUtil.createProjectionMap(keyValueSchema.getFullName(), AvroSchemaUtil.getFieldId(keyField), keyField.name(), keyField.schema(), AvroSchemaUtil.getFieldId(valueField), valueField.name(), valueProjection.schema()); @@ -199,7 +200,7 @@ public Schema array(Schema array, Supplier element) { Schema elementSchema = element.get(); // element was changed, create a new array - if (elementSchema != array.getElementType()) { + if (!Objects.equals(elementSchema, array.getElementType())) { return Schema.createArray(elementSchema); } @@ -223,7 +224,7 @@ public Schema map(Schema map, Supplier value) { Schema valueSchema = value.get(); // element was changed, create a new map - if (valueSchema != map.getValueType()) { + if (!Objects.equals(valueSchema, map.getValueType())) { return Schema.createMap(valueSchema); } diff --git a/core/src/main/java/org/apache/iceberg/avro/PruneColumns.java b/core/src/main/java/org/apache/iceberg/avro/PruneColumns.java index 617607054788..57e2c2709137 100644 --- a/core/src/main/java/org/apache/iceberg/avro/PruneColumns.java +++ b/core/src/main/java/org/apache/iceberg/avro/PruneColumns.java @@ -21,6 +21,7 @@ import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import org.apache.avro.JsonProperties; import org.apache.avro.Schema; @@ -118,7 +119,7 @@ public Schema union(Schema union, List options) { } if (pruned != null) { - if (pruned != AvroSchemaUtil.fromOption(union)) { + if (!Objects.equals(pruned, AvroSchemaUtil.fromOption(union))) { return AvroSchemaUtil.toOption(pruned); } return union; @@ -149,13 +150,14 @@ public Schema array(Schema array, Schema element) { Schema valueProjection = element.getField("value").schema(); // it is possible that key is not selected, and // key schemas can be different if new field ids were assigned to them - if (keyProjectionField != null && keyValue.getField("key").schema() != keyProjectionField.schema()) { + if (keyProjectionField != null && + !Objects.equals(keyValue.getField("key").schema(), keyProjectionField.schema())) { Preconditions.checkState( SchemaNormalization.parsingFingerprint64(keyValue.getField("key").schema()) == SchemaNormalization.parsingFingerprint64(keyProjectionField.schema()), - "Map keys should not be projected"); + "Map keys should not be projected"); return AvroSchemaUtil.createMap(keyId, keyProjectionField.schema(), valueId, valueProjection); - } else if (keyValue.getField("value").schema() != valueProjection) { + } else if (!Objects.equals(keyValue.getField("value").schema(), valueProjection)) { return AvroSchemaUtil.createMap(keyId, keyValue.getField("key").schema(), valueId, valueProjection); } else { return complexMapWithIds(array, keyId, valueId); @@ -171,7 +173,7 @@ public Schema array(Schema array, Schema element) { if (selectedIds.contains(elementId)) { return arrayWithId(array, elementId); } else if (element != null) { - if (element != array.getElementType()) { + if (!Objects.equals(element, array.getElementType())) { // the element must be a projection return arrayWithId(Schema.createArray(element), elementId); } @@ -199,7 +201,7 @@ public Schema map(Schema map, Schema value) { // e.g if we are reading data not written by Iceberg writers return mapWithIds(map, keyId, valueId); } else if (value != null) { - if (value != map.getValueType()) { + if (!Objects.equals(value, map.getValueType())) { // the value must be a projection return mapWithIds(Schema.createMap(value), keyId, valueId); } diff --git a/core/src/main/java/org/apache/iceberg/avro/RemoveIds.java b/core/src/main/java/org/apache/iceberg/avro/RemoveIds.java index 1f929bf53bb6..b9e0d5cd1799 100644 --- a/core/src/main/java/org/apache/iceberg/avro/RemoveIds.java +++ b/core/src/main/java/org/apache/iceberg/avro/RemoveIds.java @@ -21,6 +21,7 @@ import java.util.List; import java.util.Map; +import java.util.Objects; import org.apache.avro.Schema; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -74,7 +75,7 @@ private static Schema.Field copyField(Schema.Field field, Schema newSchema) { Schema.Field copy = new Schema.Field(field.name(), newSchema, field.doc(), field.defaultVal(), field.order()); for (Map.Entry prop : field.getObjectProps().entrySet()) { String key = prop.getKey(); - if (key != AvroSchemaUtil.FIELD_ID_PROP) { + if (!Objects.equals(key, AvroSchemaUtil.FIELD_ID_PROP)) { copy.addProp(key, prop.getValue()); } } diff --git a/core/src/main/java/org/apache/iceberg/avro/SchemaToType.java b/core/src/main/java/org/apache/iceberg/avro/SchemaToType.java index 577fa2330cf9..73c86226007f 100644 --- a/core/src/main/java/org/apache/iceberg/avro/SchemaToType.java +++ b/core/src/main/java/org/apache/iceberg/avro/SchemaToType.java @@ -20,6 +20,7 @@ package org.apache.iceberg.avro; import java.util.List; +import java.util.Objects; import org.apache.avro.LogicalType; import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; @@ -83,7 +84,7 @@ public Type record(Schema record, List names, List fieldTypes) { List fields = record.getFields(); List newFields = Lists.newArrayListWithExpectedSize(fields.size()); - if (root == record) { + if (Objects.equals(root, record)) { this.nextId = 0; } diff --git a/core/src/main/java/org/apache/iceberg/avro/ValueReaders.java b/core/src/main/java/org/apache/iceberg/avro/ValueReaders.java index 5e72c9b1572b..6019e80db547 100644 --- a/core/src/main/java/org/apache/iceberg/avro/ValueReaders.java +++ b/core/src/main/java/org/apache/iceberg/avro/ValueReaders.java @@ -582,8 +582,7 @@ public abstract static class StructReader implements ValueReader, Supports protected StructReader(List> readers, Schema schema) { this.readers = readers.toArray(new ValueReader[0]); - this.positions = new int[0]; - this.constants = new Object[0]; + Integer isDeletedColumnPos = null; List fields = schema.getFields(); for (int pos = 0; pos < fields.size(); pos += 1) { @@ -591,8 +590,18 @@ protected StructReader(List> readers, Schema schema) { if (AvroSchemaUtil.getFieldId(field) == MetadataColumns.ROW_POSITION.fieldId()) { // track where the _pos field is located for setRowPositionSupplier this.posField = pos; + } else if (AvroSchemaUtil.getFieldId(field) == MetadataColumns.IS_DELETED.fieldId()) { + isDeletedColumnPos = pos; } } + + if (isDeletedColumnPos == null) { + this.positions = new int[0]; + this.constants = new Object[0]; + } else { + this.positions = new int[]{isDeletedColumnPos}; + this.constants = new Object[]{false}; + } } protected StructReader(List> readers, Types.StructType struct, Map idToConstant) { @@ -609,6 +618,9 @@ protected StructReader(List> readers, Types.StructType struct, Ma } else if (field.fieldId() == MetadataColumns.ROW_POSITION.fieldId()) { // track where the _pos field is located for setRowPositionSupplier this.posField = pos; + } else if (field.fieldId() == MetadataColumns.IS_DELETED.fieldId()) { + positionList.add(pos); + constantList.add(false); } } diff --git a/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteWriter.java b/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteWriter.java index 7bacdc07ba83..8c5eecfb924e 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteWriter.java +++ b/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteWriter.java @@ -22,7 +22,6 @@ import java.io.Closeable; import java.io.IOException; import java.nio.ByteBuffer; -import java.util.Set; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.FileMetadata; @@ -41,7 +40,7 @@ public class PositionDeleteWriter implements Closeable { private final StructLike partition; private final ByteBuffer keyMetadata; private final PositionDelete delete; - private final Set pathSet; + private final CharSequenceSet pathSet; private DeleteFile deleteFile = null; public PositionDeleteWriter(FileAppender appender, FileFormat format, String location, @@ -81,7 +80,7 @@ public void close() throws IOException { } } - public Set referencedDataFiles() { + public CharSequenceSet referencedDataFiles() { return pathSet; } diff --git a/core/src/main/java/org/apache/iceberg/hadoop/HadoopConfigurable.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopConfigurable.java new file mode 100644 index 000000000000..9e79c3b41426 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopConfigurable.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.hadoop; + +import java.util.function.Function; +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.util.SerializableSupplier; + +/** + * An interface that extends the Hadoop {@link Configurable} interface to offer better serialization support for + * customizable Iceberg objects such as {@link org.apache.iceberg.io.FileIO}. + *

    + * If an object is serialized and needs to use Hadoop configuration, it is recommended for the object to implement + * this interface so that a serializable supplier of configuration can be provided instead of an actual Hadoop + * configuration which is not serializable. + */ +public interface HadoopConfigurable extends Configurable { + + /** + * Take a function that serializes Hadoop configuration into a supplier. An implementation is supposed to pass in + * its current Hadoop configuration into this function, and the result can be safely serialized for future use. + * @param confSerializer A function that takes Hadoop configuration and returns a serializable supplier of it. + */ + void serializeConfWith(Function> confSerializer); + +} diff --git a/core/src/main/java/org/apache/iceberg/hadoop/HadoopFileIO.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopFileIO.java index 34d66bfae402..1c53240db4d6 100644 --- a/core/src/main/java/org/apache/iceberg/hadoop/HadoopFileIO.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopFileIO.java @@ -20,7 +20,7 @@ package org.apache.iceberg.hadoop; import java.io.IOException; -import org.apache.hadoop.conf.Configurable; +import java.util.function.Function; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -30,7 +30,7 @@ import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.util.SerializableSupplier; -public class HadoopFileIO implements FileIO, Configurable { +public class HadoopFileIO implements FileIO, HadoopConfigurable { private SerializableSupplier hadoopConf; @@ -84,4 +84,9 @@ public void setConf(Configuration conf) { public Configuration getConf() { return hadoopConf.get(); } + + @Override + public void serializeConfWith(Function> confSerializer) { + this.hadoopConf = confSerializer.apply(getConf()); + } } diff --git a/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java b/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java index 0ba8de103fd7..d787c7cd465a 100644 --- a/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java @@ -23,7 +23,6 @@ import java.io.IOException; import java.util.List; import java.util.Map; -import java.util.Set; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; @@ -44,7 +43,7 @@ public abstract class BaseTaskWriter implements TaskWriter { private final List completedDataFiles = Lists.newArrayList(); private final List completedDeleteFiles = Lists.newArrayList(); - private final Set referencedDataFiles = CharSequenceSet.empty(); + private final CharSequenceSet referencedDataFiles = CharSequenceSet.empty(); private final PartitionSpec spec; private final FileFormat format; diff --git a/core/src/main/java/org/apache/iceberg/io/SortedPosDeleteWriter.java b/core/src/main/java/org/apache/iceberg/io/SortedPosDeleteWriter.java index 03a804099c51..0da20579a62a 100644 --- a/core/src/main/java/org/apache/iceberg/io/SortedPosDeleteWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/SortedPosDeleteWriter.java @@ -25,7 +25,6 @@ import java.util.Comparator; import java.util.List; import java.util.Map; -import java.util.Set; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.StructLike; @@ -42,7 +41,7 @@ class SortedPosDeleteWriter implements Closeable { private final Map>> posDeletes = Maps.newHashMap(); private final List completedFiles = Lists.newArrayList(); - private final Set referencedDataFiles = CharSequenceSet.empty(); + private final CharSequenceSet referencedDataFiles = CharSequenceSet.empty(); private final CharSequenceWrapper wrapper = CharSequenceWrapper.wrap(null); private final FileAppenderFactory appenderFactory; @@ -98,7 +97,7 @@ public List complete() throws IOException { return completedFiles; } - public Set referencedDataFiles() { + public CharSequenceSet referencedDataFiles() { return referencedDataFiles; } diff --git a/core/src/main/java/org/apache/iceberg/io/WriteResult.java b/core/src/main/java/org/apache/iceberg/io/WriteResult.java index e620a6164340..100a37c48a12 100644 --- a/core/src/main/java/org/apache/iceberg/io/WriteResult.java +++ b/core/src/main/java/org/apache/iceberg/io/WriteResult.java @@ -22,7 +22,6 @@ import java.io.Serializable; import java.util.Collections; import java.util.List; -import java.util.Set; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; @@ -36,7 +35,7 @@ public class WriteResult implements Serializable { private WriteResult(List dataFiles, List deleteFiles, - Set referencedDataFiles) { + CharSequenceSet referencedDataFiles) { this.dataFiles = dataFiles.toArray(new DataFile[0]); this.deleteFiles = deleteFiles.toArray(new DeleteFile[0]); this.referencedDataFiles = referencedDataFiles.toArray(new CharSequence[0]); @@ -61,7 +60,7 @@ public static Builder builder() { public static class Builder { private final List dataFiles; private final List deleteFiles; - private final Set referencedDataFiles; + private final CharSequenceSet referencedDataFiles; private Builder() { this.dataFiles = Lists.newArrayList(); diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java new file mode 100644 index 000000000000..51dbbe01ee68 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java @@ -0,0 +1,386 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.jdbc; + +import java.io.Closeable; +import java.sql.DatabaseMetaData; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.SQLIntegrityConstraintViolationException; +import java.sql.SQLNonTransientConnectionException; +import java.sql.SQLTimeoutException; +import java.sql.SQLTransientConnectionException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.BaseMetastoreCatalog; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.NamespaceNotEmptyException; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class JdbcCatalog extends BaseMetastoreCatalog implements Configurable, SupportsNamespaces, Closeable { + + public static final String PROPERTY_PREFIX = "jdbc."; + private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class); + private static final Joiner SLASH = Joiner.on("/"); + + private FileIO io; + private String catalogName = "jdbc"; + private String warehouseLocation; + private Configuration conf; + private JdbcClientPool connections; + + public JdbcCatalog() { + } + + @Override + public void initialize(String name, Map properties) { + String uri = properties.get(CatalogProperties.URI); + Preconditions.checkNotNull(uri, "JDBC connection URI is required"); + + String warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION); + Preconditions.checkNotNull(warehouse, "JDBC warehouse location is required"); + this.warehouseLocation = warehouse.replaceAll("/*$", ""); + + if (name != null) { + this.catalogName = name; + } + + String fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL); + this.io = fileIOImpl == null ? new HadoopFileIO(conf) : CatalogUtil.loadFileIO(fileIOImpl, properties, conf); + + try { + LOG.debug("Connecting to JDBC database {}", properties.get(CatalogProperties.URI)); + connections = new JdbcClientPool(uri, properties); + initializeCatalogTables(); + } catch (SQLTimeoutException e) { + throw new UncheckedSQLException(e, "Cannot initialize JDBC catalog: Query timed out"); + } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) { + throw new UncheckedSQLException(e, "Cannot initialize JDBC catalog: Connection failed"); + } catch (SQLException e) { + throw new UncheckedSQLException(e, "Cannot initialize JDBC catalog"); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted in call to initialize"); + } + } + + private void initializeCatalogTables() throws InterruptedException, SQLException { + LOG.trace("Creating database tables (if missing) to store iceberg catalog"); + connections.run(conn -> { + DatabaseMetaData dbMeta = conn.getMetaData(); + ResultSet tableExists = dbMeta.getTables(null, null, JdbcUtil.CATALOG_TABLE_NAME, null); + + if (tableExists.next()) { + return true; + } + + LOG.debug("Creating table {} to store iceberg catalog", JdbcUtil.CATALOG_TABLE_NAME); + return conn.prepareStatement(JdbcUtil.CREATE_CATALOG_TABLE).execute(); + }); + } + + @Override + protected TableOperations newTableOps(TableIdentifier tableIdentifier) { + return new JdbcTableOperations(connections, io, catalogName, tableIdentifier); + } + + @Override + protected String defaultWarehouseLocation(TableIdentifier table) { + return SLASH.join(defaultNamespaceLocation(table.namespace()), table.name()); + } + + @Override + public boolean dropTable(TableIdentifier identifier, boolean purge) { + TableOperations ops = newTableOps(identifier); + TableMetadata lastMetadata; + if (purge && ops.current() != null) { + lastMetadata = ops.current(); + } else { + lastMetadata = null; + } + + int deletedRecords; + try { + deletedRecords = connections.run(conn -> { + try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.DROP_TABLE_SQL)) { + sql.setString(1, catalogName); + sql.setString(2, JdbcUtil.namespaceToString(identifier.namespace())); + sql.setString(3, identifier.name()); + return sql.executeUpdate(); + } + }); + } catch (SQLException e) { + throw new UncheckedSQLException(e, "Failed to drop %s", identifier); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted in call to dropTable"); + } + + if (deletedRecords == 0) { + LOG.info("Skipping drop, table does not exist: {}", identifier); + return false; + } + + if (purge && lastMetadata != null) { + CatalogUtil.dropTableData(ops.io(), lastMetadata); + } + + LOG.info("Dropped table: {}", identifier); + return true; + } + + @Override + public List listTables(Namespace namespace) { + if (!namespaceExists(namespace)) { + throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace); + } + + try { + return connections.run(conn -> { + List results = Lists.newArrayList(); + try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.LIST_TABLES_SQL)) { + sql.setString(1, catalogName); + sql.setString(2, JdbcUtil.namespaceToString(namespace)); + + ResultSet rs = sql.executeQuery(); + while (rs.next()) { + results.add(JdbcUtil.stringToTableIdentifier(rs.getString(JdbcUtil.TABLE_NAMESPACE), + rs.getString(JdbcUtil.TABLE_NAME)) + ); + } + + return results; + } + }); + + } catch (SQLException e) { + throw new UncheckedSQLException(e, "Failed to list tables in namespace: %s", namespace); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during JDBC operation"); + } + } + + @Override + public void renameTable(TableIdentifier from, TableIdentifier to) { + try { + int updatedRecords = connections.run(conn -> { + try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.RENAME_TABLE_SQL)) { + // SET + sql.setString(1, JdbcUtil.namespaceToString(to.namespace())); + sql.setString(2, to.name()); + // WHERE + sql.setString(3, catalogName); + sql.setString(4, JdbcUtil.namespaceToString(from.namespace())); + sql.setString(5, from.name()); + return sql.executeUpdate(); + } + }); + + if (updatedRecords == 1) { + LOG.info("Renamed table from {}, to {}", from, to); + } else if (updatedRecords == 0) { + throw new NoSuchTableException("Table does not exist: %s", from); + } else { + LOG.warn("Rename operation affected {} rows: the catalog table's primary key assumption has been violated", + updatedRecords); + } + + } catch (SQLIntegrityConstraintViolationException e) { + throw new AlreadyExistsException("Table already exists: %s", to); + } catch (SQLException e) { + throw new UncheckedSQLException(e, "Failed to rename %s to %s", from, to); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted in call to rename"); + } + } + + @Override + public String name() { + return catalogName; + } + + @Override + public Configuration getConf() { + return conf; + } + + @Override + public void setConf(Configuration conf) { + this.conf = conf; + } + + @Override + public void createNamespace(Namespace namespace, Map metadata) { + throw new UnsupportedOperationException("Cannot create namespace " + namespace + + ": createNamespace is not supported"); + } + + @Override + public List listNamespaces(Namespace namespace) throws NoSuchNamespaceException { + if (!namespaceExists(namespace)) { + throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace); + } + + try { + List namespaces = connections.run(conn -> { + List result = Lists.newArrayList(); + + try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.LIST_NAMESPACES_SQL)) { + sql.setString(1, catalogName); + sql.setString(2, JdbcUtil.namespaceToString(namespace) + "%"); + ResultSet rs = sql.executeQuery(); + while (rs.next()) { + result.add(JdbcUtil.stringToNamespace(rs.getString(JdbcUtil.TABLE_NAMESPACE))); + } + rs.close(); + } + + return result; + }); + + int subNamespaceLevelLength = namespace.levels().length + 1; + namespaces = namespaces.stream() + // exclude itself + .filter(n -> !n.equals(namespace)) + // only get sub namespaces/children + .filter(n -> n.levels().length >= subNamespaceLevelLength) + // only get sub namespaces/children + .map(n -> Namespace.of( + Arrays.stream(n.levels()).limit(subNamespaceLevelLength).toArray(String[]::new) + ) + ) + // remove duplicates + .distinct() + .collect(Collectors.toList()); + + return namespaces; + + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, + "Interrupted in call to listNamespaces(namespace) Namespace: %s", namespace); + } catch (SQLException e) { + throw new UncheckedSQLException(e, "Failed to list all namespace: %s in catalog", namespace); + } + } + + @Override + public Map loadNamespaceMetadata(Namespace namespace) throws NoSuchNamespaceException { + if (!namespaceExists(namespace)) { + throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace); + } + + return ImmutableMap.of("location", defaultNamespaceLocation(namespace)); + } + + private String defaultNamespaceLocation(Namespace namespace) { + if (namespace.isEmpty()) { + return warehouseLocation; + } else { + return SLASH.join(warehouseLocation, SLASH.join(namespace.levels())); + } + } + + @Override + public boolean dropNamespace(Namespace namespace) throws NamespaceNotEmptyException { + if (!namespaceExists(namespace)) { + throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace); + } + + List tableIdentifiers = listTables(namespace); + if (tableIdentifiers != null && !tableIdentifiers.isEmpty()) { + throw new NamespaceNotEmptyException( + "Namespace %s is not empty. %s tables exist.", namespace, tableIdentifiers.size()); + } + + return false; + } + + @Override + public boolean setProperties(Namespace namespace, Map properties) throws + NoSuchNamespaceException { + throw new UnsupportedOperationException( + "Cannot set properties " + namespace + " : setProperties is not supported"); + } + + @Override + public boolean removeProperties(Namespace namespace, Set properties) throws NoSuchNamespaceException { + throw new UnsupportedOperationException( + "Cannot remove properties " + namespace + " : removeProperties is not supported"); + } + + @Override + public void close() { + connections.close(); + } + + @Override + public boolean namespaceExists(Namespace namespace) { + try { + return connections.run(conn -> { + boolean exists = false; + + try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.GET_NAMESPACE_SQL)) { + sql.setString(1, catalogName); + sql.setString(2, JdbcUtil.namespaceToString(namespace) + "%"); + ResultSet rs = sql.executeQuery(); + if (rs.next()) { + exists = true; + } + + rs.close(); + } + + return exists; + }); + + } catch (SQLException e) { + throw new UncheckedSQLException(e, "Failed to get namespace %s", namespace); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted in call to namespaceExists(namespace)"); + } + } + +} diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcClientPool.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcClientPool.java new file mode 100644 index 000000000000..5b73fe8d30ed --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcClientPool.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.jdbc; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.SQLNonTransientConnectionException; +import java.util.Map; +import java.util.Properties; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.ClientPoolImpl; + +class JdbcClientPool extends ClientPoolImpl { + + private final String dbUrl; + private final Map properties; + + JdbcClientPool(String dbUrl, Map props) { + this(Integer.parseInt(props.getOrDefault(CatalogProperties.CLIENT_POOL_SIZE, + String.valueOf(CatalogProperties.CLIENT_POOL_SIZE_DEFAULT))), dbUrl, props); + } + + JdbcClientPool(int poolSize, String dbUrl, Map props) { + super(poolSize, SQLNonTransientConnectionException.class); + properties = props; + this.dbUrl = dbUrl; + } + + @Override + protected Connection newClient() { + try { + Properties dbProps = new Properties(); + properties.forEach((key, value) -> dbProps.put(key.replace(JdbcCatalog.PROPERTY_PREFIX, ""), value)); + return DriverManager.getConnection(dbUrl, dbProps); + } catch (SQLException e) { + throw new UncheckedSQLException(e, "Failed to connect: %s", dbUrl); + } + } + + @Override + protected Connection reconnect(Connection client) { + close(client); + return newClient(); + } + + @Override + protected void close(Connection client) { + try { + client.close(); + } catch (SQLException e) { + throw new UncheckedSQLException(e, "Failed to close connection"); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java new file mode 100644 index 000000000000..af939f36f879 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java @@ -0,0 +1,220 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.jdbc; + +import java.sql.DataTruncation; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.SQLIntegrityConstraintViolationException; +import java.sql.SQLNonTransientConnectionException; +import java.sql.SQLTimeoutException; +import java.sql.SQLTransientConnectionException; +import java.sql.SQLWarning; +import java.util.Map; +import java.util.Objects; +import org.apache.iceberg.BaseMetastoreTableOperations; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class JdbcTableOperations extends BaseMetastoreTableOperations { + + private static final Logger LOG = LoggerFactory.getLogger(JdbcTableOperations.class); + private final String catalogName; + private final TableIdentifier tableIdentifier; + private final FileIO fileIO; + private final JdbcClientPool connections; + + protected JdbcTableOperations(JdbcClientPool dbConnPool, FileIO fileIO, String catalogName, + TableIdentifier tableIdentifier) { + this.catalogName = catalogName; + this.tableIdentifier = tableIdentifier; + this.fileIO = fileIO; + this.connections = dbConnPool; + } + + @Override + public void doRefresh() { + Map table; + + try { + table = getTable(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during refresh"); + } catch (SQLException e) { + // SQL exception happened when getting table from catalog + throw new UncheckedSQLException(e, "Failed to get table %s from catalog %s", tableIdentifier, catalogName); + } + + if (table.isEmpty()) { + if (currentMetadataLocation() != null) { + throw new NoSuchTableException("Failed to load table %s from catalog %s: dropped by another process", + tableIdentifier, catalogName); + } else { + this.disableRefresh(); + return; + } + } + + String newMetadataLocation = table.get(JdbcUtil.METADATA_LOCATION); + Preconditions.checkState(newMetadataLocation != null, "Invalid table %s: metadata location is null", + tableIdentifier); + refreshFromMetadataLocation(newMetadataLocation); + } + + @Override + public void doCommit(TableMetadata base, TableMetadata metadata) { + String newMetadataLocation = writeNewMetadata(metadata, currentVersion() + 1); + try { + Map table = getTable(); + + if (!table.isEmpty()) { + validateMetadataLocation(table, base); + String oldMetadataLocation = base.metadataFileLocation(); + // Start atomic update + LOG.debug("Committing existing table: {}", tableName()); + updateTable(newMetadataLocation, oldMetadataLocation); + } else { + // table not exists create it + LOG.debug("Committing new table: {}", tableName()); + createTable(newMetadataLocation); + } + + } catch (SQLIntegrityConstraintViolationException e) { + + if (currentMetadataLocation() == null) { + throw new AlreadyExistsException(e, "Table already exists: %s", tableIdentifier); + } else { + throw new UncheckedSQLException(e, "Table already exists: %s", tableIdentifier); + } + + } catch (SQLTimeoutException e) { + throw new UncheckedSQLException(e, "Database Connection timeout"); + } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) { + throw new UncheckedSQLException(e, "Database Connection failed"); + } catch (DataTruncation e) { + throw new UncheckedSQLException(e, "Database data truncation error"); + } catch (SQLWarning e) { + throw new UncheckedSQLException(e, "Database warning"); + } catch (SQLException e) { + throw new UncheckedSQLException(e, "Unknown failure"); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during commit"); + } + } + + private void updateTable(String newMetadataLocation, String oldMetadataLocation) + throws SQLException, InterruptedException { + int updatedRecords = connections.run(conn -> { + try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.DO_COMMIT_SQL)) { + // UPDATE + sql.setString(1, newMetadataLocation); + sql.setString(2, oldMetadataLocation); + // WHERE + sql.setString(3, catalogName); + sql.setString(4, JdbcUtil.namespaceToString(tableIdentifier.namespace())); + sql.setString(5, tableIdentifier.name()); + sql.setString(6, oldMetadataLocation); + return sql.executeUpdate(); + } + }); + + if (updatedRecords == 1) { + LOG.debug("Successfully committed to existing table: {}", tableIdentifier); + } else { + throw new CommitFailedException("Failed to update table %s from catalog %s", tableIdentifier, catalogName); + } + + } + + private void createTable(String newMetadataLocation) throws SQLException, InterruptedException { + int insertRecord = connections.run(conn -> { + try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.DO_COMMIT_CREATE_TABLE_SQL)) { + sql.setString(1, catalogName); + sql.setString(2, JdbcUtil.namespaceToString(tableIdentifier.namespace())); + sql.setString(3, tableIdentifier.name()); + sql.setString(4, newMetadataLocation); + return sql.executeUpdate(); + } + }); + + if (insertRecord == 1) { + LOG.debug("Successfully committed to new table: {}", tableIdentifier); + } else { + throw new CommitFailedException("Failed to create table %s in catalog %s", tableIdentifier, catalogName); + } + } + + private void validateMetadataLocation(Map table, TableMetadata base) { + String catalogMetadataLocation = table.get(JdbcUtil.METADATA_LOCATION); + String baseMetadataLocation = base != null ? base.metadataFileLocation() : null; + + if (!Objects.equals(baseMetadataLocation, catalogMetadataLocation)) { + throw new CommitFailedException("Cannot commit %s: metadata location %s has changed from %s", + tableIdentifier, baseMetadataLocation, catalogMetadataLocation); + } + } + + @Override + public FileIO io() { + return fileIO; + } + + @Override + protected String tableName() { + return tableIdentifier.toString(); + } + + private Map getTable() throws UncheckedSQLException, SQLException, InterruptedException { + return connections.run(conn -> { + Map table = Maps.newHashMap(); + + try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.GET_TABLE_SQL)) { + sql.setString(1, catalogName); + sql.setString(2, JdbcUtil.namespaceToString(tableIdentifier.namespace())); + sql.setString(3, tableIdentifier.name()); + ResultSet rs = sql.executeQuery(); + + if (rs.next()) { + table.put(JdbcUtil.CATALOG_NAME, rs.getString(JdbcUtil.CATALOG_NAME)); + table.put(JdbcUtil.TABLE_NAMESPACE, rs.getString(JdbcUtil.TABLE_NAMESPACE)); + table.put(JdbcUtil.TABLE_NAME, rs.getString(JdbcUtil.TABLE_NAME)); + table.put(JdbcUtil.METADATA_LOCATION, rs.getString(JdbcUtil.METADATA_LOCATION)); + table.put(JdbcUtil.PREVIOUS_METADATA_LOCATION, rs.getString(JdbcUtil.PREVIOUS_METADATA_LOCATION)); + } + + rs.close(); + } + + return table; + }); + } + +} diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java new file mode 100644 index 000000000000..b1eacbf59652 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.jdbc; + +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.base.Splitter; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; + +final class JdbcUtil { + protected static final String CATALOG_TABLE_NAME = "iceberg_tables"; + protected static final String CATALOG_NAME = "catalog_name"; + protected static final String TABLE_NAMESPACE = "table_namespace"; + protected static final String TABLE_NAME = "table_name"; + protected static final String METADATA_LOCATION = "metadata_location"; + protected static final String PREVIOUS_METADATA_LOCATION = "previous_metadata_location"; + public static final String DO_COMMIT_SQL = "UPDATE " + CATALOG_TABLE_NAME + + " SET " + METADATA_LOCATION + " = ? , " + PREVIOUS_METADATA_LOCATION + " = ? " + + " WHERE " + CATALOG_NAME + " = ? AND " + + TABLE_NAMESPACE + " = ? AND " + + TABLE_NAME + " = ? AND " + + METADATA_LOCATION + " = ?"; + protected static final String CREATE_CATALOG_TABLE = + "CREATE TABLE " + CATALOG_TABLE_NAME + + "(" + + CATALOG_NAME + " VARCHAR(255) NOT NULL," + + TABLE_NAMESPACE + " VARCHAR(255) NOT NULL," + + TABLE_NAME + " VARCHAR(255) NOT NULL," + + METADATA_LOCATION + " VARCHAR(5500)," + + PREVIOUS_METADATA_LOCATION + " VARCHAR(5500)," + + "PRIMARY KEY (" + CATALOG_NAME + ", " + TABLE_NAMESPACE + ", " + TABLE_NAME + ")" + + ")"; + protected static final String GET_TABLE_SQL = "SELECT * FROM " + CATALOG_TABLE_NAME + + " WHERE " + CATALOG_NAME + " = ? AND " + TABLE_NAMESPACE + " = ? AND " + TABLE_NAME + " = ? "; + protected static final String LIST_TABLES_SQL = "SELECT * FROM " + CATALOG_TABLE_NAME + + " WHERE " + CATALOG_NAME + " = ? AND " + TABLE_NAMESPACE + " = ?"; + protected static final String RENAME_TABLE_SQL = "UPDATE " + CATALOG_TABLE_NAME + + " SET " + TABLE_NAMESPACE + " = ? , " + TABLE_NAME + " = ? " + + " WHERE " + CATALOG_NAME + " = ? AND " + TABLE_NAMESPACE + " = ? AND " + TABLE_NAME + " = ? "; + protected static final String DROP_TABLE_SQL = "DELETE FROM " + CATALOG_TABLE_NAME + + " WHERE " + CATALOG_NAME + " = ? AND " + TABLE_NAMESPACE + " = ? AND " + TABLE_NAME + " = ? "; + protected static final String GET_NAMESPACE_SQL = "SELECT " + TABLE_NAMESPACE + " FROM " + CATALOG_TABLE_NAME + + " WHERE " + CATALOG_NAME + " = ? AND " + TABLE_NAMESPACE + " LIKE ? LIMIT 1"; + protected static final String LIST_NAMESPACES_SQL = "SELECT DISTINCT " + TABLE_NAMESPACE + + " FROM " + CATALOG_TABLE_NAME + + " WHERE " + CATALOG_NAME + " = ? AND " + TABLE_NAMESPACE + " LIKE ?"; + protected static final String DO_COMMIT_CREATE_TABLE_SQL = "INSERT INTO " + CATALOG_TABLE_NAME + + " (" + CATALOG_NAME + ", " + TABLE_NAMESPACE + ", " + TABLE_NAME + + ", " + METADATA_LOCATION + ", " + PREVIOUS_METADATA_LOCATION + ") " + + " VALUES (?,?,?,?,null)"; + private static final Joiner JOINER_DOT = Joiner.on('.'); + private static final Splitter SPLITTER_DOT = Splitter.on('.'); + + private JdbcUtil() { + } + + public static Namespace stringToNamespace(String namespace) { + Preconditions.checkArgument(namespace != null, "Invalid namespace %s", namespace); + return Namespace.of(Iterables.toArray(SPLITTER_DOT.split(namespace), String.class)); + } + + public static String namespaceToString(Namespace namespace) { + return JOINER_DOT.join(namespace.levels()); + } + + public static TableIdentifier stringToTableIdentifier(String tableNamespace, String tableName) { + return TableIdentifier.of(JdbcUtil.stringToNamespace(tableNamespace), tableName); + } + +} diff --git a/core/src/main/java/org/apache/iceberg/jdbc/UncheckedInterruptedException.java b/core/src/main/java/org/apache/iceberg/jdbc/UncheckedInterruptedException.java new file mode 100644 index 000000000000..8ab8bab51fd4 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/jdbc/UncheckedInterruptedException.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.jdbc; + +import com.google.errorprone.annotations.FormatMethod; + +public class UncheckedInterruptedException extends RuntimeException { + + @FormatMethod + public UncheckedInterruptedException(String message, Object... args) { + super(String.format(message, args)); + } + + @FormatMethod + public UncheckedInterruptedException(Throwable cause, String message, Object... args) { + super(String.format(message, args), cause); + } +} diff --git a/core/src/main/java/org/apache/iceberg/jdbc/UncheckedSQLException.java b/core/src/main/java/org/apache/iceberg/jdbc/UncheckedSQLException.java new file mode 100644 index 000000000000..504dc376a88c --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/jdbc/UncheckedSQLException.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.jdbc; + +import com.google.errorprone.annotations.FormatMethod; + +public class UncheckedSQLException extends RuntimeException { + + @FormatMethod + public UncheckedSQLException(String message, Object... args) { + super(String.format(message, args)); + } + + @FormatMethod + public UncheckedSQLException(Throwable cause, String message, Object... args) { + super(String.format(message, args), cause); + } +} diff --git a/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java b/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java index 13a75dbb17cc..2dcaa9fa9407 100644 --- a/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/DateTimeUtil.java @@ -33,6 +33,7 @@ private DateTimeUtil() { public static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC); public static final LocalDate EPOCH_DAY = EPOCH.toLocalDate(); + public static final long MICROS_PER_MILLIS = 1000L; public static LocalDate dateFromDays(int daysFromEpoch) { return ChronoUnit.DAYS.addTo(EPOCH_DAY, daysFromEpoch); @@ -66,6 +67,13 @@ public static long microsFromTimestamp(LocalDateTime dateTime) { return ChronoUnit.MICROS.between(EPOCH, dateTime.atOffset(ZoneOffset.UTC)); } + public static long microsToMillis(long micros) { + // When the timestamp is negative, i.e before 1970, we need to adjust the milliseconds portion. + // Example - 1965-01-01 10:11:12.123456 is represented as (-157700927876544) in micro precision. + // In millis precision the above needs to be represented as (-157700927877). + return Math.floorDiv(micros, MICROS_PER_MILLIS); + } + public static OffsetDateTime timestamptzFromMicros(long microsFromEpoch) { return ChronoUnit.MICROS.addTo(EPOCH, microsFromEpoch); } diff --git a/core/src/main/java/org/apache/iceberg/util/SerializationUtil.java b/core/src/main/java/org/apache/iceberg/util/SerializationUtil.java index 52b14d96b0ca..f7ccd7a8145d 100644 --- a/core/src/main/java/org/apache/iceberg/util/SerializationUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/SerializationUtil.java @@ -27,13 +27,39 @@ import java.io.UncheckedIOException; import java.nio.charset.StandardCharsets; import java.util.Base64; +import java.util.function.Function; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.hadoop.HadoopConfigurable; +import org.apache.iceberg.hadoop.SerializableConfiguration; public class SerializationUtil { private SerializationUtil() { } + /** + * Serialize an object to bytes. If the object implements {@link HadoopConfigurable}, its Hadoop configuration will + * be serialized into a {@link SerializableConfiguration}. + * @param obj object to serialize + * @return serialized bytes + */ public static byte[] serializeToBytes(Object obj) { + return serializeToBytes(obj, conf -> new SerializableConfiguration(conf)::get); + } + + /** + * Serialize an object to bytes. If the object implements {@link HadoopConfigurable}, the confSerializer will be used + * to serialize Hadoop configuration used by the object. + * @param obj object to serialize + * @param confSerializer serializer for the Hadoop configuration + * @return serialized bytes + */ + public static byte[] serializeToBytes(Object obj, + Function> confSerializer) { + if (obj instanceof HadoopConfigurable) { + ((HadoopConfigurable) obj).serializeConfWith(confSerializer); + } + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); ObjectOutputStream oos = new ObjectOutputStream(baos)) { oos.writeObject(obj); diff --git a/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java b/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java index 85f08e057340..beafcc29c90c 100644 --- a/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java @@ -25,6 +25,7 @@ import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -63,6 +64,19 @@ public static List currentAncestors(Table table) { return ancestorIds(table.currentSnapshot(), table::snapshot); } + /** + * Traverses the history of the table's current snapshot and finds the oldest Snapshot. + * @return null if there is no current snapshot in the table, else the oldest Snapshot. + */ + public static Snapshot oldestSnapshot(Table table) { + Snapshot current = table.currentSnapshot(); + while (current != null && current.parentId() != null) { + current = table.snapshot(current.parentId()); + } + + return current; + } + /** * Returns list of snapshot ids in the range - (fromSnapshotId, toSnapshotId] *

    @@ -107,4 +121,27 @@ public static List newFiles(Long baseSnapshotId, long latestSnapshotId return newFiles; } + + /** + * Traverses the history of the table's current snapshot and finds the snapshot with the given snapshot id as its + * parent. + * @return the snapshot for which the given snapshot is the parent + * @throws IllegalArgumentException when the given snapshotId is not found in the table + * @throws IllegalStateException when the given snapshotId is not an ancestor of the current table state + */ + public static Snapshot snapshotAfter(Table table, long snapshotId) { + Preconditions.checkArgument(table.snapshot(snapshotId) != null, "Cannot find parent snapshot: %s", snapshotId); + + Snapshot current = table.currentSnapshot(); + while (current != null) { + if (current.parentId() == snapshotId) { + return current; + } + + current = table.snapshot(current.parentId()); + } + + throw new IllegalStateException( + String.format("Cannot find snapshot after %s: not an ancestor of table's current snapshot", snapshotId)); + } } diff --git a/core/src/main/java/org/apache/iceberg/util/TableScanUtil.java b/core/src/main/java/org/apache/iceberg/util/TableScanUtil.java index a9c60df32cdd..4a1ad5af5df1 100644 --- a/core/src/main/java/org/apache/iceberg/util/TableScanUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/TableScanUtil.java @@ -24,6 +24,7 @@ import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.FluentIterable; public class TableScanUtil { @@ -40,6 +41,8 @@ public static boolean hasDeletes(FileScanTask task) { } public static CloseableIterable splitFiles(CloseableIterable tasks, long splitSize) { + Preconditions.checkArgument(splitSize > 0, "Invalid split size (negative or 0): %s", splitSize); + Iterable splitTasks = FluentIterable .from(tasks) .transformAndConcat(input -> input.split(splitSize)); @@ -49,6 +52,10 @@ public static CloseableIterable splitFiles(CloseableIterable planTasks(CloseableIterable splitFiles, long splitSize, int lookback, long openFileCost) { + Preconditions.checkArgument(splitSize > 0, "Invalid split size (negative or 0): %s", splitSize); + Preconditions.checkArgument(lookback > 0, "Invalid split planning lookback (negative or 0): %s", lookback); + Preconditions.checkArgument(openFileCost >= 0, "Invalid file open cost (negative): %s", openFileCost); + Function weightFunc = file -> Math.max(file.length(), openFileCost); return CloseableIterable.transform( diff --git a/core/src/main/java/org/apache/iceberg/util/Tasks.java b/core/src/main/java/org/apache/iceberg/util/Tasks.java index fb1d0ee667c9..22f47785d1e7 100644 --- a/core/src/main/java/org/apache/iceberg/util/Tasks.java +++ b/core/src/main/java/org/apache/iceberg/util/Tasks.java @@ -35,6 +35,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Predicate; +import java.util.stream.Stream; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -568,6 +569,10 @@ public static Builder foreach(I... items) { return new Builder<>(Arrays.asList(items)); } + public static Builder foreach(Stream items) { + return new Builder<>(() -> items.iterator()); + } + @SuppressWarnings("unchecked") private static void throwOne( Collection exceptions, Class allowedException) throws E { diff --git a/core/src/test/java/org/apache/iceberg/TableTestBase.java b/core/src/test/java/org/apache/iceberg/TableTestBase.java index 4bb6f66c6a6a..c0f60d4e7568 100644 --- a/core/src/test/java/org/apache/iceberg/TableTestBase.java +++ b/core/src/test/java/org/apache/iceberg/TableTestBase.java @@ -106,6 +106,30 @@ public class TableTestBase { .withPartitionPath("data_bucket=3") // easy way to set partition data for now .withRecordCount(1) .build(); + static final DataFile FILE_PARTITION_0 = DataFiles.builder(SPEC) + .withPath("/path/to/data-0.parquet") + .withFileSizeInBytes(10) + .withPartition(TestHelpers.Row.of(0)) + .withRecordCount(1) + .build(); + static final DataFile FILE_PARTITION_1 = DataFiles.builder(SPEC) + .withPath("/path/to/data-1.parquet") + .withFileSizeInBytes(10) + .withPartition(TestHelpers.Row.of(1)) + .withRecordCount(1) + .build(); + static final DataFile FILE_PARTITION_2 = DataFiles.builder(SPEC) + .withPath("/path/to/data-2.parquet") + .withFileSizeInBytes(10) + .withPartition(TestHelpers.Row.of(2)) + .withRecordCount(1) + .build(); + static final DataFile FILE_PARTITION_3 = DataFiles.builder(SPEC) + .withPath("/path/to/data-3.parquet") + .withFileSizeInBytes(10) + .withPartition(TestHelpers.Row.of(3)) + .withRecordCount(1) + .build(); static final FileIO FILE_IO = new TestTables.LocalFileIO(); @@ -308,6 +332,8 @@ void validateSnapshot(Snapshot old, Snapshot snap, Long sequenceNumber, DataFile } Assert.assertFalse("Should find all files in the manifest", newPaths.hasNext()); + + Assert.assertEquals("Schema ID should match", table.schema().schemaId(), (int) snap.schemaId()); } void validateTableFiles(Table tbl, DataFile... expectedFiles) { diff --git a/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java b/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java index a693acfce79c..283f85470723 100644 --- a/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java @@ -32,7 +32,6 @@ import org.junit.runners.Parameterized; import static org.apache.iceberg.PartitionSpec.unpartitioned; -import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; @RunWith(Parameterized.class) @@ -86,7 +85,8 @@ public void testCreateTransactionAndUpdateSchema() throws IOException { TestTables.metadataVersion("test_create")); txn.updateSchema() - .addColumn("col", Types.StringType.get()) + .allowIncompatibleChanges() + .addRequiredColumn("col", Types.StringType.get()) .setIdentifierFields("id", "col") .commit(); @@ -103,7 +103,7 @@ public void testCreateTransactionAndUpdateSchema() throws IOException { Lists.newArrayList( required(1, "id", Types.IntegerType.get()), required(2, "data", Types.StringType.get()), - optional(3, "col", Types.StringType.get())), + required(3, "col", Types.StringType.get())), Sets.newHashSet(1, 3) ); diff --git a/core/src/test/java/org/apache/iceberg/TestManifestListVersions.java b/core/src/test/java/org/apache/iceberg/TestManifestListVersions.java index 0c5b97ce4558..390ba0de12ae 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestListVersions.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestListVersions.java @@ -56,16 +56,17 @@ public class TestManifestListVersions { private static final int DELETED_FILES = 1; private static final long DELETED_ROWS = 22910L; private static final List PARTITION_SUMMARIES = ImmutableList.of(); + private static final ByteBuffer KEY_METADATA = null; private static final ManifestFile TEST_MANIFEST = new GenericManifestFile( PATH, LENGTH, SPEC_ID, ManifestContent.DATA, SEQ_NUM, MIN_SEQ_NUM, SNAPSHOT_ID, ADDED_FILES, ADDED_ROWS, EXISTING_FILES, EXISTING_ROWS, DELETED_FILES, DELETED_ROWS, - PARTITION_SUMMARIES); + PARTITION_SUMMARIES, KEY_METADATA); private static final ManifestFile TEST_DELETE_MANIFEST = new GenericManifestFile( PATH, LENGTH, SPEC_ID, ManifestContent.DELETES, SEQ_NUM, MIN_SEQ_NUM, SNAPSHOT_ID, ADDED_FILES, ADDED_ROWS, EXISTING_FILES, EXISTING_ROWS, DELETED_FILES, DELETED_ROWS, - PARTITION_SUMMARIES); + PARTITION_SUMMARIES, KEY_METADATA); @Rule public TemporaryFolder temp = new TemporaryFolder(); @@ -135,9 +136,9 @@ public void testV1ForwardCompatibility() throws IOException { Assert.assertEquals("Added rows count", ADDED_ROWS, (long) generic.get("added_rows_count")); Assert.assertEquals("Existing rows count", EXISTING_ROWS, (long) generic.get("existing_rows_count")); Assert.assertEquals("Deleted rows count", DELETED_ROWS, (long) generic.get("deleted_rows_count")); - Assert.assertNull("Content", generic.get(ManifestFile.MANIFEST_CONTENT.name())); - Assert.assertNull("Sequence number", generic.get(ManifestFile.SEQUENCE_NUMBER.name())); - Assert.assertNull("Min sequence number", generic.get(ManifestFile.MIN_SEQUENCE_NUMBER.name())); + AssertHelpers.assertEmptyAvroField(generic, ManifestFile.MANIFEST_CONTENT.name()); + AssertHelpers.assertEmptyAvroField(generic, ManifestFile.SEQUENCE_NUMBER.name()); + AssertHelpers.assertEmptyAvroField(generic, ManifestFile.MIN_SEQUENCE_NUMBER.name()); } @Test @@ -157,9 +158,9 @@ public void testV2ForwardCompatibility() throws IOException { Assert.assertEquals("Added rows count", ADDED_ROWS, (long) generic.get("added_rows_count")); Assert.assertEquals("Existing rows count", EXISTING_ROWS, (long) generic.get("existing_rows_count")); Assert.assertEquals("Deleted rows count", DELETED_ROWS, (long) generic.get("deleted_rows_count")); - Assert.assertNull("Content", generic.get(ManifestFile.MANIFEST_CONTENT.name())); - Assert.assertNull("Sequence number", generic.get(ManifestFile.SEQUENCE_NUMBER.name())); - Assert.assertNull("Min sequence number", generic.get(ManifestFile.MIN_SEQUENCE_NUMBER.name())); + AssertHelpers.assertEmptyAvroField(generic, ManifestFile.MANIFEST_CONTENT.name()); + AssertHelpers.assertEmptyAvroField(generic, ManifestFile.SEQUENCE_NUMBER.name()); + AssertHelpers.assertEmptyAvroField(generic, ManifestFile.MIN_SEQUENCE_NUMBER.name()); } @Test @@ -223,7 +224,7 @@ public void testManifestsPartitionSummary() throws IOException { ManifestFile manifest = new GenericManifestFile( PATH, LENGTH, SPEC_ID, ManifestContent.DATA, SEQ_NUM, MIN_SEQ_NUM, SNAPSHOT_ID, ADDED_FILES, ADDED_ROWS, EXISTING_FILES, EXISTING_ROWS, DELETED_FILES, DELETED_ROWS, - partitionFieldSummaries); + partitionFieldSummaries, KEY_METADATA); InputFile manifestList = writeManifestList(manifest, 2); diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java index 711aaa7d1f3a..a6a411ec3273 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java @@ -20,9 +20,12 @@ package org.apache.iceberg; import java.io.IOException; +import java.util.stream.StreamSupport; +import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Iterators; import org.apache.iceberg.types.Types; import org.junit.Assert; import org.junit.Test; @@ -159,6 +162,194 @@ public void testAllManifestsTableHonorsIgnoreResiduals() throws IOException { validateTaskScanResiduals(scan2, true); } + @Test + public void testPartitionsTableScanNoFilter() { + table.newFastAppend() + .appendFile(FILE_PARTITION_0) + .commit(); + table.newFastAppend() + .appendFile(FILE_PARTITION_1) + .commit(); + table.newFastAppend() + .appendFile(FILE_PARTITION_2) + .commit(); + table.newFastAppend() + .appendFile(FILE_PARTITION_3) + .commit(); + + Table partitionsTable = new PartitionsTable(table.ops(), table); + Types.StructType expected = new Schema( + required(1, "partition", Types.StructType.of( + optional(1000, "data_bucket", Types.IntegerType.get())))).asStruct(); + + TableScan scanNoFilter = partitionsTable.newScan().select("partition.data_bucket"); + Assert.assertEquals(expected, scanNoFilter.schema().asStruct()); + CloseableIterable tasksNoFilter = PartitionsTable.planFiles((StaticTableScan) scanNoFilter); + Assert.assertEquals(4, Iterators.size(tasksNoFilter.iterator())); + validateIncludesPartitionScan(tasksNoFilter, 0); + validateIncludesPartitionScan(tasksNoFilter, 1); + validateIncludesPartitionScan(tasksNoFilter, 2); + validateIncludesPartitionScan(tasksNoFilter, 3); + } + + @Test + public void testPartitionsTableScanAndFilter() { + table.newFastAppend() + .appendFile(FILE_PARTITION_0) + .commit(); + table.newFastAppend() + .appendFile(FILE_PARTITION_1) + .commit(); + table.newFastAppend() + .appendFile(FILE_PARTITION_2) + .commit(); + table.newFastAppend() + .appendFile(FILE_PARTITION_3) + .commit(); + + Table partitionsTable = new PartitionsTable(table.ops(), table); + + Expression andEquals = Expressions.and( + Expressions.equal("partition.data_bucket", 0), + Expressions.greaterThan("record_count", 0)); + TableScan scanAndEq = partitionsTable.newScan().filter(andEquals); + CloseableIterable tasksAndEq = PartitionsTable.planFiles((StaticTableScan) scanAndEq); + Assert.assertEquals(1, Iterators.size(tasksAndEq.iterator())); + validateIncludesPartitionScan(tasksAndEq, 0); + } + + @Test + public void testPartitionsTableScanLtFilter() { + table.newFastAppend() + .appendFile(FILE_PARTITION_0) + .commit(); + table.newFastAppend() + .appendFile(FILE_PARTITION_1) + .commit(); + table.newFastAppend() + .appendFile(FILE_PARTITION_2) + .commit(); + table.newFastAppend() + .appendFile(FILE_PARTITION_3) + .commit(); + + Table partitionsTable = new PartitionsTable(table.ops(), table); + + Expression ltAnd = Expressions.and( + Expressions.lessThan("partition.data_bucket", 2), + Expressions.greaterThan("record_count", 0)); + TableScan scanLtAnd = partitionsTable.newScan().filter(ltAnd); + CloseableIterable tasksLtAnd = PartitionsTable.planFiles((StaticTableScan) scanLtAnd); + Assert.assertEquals(2, Iterators.size(tasksLtAnd.iterator())); + validateIncludesPartitionScan(tasksLtAnd, 0); + validateIncludesPartitionScan(tasksLtAnd, 1); + } + + @Test + public void testPartitionsTableScanOrFilter() { + table.newFastAppend() + .appendFile(FILE_PARTITION_0) + .commit(); + table.newFastAppend() + .appendFile(FILE_PARTITION_1) + .commit(); + table.newFastAppend() + .appendFile(FILE_PARTITION_2) + .commit(); + table.newFastAppend() + .appendFile(FILE_PARTITION_3) + .commit(); + + Table partitionsTable = new PartitionsTable(table.ops(), table); + + Expression or = Expressions.or( + Expressions.equal("partition.data_bucket", 2), + Expressions.greaterThan("record_count", 0)); + TableScan scanOr = partitionsTable.newScan().filter(or); + CloseableIterable tasksOr = PartitionsTable.planFiles((StaticTableScan) scanOr); + Assert.assertEquals(4, Iterators.size(tasksOr.iterator())); + validateIncludesPartitionScan(tasksOr, 0); + validateIncludesPartitionScan(tasksOr, 1); + validateIncludesPartitionScan(tasksOr, 2); + validateIncludesPartitionScan(tasksOr, 3); + } + + @Test + public void testPartitionsScanNotFilter() { + table.newFastAppend() + .appendFile(FILE_PARTITION_0) + .commit(); + table.newFastAppend() + .appendFile(FILE_PARTITION_1) + .commit(); + table.newFastAppend() + .appendFile(FILE_PARTITION_2) + .commit(); + table.newFastAppend() + .appendFile(FILE_PARTITION_3) + .commit(); + Table partitionsTable = new PartitionsTable(table.ops(), table); + + Expression not = Expressions.not(Expressions.lessThan("partition.data_bucket", 2)); + TableScan scanNot = partitionsTable.newScan().filter(not); + CloseableIterable tasksNot = PartitionsTable.planFiles((StaticTableScan) scanNot); + Assert.assertEquals(2, Iterators.size(tasksNot.iterator())); + validateIncludesPartitionScan(tasksNot, 2); + validateIncludesPartitionScan(tasksNot, 3); + } + + @Test + public void testPartitionsTableScanInFilter() { + table.newFastAppend() + .appendFile(FILE_PARTITION_0) + .commit(); + table.newFastAppend() + .appendFile(FILE_PARTITION_1) + .commit(); + table.newFastAppend() + .appendFile(FILE_PARTITION_2) + .commit(); + table.newFastAppend() + .appendFile(FILE_PARTITION_3) + .commit(); + + Table partitionsTable = new PartitionsTable(table.ops(), table); + + Expression set = Expressions.in("partition.data_bucket", 2, 3); + TableScan scanSet = partitionsTable.newScan().filter(set); + CloseableIterable tasksSet = PartitionsTable.planFiles((StaticTableScan) scanSet); + Assert.assertEquals(2, Iterators.size(tasksSet.iterator())); + validateIncludesPartitionScan(tasksSet, 2); + validateIncludesPartitionScan(tasksSet, 3); + } + + @Test + public void testPartitionsTableScanNotNullFilter() { + table.newFastAppend() + .appendFile(FILE_PARTITION_0) + .commit(); + table.newFastAppend() + .appendFile(FILE_PARTITION_1) + .commit(); + table.newFastAppend() + .appendFile(FILE_PARTITION_2) + .commit(); + table.newFastAppend() + .appendFile(FILE_PARTITION_3) + .commit(); + + Table partitionsTable = new PartitionsTable(table.ops(), table); + + Expression unary = Expressions.notNull("partition.data_bucket"); + TableScan scanUnary = partitionsTable.newScan().filter(unary); + CloseableIterable tasksUnary = PartitionsTable.planFiles((StaticTableScan) scanUnary); + Assert.assertEquals(4, Iterators.size(tasksUnary.iterator())); + validateIncludesPartitionScan(tasksUnary, 0); + validateIncludesPartitionScan(tasksUnary, 1); + validateIncludesPartitionScan(tasksUnary, 2); + validateIncludesPartitionScan(tasksUnary, 3); + } + @Test public void testDataFilesTableSelection() throws IOException { table.newFastAppend() @@ -194,4 +385,10 @@ private void validateTaskScanResiduals(TableScan scan, boolean ignoreResiduals) } } } + + private void validateIncludesPartitionScan(CloseableIterable tasks, int partValue) { + Assert.assertTrue("File scan tasks do not include correct file", + StreamSupport.stream(tasks.spliterator(), false).anyMatch( + a -> a.file().partition().get(0, Object.class).equals(partValue))); + } } diff --git a/core/src/test/java/org/apache/iceberg/TestMetrics.java b/core/src/test/java/org/apache/iceberg/TestMetrics.java index 2e7cdbf5d355..47b79a655af6 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetrics.java +++ b/core/src/test/java/org/apache/iceberg/TestMetrics.java @@ -274,7 +274,7 @@ public void testMetricsForNestedStructFields() throws IOException { assertBounds(6, BinaryType.get(), ByteBuffer.wrap("A".getBytes()), ByteBuffer.wrap("A".getBytes()), metrics); assertCounts(7, 1L, 0L, 1L, metrics); - assertBounds(7, DoubleType.get(), Double.NaN, Double.NaN, metrics); + assertBounds(7, DoubleType.get(), null, null, metrics); } private Record buildNestedTestRecord() { @@ -354,9 +354,9 @@ public void testMetricsForNaNColumns() throws IOException { Assert.assertEquals(2L, (long) metrics.recordCount()); assertCounts(1, 2L, 0L, 2L, metrics); assertCounts(2, 2L, 0L, 2L, metrics); - // below: current behavior; will be null once NaN is excluded from upper/lower bound - assertBounds(1, FloatType.get(), Float.NaN, Float.NaN, metrics); - assertBounds(2, DoubleType.get(), Double.NaN, Double.NaN, metrics); + + assertBounds(1, FloatType.get(), null, null, metrics); + assertBounds(2, DoubleType.get(), null, null, metrics); } @Test @@ -367,15 +367,8 @@ public void testColumnBoundsWithNaNValueAtFront() throws IOException { assertCounts(1, 3L, 0L, 1L, metrics); assertCounts(2, 3L, 0L, 1L, metrics); - // below: current behavior; will be non-NaN values once NaN is excluded from upper/lower bound. ORC and Parquet's - // behaviors differ due to their implementation of comparison being different. - if (fileFormat() == FileFormat.ORC) { - assertBounds(1, FloatType.get(), Float.NaN, Float.NaN, metrics); - assertBounds(2, DoubleType.get(), Double.NaN, Double.NaN, metrics); - } else { - assertBounds(1, FloatType.get(), 1.2F, Float.NaN, metrics); - assertBounds(2, DoubleType.get(), 3.4D, Double.NaN, metrics); - } + assertBounds(1, FloatType.get(), 1.2F, 5.6F, metrics); + assertBounds(2, DoubleType.get(), 3.4D, 7.8D, metrics); } @Test @@ -386,15 +379,8 @@ public void testColumnBoundsWithNaNValueInMiddle() throws IOException { assertCounts(1, 3L, 0L, 1L, metrics); assertCounts(2, 3L, 0L, 1L, metrics); - // below: current behavior; will be non-NaN values once NaN is excluded from upper/lower bound. ORC and Parquet's - // behaviors differ due to their implementation of comparison being different. - if (fileFormat() == FileFormat.ORC) { - assertBounds(1, FloatType.get(), 1.2F, 5.6F, metrics); - assertBounds(2, DoubleType.get(), 3.4D, 7.8D, metrics); - } else { - assertBounds(1, FloatType.get(), 1.2F, Float.NaN, metrics); - assertBounds(2, DoubleType.get(), 3.4D, Double.NaN, metrics); - } + assertBounds(1, FloatType.get(), 1.2F, 5.6F, metrics); + assertBounds(2, DoubleType.get(), 3.4D, 7.8D, metrics); } @Test @@ -405,15 +391,8 @@ public void testColumnBoundsWithNaNValueAtEnd() throws IOException { assertCounts(1, 3L, 0L, 1L, metrics); assertCounts(2, 3L, 0L, 1L, metrics); - // below: current behavior; will be non-NaN values once NaN is excluded from upper/lower bound. ORC and Parquet's - // behaviors differ due to their implementation of comparison being different. - if (fileFormat() == FileFormat.ORC) { - assertBounds(1, FloatType.get(), 1.2F, 5.6F, metrics); - assertBounds(2, DoubleType.get(), 3.4D, 7.8D, metrics); - } else { - assertBounds(1, FloatType.get(), 1.2F, Float.NaN, metrics); - assertBounds(2, DoubleType.get(), 3.4D, Double.NaN, metrics); - } + assertBounds(1, FloatType.get(), 1.2F, 5.6F, metrics); + assertBounds(2, DoubleType.get(), 3.4D, 7.8D, metrics); } @Test @@ -506,7 +485,7 @@ public void testMetricsForNestedStructFieldsWithMultipleRowGroup() throws IOExce assertBounds(6, BinaryType.get(), ByteBuffer.wrap("A".getBytes()), ByteBuffer.wrap("A".getBytes()), metrics); assertCounts(7, 201L, 0L, 201L, metrics); - assertBounds(7, DoubleType.get(), Double.NaN, Double.NaN, metrics); + assertBounds(7, DoubleType.get(), null, null, metrics); } @Test @@ -567,7 +546,7 @@ public void testFullMetricsMode() throws IOException { assertBounds(6, Types.BinaryType.get(), ByteBuffer.wrap("A".getBytes()), ByteBuffer.wrap("A".getBytes()), metrics); assertCounts(7, 1L, 0L, 1L, metrics); - assertBounds(7, Types.DoubleType.get(), Double.NaN, Double.NaN, metrics); + assertBounds(7, Types.DoubleType.get(), null, null, metrics); } @Test diff --git a/core/src/test/java/org/apache/iceberg/TestSchemaID.java b/core/src/test/java/org/apache/iceberg/TestSchemaID.java new file mode 100644 index 000000000000..f6c49798619d --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestSchemaID.java @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg; + +import java.util.Arrays; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +@RunWith(Parameterized.class) +public class TestSchemaID extends TableTestBase { + + @Parameterized.Parameters(name = "formatVersion = {0}") + public static Object[] parameters() { + return new Object[] { 1, 2 }; + } + + public TestSchemaID(int formatVersion) { + super(formatVersion); + } + + @Test + public void testNoChange() { + int onlyId = table.schema().schemaId(); + Map onlySchemaMap = schemaMap(table.schema()); + + // add files to table + table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + + TestHelpers.assertSameSchemaMap(onlySchemaMap, table.schemas()); + Assert.assertEquals("Current snapshot's schemaId should be the current", + table.schema().schemaId(), (int) table.currentSnapshot().schemaId()); + + Assert.assertEquals("Schema ids should be correct in snapshots", + ImmutableList.of(onlyId), + Lists.transform(Lists.newArrayList(table.snapshots()), Snapshot::schemaId)); + + // remove file from table + table.newDelete().deleteFile(FILE_A).commit(); + + TestHelpers.assertSameSchemaMap(onlySchemaMap, table.schemas()); + Assert.assertEquals("Current snapshot's schemaId should be the current", + table.schema().schemaId(), (int) table.currentSnapshot().schemaId()); + + Assert.assertEquals("Schema ids should be correct in snapshots", + ImmutableList.of(onlyId, onlyId), + Lists.transform(Lists.newArrayList(table.snapshots()), Snapshot::schemaId)); + + // add file to table + table.newFastAppend().appendFile(FILE_A2).commit(); + + TestHelpers.assertSameSchemaMap(onlySchemaMap, table.schemas()); + Assert.assertEquals("Current snapshot's schemaId should be the current", + table.schema().schemaId(), (int) table.currentSnapshot().schemaId()); + + Assert.assertEquals("Schema ids should be correct in snapshots", + ImmutableList.of(onlyId, onlyId, onlyId), + Lists.transform(Lists.newArrayList(table.snapshots()), Snapshot::schemaId)); + } + + @Test + public void testSchemaIdChangeInSchemaUpdate() { + Schema originalSchema = table.schema(); + + // add files to table + table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + + TestHelpers.assertSameSchemaMap(schemaMap(table.schema()), table.schemas()); + Assert.assertEquals("Current snapshot's schemaId should be the current", + table.schema().schemaId(), (int) table.currentSnapshot().schemaId()); + + Assert.assertEquals("Schema ids should be correct in snapshots", + ImmutableList.of(originalSchema.schemaId()), + Lists.transform(Lists.newArrayList(table.snapshots()), Snapshot::schemaId)); + + // update schema + table.updateSchema().addColumn("data2", Types.StringType.get()).commit(); + + Schema updatedSchema = new Schema(1, + required(1, "id", Types.IntegerType.get()), + required(2, "data", Types.StringType.get()), + optional(3, "data2", Types.StringType.get()) + ); + + TestHelpers.assertSameSchemaMap(schemaMap(originalSchema, updatedSchema), table.schemas()); + Assert.assertEquals("Current snapshot's schemaId should be old since update schema doesn't create new snapshot", + originalSchema.schemaId(), (int) table.currentSnapshot().schemaId()); + Assert.assertEquals("Current schema should match", updatedSchema.asStruct(), table.schema().asStruct()); + + Assert.assertEquals("Schema ids should be correct in snapshots", + ImmutableList.of(originalSchema.schemaId()), + Lists.transform(Lists.newArrayList(table.snapshots()), Snapshot::schemaId)); + + // remove file from table + table.newDelete().deleteFile(FILE_A).commit(); + + TestHelpers.assertSameSchemaMap(schemaMap(originalSchema, updatedSchema), table.schemas()); + Assert.assertEquals("Current snapshot's schemaId should be the current", + updatedSchema.schemaId(), (int) table.currentSnapshot().schemaId()); + Assert.assertEquals("Current schema should match", updatedSchema.asStruct(), table.schema().asStruct()); + + Assert.assertEquals("Schema ids should be correct in snapshots", + ImmutableList.of(originalSchema.schemaId(), updatedSchema.schemaId()), + Lists.transform(Lists.newArrayList(table.snapshots()), Snapshot::schemaId)); + + // add files to table + table.newAppend().appendFile(FILE_A2).commit(); + + TestHelpers.assertSameSchemaMap(schemaMap(originalSchema, updatedSchema), table.schemas()); + Assert.assertEquals("Current snapshot's schemaId should be the current", + updatedSchema.schemaId(), (int) table.currentSnapshot().schemaId()); + Assert.assertEquals("Current schema should match", updatedSchema.asStruct(), table.schema().asStruct()); + + Assert.assertEquals("Schema ids should be correct in snapshots", + ImmutableList.of(originalSchema.schemaId(), updatedSchema.schemaId(), updatedSchema.schemaId()), + Lists.transform(Lists.newArrayList(table.snapshots()), Snapshot::schemaId)); + } + + private Map schemaMap(Schema... schemas) { + return Arrays.stream(schemas).collect(Collectors.toMap(Schema::schemaId, Function.identity())); + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java b/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java index 6b8380e31b94..28b918649a05 100644 --- a/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java +++ b/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java @@ -1238,7 +1238,8 @@ public void testAddExistingIdentifierFields() { @Test public void testAddNewIdentifierFieldColumns() { Schema newSchema = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) - .addColumn("new_field", Types.StringType.get()) + .allowIncompatibleChanges() + .addRequiredColumn("new_field", Types.StringType.get()) .setIdentifierFields("id", "new_field") .apply(); @@ -1247,8 +1248,9 @@ public void testAddNewIdentifierFieldColumns() { newSchema.identifierFieldIds()); newSchema = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) + .allowIncompatibleChanges() .setIdentifierFields("id", "new_field") - .addColumn("new_field", Types.StringType.get()) + .addRequiredColumn("new_field", Types.StringType.get()) .apply(); Assert.assertEquals("set identifier then add column should succeed", @@ -1267,8 +1269,9 @@ public void testAddNestedIdentifierFieldColumns() { newSchema.identifierFieldIds()); newSchema = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) - .addColumn("new", Types.StructType.of( - Types.NestedField.optional(SCHEMA_LAST_COLUMN_ID + 1, "field", Types.StringType.get()) + .allowIncompatibleChanges() + .addRequiredColumn("new", Types.StructType.of( + Types.NestedField.required(SCHEMA_LAST_COLUMN_ID + 1, "field", Types.StringType.get()) )) .setIdentifierFields("new.field") .apply(); @@ -1278,9 +1281,10 @@ public void testAddNestedIdentifierFieldColumns() { newSchema.identifierFieldIds()); newSchema = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) - .addColumn("new", Types.StructType.of( - Types.NestedField.optional(SCHEMA_LAST_COLUMN_ID + 1, "field", Types.StructType.of( - Types.NestedField.optional(SCHEMA_LAST_COLUMN_ID + 2, "nested", Types.StringType.get()))))) + .allowIncompatibleChanges() + .addRequiredColumn("new", Types.StructType.of( + Types.NestedField.required(SCHEMA_LAST_COLUMN_ID + 1, "field", Types.StructType.of( + Types.NestedField.required(SCHEMA_LAST_COLUMN_ID + 2, "nested", Types.StringType.get()))))) .setIdentifierFields("new.field.nested") .apply(); @@ -1292,7 +1296,8 @@ public void testAddNestedIdentifierFieldColumns() { @Test public void testAddDottedIdentifierFieldColumns() { Schema newSchema = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) - .addColumn(null, "dot.field", Types.StringType.get()) + .allowIncompatibleChanges() + .addRequiredColumn(null, "dot.field", Types.StringType.get()) .setIdentifierFields("id", "dot.field") .apply(); @@ -1304,8 +1309,9 @@ public void testAddDottedIdentifierFieldColumns() { @Test public void testRemoveIdentifierFields() { Schema newSchema = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) - .addColumn("new_field", Types.StringType.get()) - .addColumn("new_field2", Types.StringType.get()) + .allowIncompatibleChanges() + .addRequiredColumn("new_field", Types.StringType.get()) + .addRequiredColumn("new_field2", Types.StringType.get()) .setIdentifierFields("id", "new_field", "new_field2") .apply(); @@ -1342,18 +1348,18 @@ public void testSetIdentifierFieldsFails() { .setIdentifierFields("locations") .apply()); - AssertHelpers.assertThrows("add a map key nested field should fail", + AssertHelpers.assertThrows("add an optional field should fail", IllegalArgumentException.class, - "must not be nested in " + SCHEMA.findField("locations"), + "not a required field", () -> new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) - .setIdentifierFields("locations.key.zip") + .setIdentifierFields("data") .apply()); - AssertHelpers.assertThrows("add a map value nested field should fail", + AssertHelpers.assertThrows("add a map key nested field should fail", IllegalArgumentException.class, "must not be nested in " + SCHEMA.findField("locations"), () -> new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) - .setIdentifierFields("locations.value.lat") + .setIdentifierFields("locations.key.zip") .apply()); AssertHelpers.assertThrows("add a nested field in list should fail", @@ -1364,19 +1370,52 @@ public void testSetIdentifierFieldsFails() { .apply()); Schema newSchema = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) - .addColumn("new", Types.StructType.of( - Types.NestedField.optional(SCHEMA_LAST_COLUMN_ID + 1, "fields", Types.ListType.ofOptional( - SCHEMA_LAST_COLUMN_ID + 2, Types.StructType.of( - Types.NestedField.optional(SCHEMA_LAST_COLUMN_ID + 3, "nested", Types.StringType.get()) + .allowIncompatibleChanges() + .addRequiredColumn("col_float", Types.FloatType.get()) + .addRequiredColumn("col_double", Types.DoubleType.get()) + .addRequiredColumn("new", Types.StructType.of( + Types.NestedField.required(SCHEMA_LAST_COLUMN_ID + 3, "fields", Types.ListType.ofOptional( + SCHEMA_LAST_COLUMN_ID + 4, Types.StructType.of( + Types.NestedField.required(SCHEMA_LAST_COLUMN_ID + 5, "nested", Types.StringType.get()) )) ) )) + .addRequiredColumn("new_map", Types.MapType.ofRequired(SCHEMA_LAST_COLUMN_ID + 6, 11, + Types.StructType.of( + required(SCHEMA_LAST_COLUMN_ID + 7, "key_col", Types.StringType.get()) + ), + Types.StructType.of( + required(SCHEMA_LAST_COLUMN_ID + 8, "val_col", Types.StringType.get()) + )), "map of address to coordinate") .apply(); + int lastColId = SCHEMA_LAST_COLUMN_ID + 8; + + AssertHelpers.assertThrows("add a double field should fail", + IllegalArgumentException.class, + "must not be float or double field", + () -> new SchemaUpdate(newSchema, lastColId) + .setIdentifierFields("col_double") + .apply()); + + AssertHelpers.assertThrows("add a float field should fail", + IllegalArgumentException.class, + "must not be float or double field", + () -> new SchemaUpdate(newSchema, lastColId) + .setIdentifierFields("col_float") + .apply()); + + AssertHelpers.assertThrows("add a map value nested field should fail", + IllegalArgumentException.class, + "must not be nested in " + newSchema.findField("new_map"), + () -> new SchemaUpdate(newSchema, lastColId) + .setIdentifierFields("new_map.value.val_col") + .apply()); + AssertHelpers.assertThrows("add a nested field in struct of a map should fail", IllegalArgumentException.class, "must not be nested in " + newSchema.findField("new.fields"), - () -> new SchemaUpdate(newSchema, SCHEMA_LAST_COLUMN_ID + 3) + () -> new SchemaUpdate(newSchema, lastColId) .setIdentifierFields("new.fields.element.nested") .apply()); } diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java index 664cdc5756fd..22f0b0ccdbc6 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java @@ -39,7 +39,7 @@ public class TestSnapshotJson { @Test public void testJsonConversion() { - Snapshot expected = new BaseSnapshot(ops.io(), System.currentTimeMillis(), + Snapshot expected = new BaseSnapshot(ops.io(), System.currentTimeMillis(), 1, "file:/tmp/manifest1.avro", "file:/tmp/manifest2.avro"); String json = SnapshotParser.toJson(expected); Snapshot snapshot = SnapshotParser.fromJson(ops.io(), json); @@ -50,6 +50,23 @@ public void testJsonConversion() { expected.allManifests(), snapshot.allManifests()); Assert.assertNull("Operation should be null", snapshot.operation()); Assert.assertNull("Summary should be null", snapshot.summary()); + Assert.assertEquals("Schema ID should match", Integer.valueOf(1), snapshot.schemaId()); + } + + @Test + public void testJsonConversionWithoutSchemaId() { + Snapshot expected = new BaseSnapshot(ops.io(), System.currentTimeMillis(), null, + "file:/tmp/manifest1.avro", "file:/tmp/manifest2.avro"); + String json = SnapshotParser.toJson(expected); + Snapshot snapshot = SnapshotParser.fromJson(ops.io(), json); + + Assert.assertEquals("Snapshot ID should match", + expected.snapshotId(), snapshot.snapshotId()); + Assert.assertEquals("Files should match", + expected.allManifests(), snapshot.allManifests()); + Assert.assertNull("Operation should be null", snapshot.operation()); + Assert.assertNull("Summary should be null", snapshot.summary()); + Assert.assertNull("Schema ID should be null", snapshot.schemaId()); } @Test @@ -62,7 +79,7 @@ public void testJsonConversionWithOperation() { Snapshot expected = new BaseSnapshot(ops.io(), id, parentId, System.currentTimeMillis(), DataOperations.REPLACE, ImmutableMap.of("files-added", "4", "files-deleted", "100"), - manifests); + 3, manifests); String json = SnapshotParser.toJson(expected); Snapshot snapshot = SnapshotParser.fromJson(ops.io(), json); @@ -83,6 +100,8 @@ public void testJsonConversionWithOperation() { expected.operation(), snapshot.operation()); Assert.assertEquals("Summary should match", expected.summary(), snapshot.summary()); + Assert.assertEquals("Schema ID should match", + expected.schemaId(), snapshot.schemaId()); } @Test @@ -102,9 +121,10 @@ public void testJsonConversionWithManifestList() throws IOException { } Snapshot expected = new BaseSnapshot( - ops.io(), id, 34, parentId, System.currentTimeMillis(), null, null, localInput(manifestList).location()); + ops.io(), id, 34, parentId, System.currentTimeMillis(), + null, null, 4, localInput(manifestList).location()); Snapshot inMemory = new BaseSnapshot( - ops.io(), id, parentId, expected.timestampMillis(), null, null, manifests); + ops.io(), id, parentId, expected.timestampMillis(), null, null, 4, manifests); Assert.assertEquals("Files should match in memory list", inMemory.allManifests(), expected.allManifests()); @@ -126,5 +146,6 @@ public void testJsonConversionWithManifestList() throws IOException { expected.allManifests(), snapshot.allManifests()); Assert.assertNull("Operation should be null", snapshot.operation()); Assert.assertNull("Summary should be null", snapshot.summary()); + Assert.assertEquals("Schema ID should match", expected.schemaId(), snapshot.schemaId()); } } diff --git a/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java b/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java index c5b9ac84f6a7..f211ac9175e3 100644 --- a/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java +++ b/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java @@ -173,6 +173,39 @@ public void testSplitPlanningWithOverridenOpenCostSize() { Assert.assertEquals(4, Iterables.size(scan.planTasks())); } + @Test + public void testSplitPlanningWithNegativeValues() { + AssertHelpers.assertThrows( + "User provided split size should be validated", + IllegalArgumentException.class, + "Invalid split size (negative or 0): -10", + () -> { + table.newScan() + .option(TableProperties.SPLIT_SIZE, String.valueOf(-10)) + .planTasks(); + }); + + AssertHelpers.assertThrows( + "User provided split planning lookback should be validated", + IllegalArgumentException.class, + "Invalid split planning lookback (negative or 0): -10", + () -> { + table.newScan() + .option(TableProperties.SPLIT_LOOKBACK, String.valueOf(-10)) + .planTasks(); + }); + + AssertHelpers.assertThrows( + "User provided split open file cost should be validated", + IllegalArgumentException.class, + "Invalid file open cost (negative): -10", + () -> { + table.newScan() + .option(TableProperties.SPLIT_OPEN_FILE_COST, String.valueOf(-10)) + .planTasks(); + }); + } + private void appendFiles(Iterable files) { AppendFiles appendFiles = table.newAppend(); files.forEach(appendFiles::appendFile); diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index c44ca63a126a..d3cebec1f328 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -89,11 +89,11 @@ public class TestTableMetadata { public void testJsonConversion() throws Exception { long previousSnapshotId = System.currentTimeMillis() - new Random(1234).nextInt(3600); Snapshot previousSnapshot = new BaseSnapshot( - ops.io(), previousSnapshotId, null, previousSnapshotId, null, null, ImmutableList.of( + ops.io(), previousSnapshotId, null, previousSnapshotId, null, null, null, ImmutableList.of( new GenericManifestFile(localInput("file:/tmp/manfiest.1.avro"), SPEC_5.specId()))); long currentSnapshotId = System.currentTimeMillis(); Snapshot currentSnapshot = new BaseSnapshot( - ops.io(), currentSnapshotId, previousSnapshotId, currentSnapshotId, null, null, ImmutableList.of( + ops.io(), currentSnapshotId, previousSnapshotId, currentSnapshotId, null, null, 7, ImmutableList.of( new GenericManifestFile(localInput("file:/tmp/manfiest.2.avro"), SPEC_5.specId()))); List snapshotLog = ImmutableList.builder() @@ -108,7 +108,6 @@ public void testJsonConversion() throws Exception { SEQ_NO, System.currentTimeMillis(), 3, 7, ImmutableList.of(TEST_SCHEMA, schema), 5, ImmutableList.of(SPEC_5), SPEC_5.lastAssignedFieldId(), - 3, ImmutableList.of(SORT_ORDER_3), ImmutableMap.of("property", "value"), currentSnapshotId, Arrays.asList(previousSnapshot, currentSnapshot), snapshotLog, ImmutableList.of()); @@ -153,11 +152,15 @@ public void testJsonConversion() throws Exception { (Long) previousSnapshotId, metadata.currentSnapshot().parentId()); Assert.assertEquals("Current snapshot files should match", currentSnapshot.allManifests(), metadata.currentSnapshot().allManifests()); + Assert.assertEquals("Schema ID for current snapshot should match", + (Integer) 7, metadata.currentSnapshot().schemaId()); Assert.assertEquals("Previous snapshot ID should match", previousSnapshotId, metadata.snapshot(previousSnapshotId).snapshotId()); Assert.assertEquals("Previous snapshot files should match", previousSnapshot.allManifests(), metadata.snapshot(previousSnapshotId).allManifests()); + Assert.assertNull("Previous snapshot's schema ID should be null", + metadata.snapshot(previousSnapshotId).schemaId()); } @Test @@ -168,11 +171,11 @@ public void testBackwardCompat() throws Exception { long previousSnapshotId = System.currentTimeMillis() - new Random(1234).nextInt(3600); Snapshot previousSnapshot = new BaseSnapshot( - ops.io(), previousSnapshotId, null, previousSnapshotId, null, null, ImmutableList.of( + ops.io(), previousSnapshotId, null, previousSnapshotId, null, null, null, ImmutableList.of( new GenericManifestFile(localInput("file:/tmp/manfiest.1.avro"), spec.specId()))); long currentSnapshotId = System.currentTimeMillis(); Snapshot currentSnapshot = new BaseSnapshot( - ops.io(), currentSnapshotId, previousSnapshotId, currentSnapshotId, null, null, ImmutableList.of( + ops.io(), currentSnapshotId, previousSnapshotId, currentSnapshotId, null, null, null, ImmutableList.of( new GenericManifestFile(localInput("file:/tmp/manfiest.2.avro"), spec.specId()))); TableMetadata expected = new TableMetadata(null, 1, null, TEST_LOCATION, @@ -222,6 +225,8 @@ public void testBackwardCompat() throws Exception { (Long) previousSnapshotId, metadata.currentSnapshot().parentId()); Assert.assertEquals("Current snapshot files should match", currentSnapshot.allManifests(), metadata.currentSnapshot().allManifests()); + Assert.assertNull("Current snapshot's schema ID should be null", + metadata.currentSnapshot().schemaId()); Assert.assertEquals("Previous snapshot ID should match", previousSnapshotId, metadata.snapshot(previousSnapshotId).snapshotId()); Assert.assertEquals("Previous snapshot files should match", @@ -229,6 +234,8 @@ public void testBackwardCompat() throws Exception { metadata.snapshot(previousSnapshotId).allManifests()); Assert.assertEquals("Snapshot logs should match", expected.previousFiles(), metadata.previousFiles()); + Assert.assertNull("Previous snapshot's schema ID should be null", + metadata.snapshot(previousSnapshotId).schemaId()); } private static String toJsonWithoutSpecAndSchemaList(TableMetadata metadata) { @@ -245,7 +252,7 @@ private static String toJsonWithoutSpecAndSchemaList(TableMetadata metadata) { // mimic an old writer by writing only schema and not the current ID or schema list generator.writeFieldName(SCHEMA); - SchemaParser.toJson(metadata.schema(), generator); + SchemaParser.toJson(metadata.schema().asStruct(), generator); // mimic an old writer by writing only partition-spec and not the default ID or spec list generator.writeFieldName(PARTITION_SPEC); @@ -280,11 +287,11 @@ private static String toJsonWithoutSpecAndSchemaList(TableMetadata metadata) { public void testJsonWithPreviousMetadataLog() throws Exception { long previousSnapshotId = System.currentTimeMillis() - new Random(1234).nextInt(3600); Snapshot previousSnapshot = new BaseSnapshot( - ops.io(), previousSnapshotId, null, previousSnapshotId, null, null, ImmutableList.of( + ops.io(), previousSnapshotId, null, previousSnapshotId, null, null, null, ImmutableList.of( new GenericManifestFile(localInput("file:/tmp/manfiest.1.avro"), SPEC_5.specId()))); long currentSnapshotId = System.currentTimeMillis(); Snapshot currentSnapshot = new BaseSnapshot( - ops.io(), currentSnapshotId, previousSnapshotId, currentSnapshotId, null, null, ImmutableList.of( + ops.io(), currentSnapshotId, previousSnapshotId, currentSnapshotId, null, null, null, ImmutableList.of( new GenericManifestFile(localInput("file:/tmp/manfiest.2.avro"), SPEC_5.specId()))); List reversedSnapshotLog = Lists.newArrayList(); @@ -311,11 +318,11 @@ public void testJsonWithPreviousMetadataLog() throws Exception { public void testAddPreviousMetadataRemoveNone() { long previousSnapshotId = System.currentTimeMillis() - new Random(1234).nextInt(3600); Snapshot previousSnapshot = new BaseSnapshot( - ops.io(), previousSnapshotId, null, previousSnapshotId, null, null, ImmutableList.of( + ops.io(), previousSnapshotId, null, previousSnapshotId, null, null, null, ImmutableList.of( new GenericManifestFile(localInput("file:/tmp/manfiest.1.avro"), SPEC_5.specId()))); long currentSnapshotId = System.currentTimeMillis(); Snapshot currentSnapshot = new BaseSnapshot( - ops.io(), currentSnapshotId, previousSnapshotId, currentSnapshotId, null, null, ImmutableList.of( + ops.io(), currentSnapshotId, previousSnapshotId, currentSnapshotId, null, null, null, ImmutableList.of( new GenericManifestFile(localInput("file:/tmp/manfiest.2.avro"), SPEC_5.specId()))); List reversedSnapshotLog = Lists.newArrayList(); @@ -351,11 +358,11 @@ public void testAddPreviousMetadataRemoveNone() { public void testAddPreviousMetadataRemoveOne() { long previousSnapshotId = System.currentTimeMillis() - new Random(1234).nextInt(3600); Snapshot previousSnapshot = new BaseSnapshot( - ops.io(), previousSnapshotId, null, previousSnapshotId, null, null, ImmutableList.of( + ops.io(), previousSnapshotId, null, previousSnapshotId, null, null, null, ImmutableList.of( new GenericManifestFile(localInput("file:/tmp/manfiest.1.avro"), SPEC_5.specId()))); long currentSnapshotId = System.currentTimeMillis(); Snapshot currentSnapshot = new BaseSnapshot( - ops.io(), currentSnapshotId, previousSnapshotId, currentSnapshotId, null, null, ImmutableList.of( + ops.io(), currentSnapshotId, previousSnapshotId, currentSnapshotId, null, null, null, ImmutableList.of( new GenericManifestFile(localInput("file:/tmp/manfiest.2.avro"), SPEC_5.specId()))); List reversedSnapshotLog = Lists.newArrayList(); @@ -403,11 +410,11 @@ public void testAddPreviousMetadataRemoveOne() { public void testAddPreviousMetadataRemoveMultiple() { long previousSnapshotId = System.currentTimeMillis() - new Random(1234).nextInt(3600); Snapshot previousSnapshot = new BaseSnapshot( - ops.io(), previousSnapshotId, null, previousSnapshotId, null, null, ImmutableList.of( + ops.io(), previousSnapshotId, null, previousSnapshotId, null, null, null, ImmutableList.of( new GenericManifestFile(localInput("file:/tmp/manfiest.1.avro"), SPEC_5.specId()))); long currentSnapshotId = System.currentTimeMillis(); Snapshot currentSnapshot = new BaseSnapshot( - ops.io(), currentSnapshotId, previousSnapshotId, currentSnapshotId, null, null, ImmutableList.of( + ops.io(), currentSnapshotId, previousSnapshotId, currentSnapshotId, null, null, null, ImmutableList.of( new GenericManifestFile(localInput("file:/tmp/manfiest.2.avro"), SPEC_5.specId()))); List reversedSnapshotLog = Lists.newArrayList(); @@ -707,7 +714,7 @@ public void testUpdateSchema() { Types.NestedField.required(2, "x", Types.StringType.get()) ); TableMetadata sameSchemaTable = twoSchemasTable.updateSchema(sameSchema2, 2); - Assert.assertEquals("Should return same table metadata", + Assert.assertSame("Should return same table metadata", twoSchemasTable, sameSchemaTable); // update schema with the the same schema and different last column ID as current should create a new table @@ -738,7 +745,7 @@ public void testUpdateSchema() { Types.NestedField.required(4, "x", Types.StringType.get()), Types.NestedField.required(6, "z", Types.IntegerType.get()) ); - TableMetadata threeSchemaTable = revertSchemaTable.updateSchema(schema3, 3); + TableMetadata threeSchemaTable = revertSchemaTable.updateSchema(schema3, 6); Assert.assertEquals("Should have current schema id as 2", 2, threeSchemaTable.currentSchemaId()); assertSameSchemaList(ImmutableList.of(schema, @@ -747,6 +754,6 @@ public void testUpdateSchema() { Assert.assertEquals("Should have expected schema upon return", schema3.asStruct(), threeSchemaTable.schema().asStruct()); Assert.assertEquals("Should return expected last column id", - 3, threeSchemaTable.lastColumnId()); + 6, threeSchemaTable.lastColumnId()); } } diff --git a/core/src/test/java/org/apache/iceberg/TestUpdatePartitionSpec.java b/core/src/test/java/org/apache/iceberg/TestUpdatePartitionSpec.java index 1c0fc76534a2..c65dd2e3343f 100644 --- a/core/src/test/java/org/apache/iceberg/TestUpdatePartitionSpec.java +++ b/core/src/test/java/org/apache/iceberg/TestUpdatePartitionSpec.java @@ -569,6 +569,51 @@ public void testRenameAndDelete() { .renameField("shard", "id_bucket")); } + @Test + public void testRemoveAndAddMultiTimes() { + PartitionSpec addFirstTime = new BaseUpdatePartitionSpec(formatVersion, UNPARTITIONED) + .addField("ts_date", day("ts")) + .apply(); + PartitionSpec removeFirstTime = new BaseUpdatePartitionSpec(formatVersion, addFirstTime) + .removeField(day("ts")) + .apply(); + PartitionSpec addSecondTime = new BaseUpdatePartitionSpec(formatVersion, removeFirstTime) + .addField("ts_date", day("ts")) + .apply(); + PartitionSpec removeSecondTime = new BaseUpdatePartitionSpec(formatVersion, addSecondTime) + .removeField(day("ts")) + .apply(); + PartitionSpec addThirdTime = new BaseUpdatePartitionSpec(formatVersion, removeSecondTime) + .addField(month("ts")) + .apply(); + PartitionSpec updated = new BaseUpdatePartitionSpec(formatVersion, addThirdTime) + .renameField("ts_month", "ts_date") + .apply(); + + if (formatVersion == 1) { + Assert.assertEquals("Should match expected spec field size", 3, updated.fields().size()); + + Assert.assertTrue("Should match expected field name", + updated.fields().get(0).name().matches("^ts_date(?:_\\d+)+$")); + Assert.assertTrue("Should match expected field name", + updated.fields().get(1).name().matches("^ts_date_(?:\\d+)+$")); + Assert.assertEquals("Should match expected field name", "ts_date", updated.fields().get(2).name()); + + Assert.assertEquals("Should match expected field transform", "void", + updated.fields().get(0).transform().toString()); + Assert.assertEquals("Should match expected field transform", "void", + updated.fields().get(1).transform().toString()); + Assert.assertEquals("Should match expected field transform", "month", + updated.fields().get(2).transform().toString()); + } + + PartitionSpec v2Expected = PartitionSpec.builderFor(SCHEMA) + .month("ts", "ts_date") + .build(); + + V2Assert.assertEquals("Should match expected spec", v2Expected, updated); + } + private static int id(String name) { return SCHEMA.findField(name).fieldId(); } diff --git a/core/src/test/java/org/apache/iceberg/actions/TestBinPackStrategy.java b/core/src/test/java/org/apache/iceberg/actions/TestBinPackStrategy.java index b42496fc78b7..69cbe8f4652b 100644 --- a/core/src/test/java/org/apache/iceberg/actions/TestBinPackStrategy.java +++ b/core/src/test/java/org/apache/iceberg/actions/TestBinPackStrategy.java @@ -22,6 +22,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Set; import java.util.stream.Collectors; import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.DataFile; @@ -59,7 +60,7 @@ public Table table() { } @Override - public List rewriteFiles(List filesToRewrite) { + public Set rewriteFiles(List filesToRewrite) { throw new UnsupportedOperationException(); } } @@ -163,6 +164,24 @@ public void testMaxGroupSize() { 2, Iterables.size(grouped)); } + @Test + public void testNumOuputFiles() { + BinPackStrategy strategy = (BinPackStrategy) defaultBinPack(); + long targetFileSize = strategy.targetFileSize(); + Assert.assertEquals("Should keep remainder if the remainder is a valid size", + 2, strategy.numOutputFiles(targetFileSize + 450 * MB)); + Assert.assertEquals("Should discard remainder file if the remainder is very small", + 1, strategy.numOutputFiles(targetFileSize + 40 * MB)); + Assert.assertEquals("Should keep remainder file if it would change average file size greatly", + 2, strategy.numOutputFiles((long) (targetFileSize + 0.40 * targetFileSize))); + Assert.assertEquals("Should discard remainder if file is small and wouldn't change average that much", + 200, strategy.numOutputFiles(200 * targetFileSize + 13 * MB)); + Assert.assertEquals("Should keep remainder if it's a valid size", + 201, strategy.numOutputFiles(200 * targetFileSize + 499 * MB)); + Assert.assertEquals("Should not return 0 even for very small files", + 1, strategy.numOutputFiles(1)); + } + @Test public void testInvalidOptions() { AssertHelpers.assertThrows("Should not allow max size smaller than target", @@ -188,6 +207,5 @@ public void testInvalidOptions() { defaultBinPack().options(ImmutableMap.of( RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Long.toString(-5))); }); - } } diff --git a/core/src/test/java/org/apache/iceberg/avro/TestReadProjection.java b/core/src/test/java/org/apache/iceberg/avro/TestReadProjection.java index 9243a809ff1e..e71034483bdc 100644 --- a/core/src/test/java/org/apache/iceberg/avro/TestReadProjection.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestReadProjection.java @@ -23,6 +23,7 @@ import java.util.List; import java.util.Map; import org.apache.avro.generic.GenericData.Record; +import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Schema; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -148,7 +149,7 @@ public void testBasicProjection() throws Exception { ); Record projected = writeAndRead("basic_projection_id", writeSchema, idOnly, record); - Assert.assertNull("Should not project data", projected.get("data")); + AssertHelpers.assertEmptyAvroField(projected, "data"); Assert.assertEquals("Should contain the correct id value", 34L, (long) projected.get("id")); Schema dataOnly = new Schema( @@ -157,10 +158,10 @@ public void testBasicProjection() throws Exception { projected = writeAndRead("basic_projection_data", writeSchema, dataOnly, record); - Assert.assertNull("Should not project id", projected.get("id")); + AssertHelpers.assertEmptyAvroField(projected, "id"); int cmp = Comparators.charSequences() .compare("test", (CharSequence) projected.get("data")); - Assert.assertTrue("Should contain the correct data value", cmp == 0); + Assert.assertEquals("Should contain the correct data value", 0, cmp); } @Test @@ -184,7 +185,7 @@ public void testRename() throws Exception { Assert.assertEquals("Should contain the correct id value", 34L, (long) projected.get("id")); int cmp = Comparators.charSequences() .compare("test", (CharSequence) projected.get("renamed")); - Assert.assertTrue("Should contain the correct data/renamed value", cmp == 0); + Assert.assertEquals("Should contain the correct data/renamed value", 0, cmp); } @Test @@ -210,9 +211,8 @@ public void testNestedStructProjection() throws Exception { ); Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - Record projectedLocation = (Record) projected.get("location"); + AssertHelpers.assertEmptyAvroField(projected, "location"); Assert.assertEquals("Should contain the correct id value", 34L, (long) projected.get("id")); - Assert.assertNull("Should not project location", projectedLocation); Schema latOnly = new Schema( Types.NestedField.optional(3, "location", Types.StructType.of( @@ -221,10 +221,10 @@ public void testNestedStructProjection() throws Exception { ); projected = writeAndRead("latitude_only", writeSchema, latOnly, record); - projectedLocation = (Record) projected.get("location"); - Assert.assertNull("Should not project id", projected.get("id")); + Record projectedLocation = (Record) projected.get("location"); + AssertHelpers.assertEmptyAvroField(projected, "id"); Assert.assertNotNull("Should project location", projected.get("location")); - Assert.assertNull("Should not project longitude", projectedLocation.get("long")); + AssertHelpers.assertEmptyAvroField(projectedLocation, "long"); Assert.assertEquals("Should project latitude", 52.995143f, (float) projectedLocation.get("lat"), 0.000001f); @@ -236,16 +236,16 @@ public void testNestedStructProjection() throws Exception { projected = writeAndRead("longitude_only", writeSchema, longOnly, record); projectedLocation = (Record) projected.get("location"); - Assert.assertNull("Should not project id", projected.get("id")); + AssertHelpers.assertEmptyAvroField(projected, "id"); Assert.assertNotNull("Should project location", projected.get("location")); - Assert.assertNull("Should not project latitutde", projectedLocation.get("lat")); + AssertHelpers.assertEmptyAvroField(projectedLocation, "lat"); Assert.assertEquals("Should project longitude", -1.539054f, (float) projectedLocation.get("long"), 0.000001f); Schema locationOnly = writeSchema.select("location"); projected = writeAndRead("location_only", writeSchema, locationOnly, record); projectedLocation = (Record) projected.get("location"); - Assert.assertNull("Should not project id", projected.get("id")); + AssertHelpers.assertEmptyAvroField(projected, "id"); Assert.assertNotNull("Should project location", projected.get("location")); Assert.assertEquals("Should project latitude", 52.995143f, (float) projectedLocation.get("lat"), 0.000001f); @@ -273,23 +273,23 @@ public void testMapProjection() throws IOException { Record projected = writeAndRead("id_only", writeSchema, idOnly, record); Assert.assertEquals("Should contain the correct id value", 34L, (long) projected.get("id")); - Assert.assertNull("Should not project properties map", projected.get("properties")); + AssertHelpers.assertEmptyAvroField(projected, "properties"); Schema keyOnly = writeSchema.select("properties.key"); projected = writeAndRead("key_only", writeSchema, keyOnly, record); - Assert.assertNull("Should not project id", projected.get("id")); + AssertHelpers.assertEmptyAvroField(projected, "id"); Assert.assertEquals("Should project entire map", properties, toStringMap((Map) projected.get("properties"))); Schema valueOnly = writeSchema.select("properties.value"); projected = writeAndRead("value_only", writeSchema, valueOnly, record); - Assert.assertNull("Should not project id", projected.get("id")); + AssertHelpers.assertEmptyAvroField(projected, "id"); Assert.assertEquals("Should project entire map", properties, toStringMap((Map) projected.get("properties"))); Schema mapOnly = writeSchema.select("properties"); projected = writeAndRead("map_only", writeSchema, mapOnly, record); - Assert.assertNull("Should not project id", projected.get("id")); + AssertHelpers.assertEmptyAvroField(projected, "id"); Assert.assertEquals("Should project entire map", properties, toStringMap((Map) projected.get("properties"))); } @@ -337,16 +337,16 @@ public void testMapOfStructsProjection() throws IOException { Record projected = writeAndRead("id_only", writeSchema, idOnly, record); Assert.assertEquals("Should contain the correct id value", 34L, (long) projected.get("id")); - Assert.assertNull("Should not project locations map", projected.get("locations")); + AssertHelpers.assertEmptyAvroField(projected, "locations"); projected = writeAndRead("all_locations", writeSchema, writeSchema.select("locations"), record); - Assert.assertNull("Should not project id", projected.get("id")); + AssertHelpers.assertEmptyAvroField(projected, "id"); Assert.assertEquals("Should project locations map", record.get("locations"), toStringMap((Map) projected.get("locations"))); projected = writeAndRead("lat_only", writeSchema, writeSchema.select("locations.lat"), record); - Assert.assertNull("Should not project id", projected.get("id")); + AssertHelpers.assertEmptyAvroField(projected, "id"); Map locations = toStringMap((Map) projected.get("locations")); Assert.assertNotNull("Should project locations map", locations); Assert.assertEquals("Should contain L1 and L2", @@ -355,28 +355,28 @@ public void testMapOfStructsProjection() throws IOException { Assert.assertNotNull("L1 should not be null", projectedL1); Assert.assertEquals("L1 should contain lat", 53.992811f, (float) projectedL1.get("lat"), 0.000001); - Assert.assertNull("L1 should not contain long", projectedL1.get("long")); + AssertHelpers.assertEmptyAvroField(projectedL1, "long"); Record projectedL2 = (Record) locations.get("L2"); Assert.assertNotNull("L2 should not be null", projectedL2); Assert.assertEquals("L2 should contain lat", 52.995143f, (float) projectedL2.get("lat"), 0.000001); - Assert.assertNull("L2 should not contain long", projectedL2.get("long")); + AssertHelpers.assertEmptyAvroField(projectedL2, "y"); projected = writeAndRead("long_only", writeSchema, writeSchema.select("locations.long"), record); - Assert.assertNull("Should not project id", projected.get("id")); + AssertHelpers.assertEmptyAvroField(projected, "id"); locations = toStringMap((Map) projected.get("locations")); Assert.assertNotNull("Should project locations map", locations); Assert.assertEquals("Should contain L1 and L2", Sets.newHashSet("L1", "L2"), locations.keySet()); projectedL1 = (Record) locations.get("L1"); Assert.assertNotNull("L1 should not be null", projectedL1); - Assert.assertNull("L1 should not contain lat", projectedL1.get("lat")); + AssertHelpers.assertEmptyAvroField(projectedL1, "lat"); Assert.assertEquals("L1 should contain long", -1.542616f, (float) projectedL1.get("long"), 0.000001); projectedL2 = (Record) locations.get("L2"); Assert.assertNotNull("L2 should not be null", projectedL2); - Assert.assertNull("L2 should not contain lat", projectedL2.get("lat")); + AssertHelpers.assertEmptyAvroField(projectedL2, "lat"); Assert.assertEquals("L2 should contain long", -1.539054f, (float) projectedL2.get("long"), 0.000001); @@ -390,7 +390,7 @@ public void testMapOfStructsProjection() throws IOException { ); projected = writeAndRead("latitude_renamed", writeSchema, latitiudeRenamed, record); - Assert.assertNull("Should not project id", projected.get("id")); + AssertHelpers.assertEmptyAvroField(projected, "id"); locations = toStringMap((Map) projected.get("locations")); Assert.assertNotNull("Should project locations map", locations); Assert.assertEquals("Should contain L1 and L2", @@ -399,14 +399,14 @@ public void testMapOfStructsProjection() throws IOException { Assert.assertNotNull("L1 should not be null", projectedL1); Assert.assertEquals("L1 should contain latitude", 53.992811f, (float) projectedL1.get("latitude"), 0.000001); - Assert.assertNull("L1 should not contain lat", projectedL1.get("lat")); - Assert.assertNull("L1 should not contain long", projectedL1.get("long")); + AssertHelpers.assertEmptyAvroField(projectedL1, "lat"); + AssertHelpers.assertEmptyAvroField(projectedL1, "long"); projectedL2 = (Record) locations.get("L2"); Assert.assertNotNull("L2 should not be null", projectedL2); Assert.assertEquals("L2 should contain latitude", 52.995143f, (float) projectedL2.get("latitude"), 0.000001); - Assert.assertNull("L2 should not contain lat", projectedL2.get("lat")); - Assert.assertNull("L2 should not contain long", projectedL2.get("long")); + AssertHelpers.assertEmptyAvroField(projectedL2, "lat"); + AssertHelpers.assertEmptyAvroField(projectedL2, "long"); } @Test @@ -429,16 +429,16 @@ public void testListProjection() throws IOException { Record projected = writeAndRead("id_only", writeSchema, idOnly, record); Assert.assertEquals("Should contain the correct id value", 34L, (long) projected.get("id")); - Assert.assertNull("Should not project values list", projected.get("values")); + AssertHelpers.assertEmptyAvroField(projected, "values"); Schema elementOnly = writeSchema.select("values.element"); projected = writeAndRead("element_only", writeSchema, elementOnly, record); - Assert.assertNull("Should not project id", projected.get("id")); + AssertHelpers.assertEmptyAvroField(projected, "id"); Assert.assertEquals("Should project entire list", values, projected.get("values")); Schema listOnly = writeSchema.select("values"); projected = writeAndRead("list_only", writeSchema, listOnly, record); - Assert.assertNull("Should not project id", projected.get("id")); + AssertHelpers.assertEmptyAvroField(projected, "id"); Assert.assertEquals("Should project entire list", values, projected.get("values")); } @@ -473,35 +473,35 @@ public void testListOfStructsProjection() throws IOException { Record projected = writeAndRead("id_only", writeSchema, idOnly, record); Assert.assertEquals("Should contain the correct id value", 34L, (long) projected.get("id")); - Assert.assertNull("Should not project points list", projected.get("points")); + AssertHelpers.assertEmptyAvroField(projected, "points"); projected = writeAndRead("all_points", writeSchema, writeSchema.select("points"), record); - Assert.assertNull("Should not project id", projected.get("id")); + AssertHelpers.assertEmptyAvroField(projected, "id"); Assert.assertEquals("Should project points list", record.get("points"), projected.get("points")); projected = writeAndRead("x_only", writeSchema, writeSchema.select("points.x"), record); - Assert.assertNull("Should not project id", projected.get("id")); + AssertHelpers.assertEmptyAvroField(projected, "id"); Assert.assertNotNull("Should project points list", projected.get("points")); List points = (List) projected.get("points"); Assert.assertEquals("Should read 2 points", 2, points.size()); Record projectedP1 = points.get(0); Assert.assertEquals("Should project x", 1, (int) projectedP1.get("x")); - Assert.assertNull("Should not project y", projectedP1.get("y")); + AssertHelpers.assertEmptyAvroField(projectedP1, "y"); Record projectedP2 = points.get(1); Assert.assertEquals("Should project x", 3, (int) projectedP2.get("x")); - Assert.assertNull("Should not project y", projectedP2.get("y")); + AssertHelpers.assertEmptyAvroField(projectedP2, "y"); projected = writeAndRead("y_only", writeSchema, writeSchema.select("points.y"), record); - Assert.assertNull("Should not project id", projected.get("id")); + AssertHelpers.assertEmptyAvroField(projected, "id"); Assert.assertNotNull("Should project points list", projected.get("points")); points = (List) projected.get("points"); Assert.assertEquals("Should read 2 points", 2, points.size()); projectedP1 = points.get(0); - Assert.assertNull("Should not project x", projectedP1.get("x")); + AssertHelpers.assertEmptyAvroField(projectedP1, "x"); Assert.assertEquals("Should project y", 2, (int) projectedP1.get("y")); projectedP2 = points.get(1); - Assert.assertNull("Should not project x", projectedP2.get("x")); + AssertHelpers.assertEmptyAvroField(projectedP2, "x"); Assert.assertEquals("Should project null y", null, projectedP2.get("y")); Schema yRenamed = new Schema( @@ -513,17 +513,17 @@ public void testListOfStructsProjection() throws IOException { ); projected = writeAndRead("y_renamed", writeSchema, yRenamed, record); - Assert.assertNull("Should not project id", projected.get("id")); + AssertHelpers.assertEmptyAvroField(projected, "id"); Assert.assertNotNull("Should project points list", projected.get("points")); points = (List) projected.get("points"); Assert.assertEquals("Should read 2 points", 2, points.size()); projectedP1 = points.get(0); - Assert.assertNull("Should not project x", projectedP1.get("x")); - Assert.assertNull("Should not project y", projectedP1.get("y")); + AssertHelpers.assertEmptyAvroField(projectedP1, "x"); + AssertHelpers.assertEmptyAvroField(projectedP1, "y"); Assert.assertEquals("Should project z", 2, (int) projectedP1.get("z")); projectedP2 = points.get(1); - Assert.assertNull("Should not project x", projectedP2.get("x")); - Assert.assertNull("Should not project y", projectedP2.get("y")); - Assert.assertEquals("Should project null z", null, projectedP2.get("z")); + AssertHelpers.assertEmptyAvroField(projectedP2, "x"); + AssertHelpers.assertEmptyAvroField(projectedP2, "y"); + Assert.assertNull("Should project null z", projectedP2.get("z")); } } diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestStaticTable.java b/core/src/test/java/org/apache/iceberg/hadoop/TestStaticTable.java index 16a5ddd582fe..4569951458f3 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestStaticTable.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestStaticTable.java @@ -60,6 +60,19 @@ public void testCannotBeDeletedFrom() { () -> staticTable.newDelete().deleteFile(FILE_A).commit()); } + @Test + public void testCannotDoIncrementalScanOnMetadataTable() { + table.newAppend().appendFile(FILE_A).commit(); + + for (MetadataTableType type : MetadataTableType.values()) { + Table staticTable = getStaticTable(type); + AssertHelpers.assertThrows("Static tables do not support incremental scans", + UnsupportedOperationException.class, + String.format("Cannot incrementally scan table of type %s", type), + () -> staticTable.newScan().appendsAfter(1)); + } + } + @Test public void testHasSameProperties() { table.newAppend().appendFile(FILE_A).commit(); diff --git a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java new file mode 100644 index 000000000000..5292e3146ec7 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java @@ -0,0 +1,559 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.jdbc; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.AssertHelpers; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.Table; +import org.apache.iceberg.Transaction; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.NamespaceNotEmptyException; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.hadoop.Util; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.transforms.Transform; +import org.apache.iceberg.transforms.Transforms; +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 static org.apache.iceberg.NullOrder.NULLS_FIRST; +import static org.apache.iceberg.SortDirection.ASC; +import static org.apache.iceberg.types.Types.NestedField.required; + +public class TestJdbcCatalog { + + static final Schema SCHEMA = new Schema( + required(1, "id", Types.IntegerType.get(), "unique ID"), + required(2, "data", Types.StringType.get()) + ); + static final PartitionSpec PARTITION_SPEC = PartitionSpec.builderFor(SCHEMA) + .bucket("data", 16) + .build(); + + static Configuration conf = new Configuration(); + private static JdbcCatalog catalog; + private static String warehouseLocation; + @Rule + public TemporaryFolder temp = new TemporaryFolder(); + File tableDir = null; + + protected List metadataVersionFiles(String location) { + return Stream.of(new File(location).listFiles()) + .filter(file -> !file.isDirectory()) + .map(File::getName) + .filter(fileName -> fileName.endsWith("metadata.json")) + .collect(Collectors.toList()) + ; + } + + protected List manifestFiles(String location) { + return Stream.of(new File(location).listFiles()) + .filter(file -> !file.isDirectory()) + .map(File::getName) + .filter(fileName -> fileName.endsWith(".avro")) + .collect(Collectors.toList()) + ; + } + + @Before + public void setupTable() throws Exception { + this.tableDir = temp.newFolder(); + tableDir.delete(); // created by table create + Map properties = new HashMap<>(); + properties.put(CatalogProperties.URI, + "jdbc:sqlite:file::memory:?ic" + UUID.randomUUID().toString().replace("-", "")); + + properties.put(JdbcCatalog.PROPERTY_PREFIX + "username", "user"); + properties.put(JdbcCatalog.PROPERTY_PREFIX + "password", "password"); + warehouseLocation = this.tableDir.getAbsolutePath(); + properties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation); + catalog = new JdbcCatalog(); + catalog.setConf(conf); + catalog.initialize("test_jdbc_catalog", properties); + } + + @Test + public void testInitialize() { + Map properties = new HashMap<>(); + properties.put(CatalogProperties.WAREHOUSE_LOCATION, this.tableDir.getAbsolutePath()); + properties.put(CatalogProperties.URI, "jdbc:sqlite:file::memory:?icebergDB"); + JdbcCatalog jdbcCatalog = new JdbcCatalog(); + jdbcCatalog.setConf(conf); + jdbcCatalog.initialize("test_jdbc_catalog", properties); + // second initialization should not fail even if tables are already created + jdbcCatalog.initialize("test_jdbc_catalog", properties); + jdbcCatalog.initialize("test_jdbc_catalog", properties); + } + + @Test + public void testCreateTableBuilder() { + TableIdentifier tableIdent = TableIdentifier.of("db", "ns1", "ns2", "tbl"); + Table table = catalog.buildTable(tableIdent, SCHEMA) + .withPartitionSpec(PARTITION_SPEC) + .withProperties(null) + .withProperty("key1", "value1") + .withProperties(ImmutableMap.of("key2", "value2")) + .create(); + + Assert.assertEquals(SCHEMA.toString(), table.schema().toString()); + Assert.assertEquals(1, table.spec().fields().size()); + Assert.assertEquals("value1", table.properties().get("key1")); + Assert.assertEquals("value2", table.properties().get("key2")); + } + + @Test + public void testCreateTableTxnBuilder() { + TableIdentifier tableIdent = TableIdentifier.of("db", "ns1", "ns2", "tbl"); + Transaction txn = catalog.buildTable(tableIdent, SCHEMA) + .withPartitionSpec(null) + .withProperty("key1", "testval1") + .createTransaction(); + txn.commitTransaction(); + Table table = catalog.loadTable(tableIdent); + + Assert.assertEquals(SCHEMA.toString(), table.schema().toString()); + Assert.assertTrue(table.spec().isUnpartitioned()); + Assert.assertEquals("testval1", table.properties().get("key1")); + } + + @Test + public void testReplaceTxnBuilder() { + TableIdentifier tableIdent = TableIdentifier.of("db", "ns1", "ns2", "tbl"); + + final DataFile fileA = DataFiles.builder(PARTITION_SPEC) + .withPath("/path/to/data-a.parquet") + .withFileSizeInBytes(0) + .withPartitionPath("data_bucket=0") // easy way to set partition data for now + .withRecordCount(2) // needs at least one record or else metrics will filter it out + .build(); + + Transaction createTxn = catalog.buildTable(tableIdent, SCHEMA) + .withPartitionSpec(PARTITION_SPEC) + .withProperty("key1", "value1") + .createOrReplaceTransaction(); + + createTxn.newAppend() + .appendFile(fileA) + .commit(); + + createTxn.commitTransaction(); + + Table table = catalog.loadTable(tableIdent); + Assert.assertNotNull(table.currentSnapshot()); + + Transaction replaceTxn = catalog.buildTable(tableIdent, SCHEMA) + .withProperty("key2", "value2") + .replaceTransaction(); + replaceTxn.commitTransaction(); + + table = catalog.loadTable(tableIdent); + Assert.assertNull(table.currentSnapshot()); + Assert.assertTrue(table.spec().isUnpartitioned()); + Assert.assertEquals("value1", table.properties().get("key1")); + Assert.assertEquals("value2", table.properties().get("key2")); + } + + @Test + public void testCreateTableDefaultSortOrder() { + TableIdentifier tableIdent = TableIdentifier.of("db", "ns1", "ns2", "tbl"); + Table table = catalog.createTable(tableIdent, SCHEMA, PARTITION_SPEC); + + SortOrder sortOrder = table.sortOrder(); + Assert.assertEquals("Order ID must match", 0, sortOrder.orderId()); + Assert.assertTrue("Order must unsorted", sortOrder.isUnsorted()); + } + + @Test + public void testCreateTableCustomSortOrder() { + TableIdentifier tableIdent = TableIdentifier.of("db", "ns1", "ns2", "tbl"); + SortOrder order = SortOrder.builderFor(SCHEMA) + .asc("id", NULLS_FIRST) + .build(); + Table table = catalog.buildTable(tableIdent, SCHEMA) + .withPartitionSpec(PARTITION_SPEC) + .withSortOrder(order) + .create(); + + SortOrder sortOrder = table.sortOrder(); + Assert.assertEquals("Order ID must match", 1, sortOrder.orderId()); + Assert.assertEquals("Order must have 1 field", 1, sortOrder.fields().size()); + Assert.assertEquals("Direction must match ", ASC, sortOrder.fields().get(0).direction()); + Assert.assertEquals("Null order must match ", NULLS_FIRST, sortOrder.fields().get(0).nullOrder()); + Transform transform = Transforms.identity(Types.IntegerType.get()); + Assert.assertEquals("Transform must match", transform, sortOrder.fields().get(0).transform()); + } + + @Test + public void testBasicCatalog() throws Exception { + TableIdentifier testTable = TableIdentifier.of("db", "ns1", "ns2", "tbl"); + catalog.createTable(testTable, SCHEMA, PartitionSpec.unpartitioned()); + String metaLocation = catalog.defaultWarehouseLocation(testTable); + + FileSystem fs = Util.getFs(new Path(metaLocation), conf); + Assert.assertTrue(fs.isDirectory(new Path(metaLocation))); + + AssertHelpers.assertThrows("should throw exception", AlreadyExistsException.class, + "already exists", () -> + catalog.createTable(testTable, SCHEMA, PartitionSpec.unpartitioned()) + ); + + catalog.dropTable(testTable); + } + + @Test + public void testCreateAndDropTableWithoutNamespace() throws Exception { + TableIdentifier testTable = TableIdentifier.of("tbl"); + Table table = catalog.createTable(testTable, SCHEMA, PartitionSpec.unpartitioned()); + + Assert.assertEquals(table.schema().toString(), SCHEMA.toString()); + Assert.assertEquals(catalog.name() + ".tbl", table.name()); + String metaLocation = catalog.defaultWarehouseLocation(testTable); + + FileSystem fs = Util.getFs(new Path(metaLocation), conf); + Assert.assertTrue(fs.isDirectory(new Path(metaLocation))); + + catalog.dropTable(testTable, true); + } + + @Test + public void testDefaultWarehouseLocation() throws Exception { + TableIdentifier testTable = TableIdentifier.of("tbl"); + TableIdentifier testTable2 = TableIdentifier.of(Namespace.of("ns"), "tbl"); + Assert.assertEquals(catalog.defaultWarehouseLocation(testTable), + warehouseLocation + "/" + testTable.name()); + Assert.assertEquals(catalog.defaultWarehouseLocation(testTable2), + warehouseLocation + "/" + testTable2.namespace() + "/" + testTable2.name()); + } + + @Test + public void testConcurrentCommit() throws IOException { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "table"); + Table table = catalog.createTable(tableIdentifier, SCHEMA, PartitionSpec.unpartitioned()); + // append file and commit! + String data = temp.newFile("data.parquet").getPath(); + Files.write(Paths.get(data), new ArrayList<>(), StandardCharsets.UTF_8); + DataFile dataFile = DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(data) + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + table.newAppend().appendFile(dataFile).commit(); + Assert.assertEquals(1, table.history().size()); + catalog.dropTable(tableIdentifier); + data = temp.newFile("data2.parquet").getPath(); + Files.write(Paths.get(data), new ArrayList<>(), StandardCharsets.UTF_8); + DataFile dataFile2 = DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(data) + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + + AssertHelpers.assertThrows("Should fail", NoSuchTableException.class, + "Failed to load table", () -> table.newAppend().appendFile(dataFile2).commit() + ); + } + + @Test + public void testCommitHistory() throws IOException { + TableIdentifier testTable = TableIdentifier.of("db", "ns", "tbl"); + catalog.createTable(testTable, SCHEMA, PartitionSpec.unpartitioned()); + Table table = catalog.loadTable(testTable); + + String data = temp.newFile("data.parquet").getPath(); + Files.write(Paths.get(data), new ArrayList<>(), StandardCharsets.UTF_8); + DataFile dataFile = DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(data) + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + table.newAppend().appendFile(dataFile).commit(); + Assert.assertEquals(1, table.history().size()); + + data = temp.newFile("data2.parquet").getPath(); + Files.write(Paths.get(data), new ArrayList<>(), StandardCharsets.UTF_8); + dataFile = DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(data) + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + table.newAppend().appendFile(dataFile).commit(); + Assert.assertEquals(2, table.history().size()); + + data = temp.newFile("data3.parquet").getPath(); + Files.write(Paths.get(data), new ArrayList<>(), StandardCharsets.UTF_8); + dataFile = DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(data) + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + table.newAppend().appendFile(dataFile).commit(); + Assert.assertEquals(3, table.history().size()); + } + + @Test + public void testDropTable() { + TableIdentifier testTable = TableIdentifier.of("db", "ns1", "ns2", "tbl"); + TableIdentifier testTable2 = TableIdentifier.of("db", "ns1", "ns2", "tbl2"); + catalog.createTable(testTable, SCHEMA, PartitionSpec.unpartitioned()); + catalog.createTable(testTable2, SCHEMA, PartitionSpec.unpartitioned()); + catalog.dropTable(testTable); + Assert.assertFalse(catalog.listTables(testTable.namespace()).contains(testTable)); + catalog.dropTable(testTable2); + AssertHelpers.assertThrows("should throw exception", NoSuchNamespaceException.class, + "not exist", () -> catalog.listTables(testTable2.namespace()) + ); + + Assert.assertFalse(catalog.dropTable(TableIdentifier.of("db", "tbl-not-exists"))); + } + + @Test + public void testRenameTable() { + TableIdentifier from = TableIdentifier.of("db", "tbl1"); + TableIdentifier to = TableIdentifier.of("db", "tbl2-newtable"); + catalog.createTable(from, SCHEMA, PartitionSpec.unpartitioned()); + catalog.renameTable(from, to); + Assert.assertTrue(catalog.listTables(to.namespace()).contains(to)); + Assert.assertFalse(catalog.listTables(to.namespace()).contains(from)); + Assert.assertTrue(catalog.loadTable(to).name().endsWith(to.name())); + + AssertHelpers.assertThrows("should throw exception", NoSuchTableException.class, + "Table does not exist", () -> + catalog.renameTable(TableIdentifier.of("db", "tbl-not-exists"), to) + ); + + // rename table to existing table name! + TableIdentifier from2 = TableIdentifier.of("db", "tbl2"); + catalog.createTable(from2, SCHEMA, PartitionSpec.unpartitioned()); + AssertHelpers.assertThrows("should throw exception", UncheckedSQLException.class, + "Failed to rename db.tbl2 to db.tbl2-newtable", () -> catalog.renameTable(from2, to) + ); + } + + @Test + public void testListTables() { + TableIdentifier tbl1 = TableIdentifier.of("db", "tbl1"); + TableIdentifier tbl2 = TableIdentifier.of("db", "tbl2"); + TableIdentifier tbl3 = TableIdentifier.of("db", "tbl2", "subtbl2"); + TableIdentifier tbl4 = TableIdentifier.of("db", "ns1", "tbl3"); + TableIdentifier tbl5 = TableIdentifier.of("db", "metadata", "metadata"); + + Lists.newArrayList(tbl1, tbl2, tbl3, tbl4, tbl5).forEach(t -> + catalog.createTable(t, SCHEMA, PartitionSpec.unpartitioned()) + ); + + List tbls1 = catalog.listTables(Namespace.of("db")); + Set tblSet = Sets.newHashSet(tbls1.stream().map(TableIdentifier::name).iterator()); + Assert.assertEquals(tblSet.size(), 2); + Assert.assertTrue(tblSet.contains("tbl1")); + Assert.assertTrue(tblSet.contains("tbl2")); + + List tbls2 = catalog.listTables(Namespace.of("db", "ns1")); + Assert.assertEquals(tbls2.size(), 1); + Assert.assertEquals("tbl3", tbls2.get(0).name()); + + AssertHelpers.assertThrows("should throw exception", NoSuchNamespaceException.class, + "does not exist", () -> catalog.listTables(Namespace.of("db", "ns1", "ns2"))); + } + + @Test + public void testCallingLocationProviderWhenNoCurrentMetadata() { + TableIdentifier tableIdent = TableIdentifier.of("ns1", "ns2", "table1"); + Transaction create = catalog.newCreateTableTransaction(tableIdent, SCHEMA); + create.table().locationProvider(); // NPE triggered if not handled appropriately + create.commitTransaction(); + + Assert.assertEquals("1 table expected", 1, catalog.listTables(Namespace.of("ns1", "ns2")).size()); + catalog.dropTable(tableIdent, true); + } + + @Test + public void testExistingTableUpdate() { + TableIdentifier tableIdent = TableIdentifier.of("ns1", "ns2", "table1"); + Transaction create = catalog.newCreateTableTransaction(tableIdent, SCHEMA); + create.table().locationProvider(); // NPE triggered if not handled appropriately + create.commitTransaction(); + Table icebergTable = catalog.loadTable(tableIdent); + // add a column + icebergTable.updateSchema().addColumn("Coll3", Types.LongType.get()).commit(); + icebergTable = catalog.loadTable(tableIdent); + // Only 2 snapshotFile Should exist and no manifests should exist + Assert.assertEquals(2, metadataVersionFiles(icebergTable.location() + "/metadata/").size()); + Assert.assertEquals(0, manifestFiles(icebergTable.location() + "/metadata/").size()); + Assert.assertNotEquals(SCHEMA.asStruct(), icebergTable.schema().asStruct()); + Assert.assertTrue(icebergTable.schema().asStruct().toString().contains("Coll3")); + } + + @Test + public void testTableName() { + TableIdentifier tableIdent = TableIdentifier.of("db", "ns1", "ns2", "tbl"); + catalog.buildTable(tableIdent, SCHEMA) + .withPartitionSpec(PARTITION_SPEC) + .create(); + Table table = catalog.loadTable(tableIdent); + Assert.assertEquals("Name must match", catalog.name() + ".db.ns1.ns2.tbl", table.name()); + + TableIdentifier snapshotsTableIdent = TableIdentifier.of("db", "ns1", "ns2", "tbl", "snapshots"); + Table snapshotsTable = catalog.loadTable(snapshotsTableIdent); + Assert.assertEquals( + "Name must match", catalog.name() + ".db.ns1.ns2.tbl.snapshots", snapshotsTable.name()); + } + + @Test + public void testListNamespace() { + TableIdentifier tbl1 = TableIdentifier.of("db", "ns1", "ns2", "metadata"); + TableIdentifier tbl2 = TableIdentifier.of("db", "ns2", "ns3", "tbl2"); + TableIdentifier tbl3 = TableIdentifier.of("db", "ns3", "tbl4"); + TableIdentifier tbl4 = TableIdentifier.of("db", "metadata"); + TableIdentifier tbl5 = TableIdentifier.of("db2", "metadata"); + TableIdentifier tbl6 = TableIdentifier.of("tbl6"); + + Lists.newArrayList(tbl1, tbl2, tbl3, tbl4, tbl5, tbl6).forEach(t -> + catalog.createTable(t, SCHEMA, PartitionSpec.unpartitioned()) + ); + + List nsp1 = catalog.listNamespaces(Namespace.of("db")); + Assert.assertEquals(nsp1.size(), 3); + Set tblSet = Sets.newHashSet(nsp1.stream().map(Namespace::toString).iterator()); + Assert.assertEquals(tblSet.size(), 3); + Assert.assertTrue(tblSet.contains("db.ns1")); + Assert.assertTrue(tblSet.contains("db.ns2")); + Assert.assertTrue(tblSet.contains("db.ns3")); + + List nsp2 = catalog.listNamespaces(Namespace.of("db", "ns1")); + Assert.assertEquals(nsp2.size(), 1); + Assert.assertEquals("db.ns1.ns2", nsp2.get(0).toString()); + + List nsp3 = catalog.listNamespaces(); + Set tblSet2 = Sets.newHashSet(nsp3.stream().map(Namespace::toString).iterator()); + System.out.println(tblSet2.toString()); + Assert.assertEquals(tblSet2.size(), 3); + Assert.assertTrue(tblSet2.contains("db")); + Assert.assertTrue(tblSet2.contains("db2")); + Assert.assertTrue(tblSet2.contains("")); + + List nsp4 = catalog.listNamespaces(); + Set tblSet3 = Sets.newHashSet(nsp4.stream().map(Namespace::toString).iterator()); + Assert.assertEquals(tblSet3.size(), 3); + Assert.assertTrue(tblSet3.contains("db")); + Assert.assertTrue(tblSet3.contains("db2")); + Assert.assertTrue(tblSet3.contains("")); + + AssertHelpers.assertThrows("Should fail to list namespace doesn't exist", NoSuchNamespaceException.class, + "Namespace does not exist", () -> catalog.listNamespaces(Namespace.of("db", "db2", "ns2") + )); + } + + @Test + public void testLoadNamespaceMeta() { + TableIdentifier tbl1 = TableIdentifier.of("db", "ns1", "ns2", "metadata"); + TableIdentifier tbl2 = TableIdentifier.of("db", "ns2", "ns3", "tbl2"); + TableIdentifier tbl3 = TableIdentifier.of("db", "ns3", "tbl4"); + TableIdentifier tbl4 = TableIdentifier.of("db", "metadata"); + + Lists.newArrayList(tbl1, tbl2, tbl3, tbl4).forEach(t -> + catalog.createTable(t, SCHEMA, PartitionSpec.unpartitioned()) + ); + + Assert.assertTrue(catalog.loadNamespaceMetadata(Namespace.of("db")).containsKey("location")); + + AssertHelpers.assertThrows("Should fail to load namespace doesn't exist", + NoSuchNamespaceException.class, "Namespace does not exist", () -> + catalog.loadNamespaceMetadata(Namespace.of("db", "db2", "ns2"))); + } + + @Test + public void testNamespaceExists() { + TableIdentifier tbl1 = TableIdentifier.of("db", "ns1", "ns2", "metadata"); + TableIdentifier tbl2 = TableIdentifier.of("db", "ns2", "ns3", "tbl2"); + TableIdentifier tbl3 = TableIdentifier.of("db", "ns3", "tbl4"); + TableIdentifier tbl4 = TableIdentifier.of("db", "metadata"); + + Lists.newArrayList(tbl1, tbl2, tbl3, tbl4).forEach(t -> + catalog.createTable(t, SCHEMA, PartitionSpec.unpartitioned()) + ); + Assert.assertTrue("Should true to namespace exist", + catalog.namespaceExists(Namespace.of("db", "ns1", "ns2"))); + Assert.assertFalse("Should false to namespace doesn't exist", + catalog.namespaceExists(Namespace.of("db", "db2", "not_exist"))); + } + + + @Test + public void testDropNamespace() { + + AssertHelpers.assertThrows("Should fail to drop namespace doesn't exist", NoSuchNamespaceException.class, + "Namespace does not exist", () -> catalog.dropNamespace(Namespace.of("db", "ns1_not_exitss"))); + + TableIdentifier tbl0 = TableIdentifier.of("db", "ns1", "ns2", "tbl2"); + TableIdentifier tbl1 = TableIdentifier.of("db", "ns1", "ns2", "tbl1"); + TableIdentifier tbl2 = TableIdentifier.of("db", "ns1", "tbl2"); + TableIdentifier tbl3 = TableIdentifier.of("db", "ns3", "tbl4"); + TableIdentifier tbl4 = TableIdentifier.of("db", "tbl"); + + Lists.newArrayList(tbl0, tbl1, tbl2, tbl3, tbl4).forEach(t -> + catalog.createTable(t, SCHEMA, PartitionSpec.unpartitioned()) + ); + + AssertHelpers.assertThrows("Should fail to drop namespace has tables", NamespaceNotEmptyException.class, + "is not empty. 2 tables exist.", () -> catalog.dropNamespace(tbl1.namespace())); + AssertHelpers.assertThrows("Should fail to drop namespace has tables", NamespaceNotEmptyException.class, + "is not empty. 1 tables exist.", () -> catalog.dropNamespace(tbl2.namespace())); + AssertHelpers.assertThrows("Should fail to drop namespace has tables", NamespaceNotEmptyException.class, + "is not empty. 1 tables exist.", () -> catalog.dropNamespace(tbl4.namespace())); + } + + @Test + public void testConversions() { + Namespace ns = Namespace.of("db", "db2", "ns2"); + String nsString = JdbcUtil.namespaceToString(ns); + Assert.assertEquals(ns, JdbcUtil.stringToNamespace(nsString)); + } + +} diff --git a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcTableConcurrency.java b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcTableConcurrency.java new file mode 100644 index 000000000000..9a8e34352731 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcTableConcurrency.java @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.jdbc; + +import java.io.File; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Tasks; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES; +import static org.apache.iceberg.types.Types.NestedField.required; + +public class TestJdbcTableConcurrency { + + static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of("db", "test_table"); + static final Schema SCHEMA = new Schema( + required(1, "id", Types.IntegerType.get(), "unique ID"), + required(2, "data", Types.StringType.get()) + ); + @Rule + public TemporaryFolder temp = new TemporaryFolder(); + File tableDir; + + @Test + public synchronized void testConcurrentFastAppends() throws IOException { + Map properties = new HashMap<>(); + this.tableDir = temp.newFolder(); + properties.put(CatalogProperties.WAREHOUSE_LOCATION, tableDir.getAbsolutePath()); + String sqliteDb = "jdbc:sqlite:" + tableDir.getAbsolutePath() + "concurentFastAppend.db"; + properties.put(CatalogProperties.URI, sqliteDb); + JdbcCatalog catalog = new JdbcCatalog(); + catalog.setConf(new Configuration()); + catalog.initialize("jdbc", properties); + catalog.createTable(TABLE_IDENTIFIER, SCHEMA); + + Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER); + + String fileName = UUID.randomUUID().toString(); + DataFile file = DataFiles.builder(icebergTable.spec()) + .withPath(FileFormat.PARQUET.addExtension(fileName)) + .withRecordCount(2) + .withFileSizeInBytes(0) + .build(); + + ExecutorService executorService = MoreExecutors.getExitingExecutorService( + (ThreadPoolExecutor) Executors.newFixedThreadPool(2)); + + AtomicInteger barrier = new AtomicInteger(0); + Tasks.range(2) + .stopOnFailure().throwFailureWhenFinished() + .executeWith(executorService) + .run(index -> { + for (int numCommittedFiles = 0; numCommittedFiles < 10; numCommittedFiles++) { + while (barrier.get() < numCommittedFiles * 2) { + try { + Thread.sleep(10); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + icebergTable.newFastAppend().appendFile(file).commit(); + barrier.incrementAndGet(); + } + }); + + icebergTable.refresh(); + Assert.assertEquals(20, icebergTable.currentSnapshot().allManifests().size()); + } + + @Test + public synchronized void testConcurrentConnections() throws InterruptedException, IOException { + Map properties = new HashMap<>(); + this.tableDir = temp.newFolder(); + properties.put(CatalogProperties.WAREHOUSE_LOCATION, tableDir.getAbsolutePath()); + String sqliteDb = "jdbc:sqlite:" + tableDir.getAbsolutePath() + "concurentConnections.db"; + properties.put(CatalogProperties.URI, sqliteDb); + JdbcCatalog catalog = new JdbcCatalog(); + catalog.setConf(new Configuration()); + catalog.initialize("jdbc", properties); + catalog.createTable(TABLE_IDENTIFIER, SCHEMA); + + Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER); + + icebergTable.updateProperties() + .set(COMMIT_NUM_RETRIES, "20") + .set(COMMIT_MIN_RETRY_WAIT_MS, "25") + .set(COMMIT_MAX_RETRY_WAIT_MS, "25") + .commit(); + + String fileName = UUID.randomUUID().toString(); + DataFile file = DataFiles.builder(icebergTable.spec()) + .withPath(FileFormat.PARQUET.addExtension(fileName)) + .withRecordCount(2) + .withFileSizeInBytes(0) + .build(); + + ExecutorService executorService = MoreExecutors.getExitingExecutorService( + (ThreadPoolExecutor) Executors.newFixedThreadPool(7)); + + for (int i = 0; i < 7; i++) { + executorService.submit(() -> icebergTable.newAppend().appendFile(file).commit()); + } + + executorService.shutdown(); + Assert.assertTrue("Timeout", executorService.awaitTermination(3, TimeUnit.MINUTES)); + Assert.assertEquals(7, Iterables.size(icebergTable.snapshots())); + } +} diff --git a/core/src/test/resources/TableMetadataV2Valid.json b/core/src/test/resources/TableMetadataV2Valid.json index d43e0a20619f..0dc89de58c42 100644 --- a/core/src/test/resources/TableMetadataV2Valid.json +++ b/core/src/test/resources/TableMetadataV2Valid.json @@ -85,8 +85,38 @@ } ], "properties": {}, - "current-snapshot-id": -1, - "snapshots": [], - "snapshot-log": [], + "current-snapshot-id": 3055729675574597004, + "snapshots": [ + { + "snapshot-id": 3051729675574597004, + "timestamp-ms": 1515100955770, + "sequence-number": 0, + "summary": { + "operation": "append" + }, + "manifest-list": "s3://a/b/1.avro" + }, + { + "snapshot-id": 3055729675574597004, + "parent-snapshot-id": 3051729675574597004, + "timestamp-ms": 1555100955770, + "sequence-number": 1, + "summary": { + "operation": "append" + }, + "manifest-list": "s3://a/b/2.avro", + "schema-id": 1 + } + ], + "snapshot-log": [ + { + "snapshot-id": 3051729675574597004, + "timestamp-ms": 1515100955770 + }, + { + "snapshot-id": 3055729675574597004, + "timestamp-ms": 1555100955770 + } + ], "metadata-log": [] -} +} \ No newline at end of file diff --git a/data/src/main/java/org/apache/iceberg/data/DeleteFilter.java b/data/src/main/java/org/apache/iceberg/data/DeleteFilter.java index 7a0752975e80..a8eb13cdfa68 100644 --- a/data/src/main/java/org/apache/iceberg/data/DeleteFilter.java +++ b/data/src/main/java/org/apache/iceberg/data/DeleteFilter.java @@ -243,6 +243,8 @@ private static Schema fileProjection(Schema tableSchema, Schema requestedSchema, requiredIds.addAll(eqDelete.equalityFieldIds()); } + requiredIds.add(MetadataColumns.IS_DELETED.fieldId()); + Set missingIds = Sets.newLinkedHashSet( Sets.difference(requiredIds, TypeUtil.getProjectedIds(requestedSchema))); @@ -253,8 +255,8 @@ private static Schema fileProjection(Schema tableSchema, Schema requestedSchema, // TODO: support adding nested columns. this will currently fail when finding nested columns to add List columns = Lists.newArrayList(requestedSchema.columns()); for (int fieldId : missingIds) { - if (fieldId == MetadataColumns.ROW_POSITION.fieldId()) { - continue; // add _pos at the end + if (fieldId == MetadataColumns.ROW_POSITION.fieldId() || fieldId == MetadataColumns.IS_DELETED.fieldId()) { + continue; // add _pos and _deleted at the end } Types.NestedField field = tableSchema.asStruct().field(fieldId); @@ -267,6 +269,10 @@ private static Schema fileProjection(Schema tableSchema, Schema requestedSchema, columns.add(MetadataColumns.ROW_POSITION); } + if (missingIds.contains(MetadataColumns.IS_DELETED.fieldId())) { + columns.add(MetadataColumns.IS_DELETED); + } + return new Schema(columns); } } diff --git a/data/src/main/java/org/apache/iceberg/data/InternalRecordWrapper.java b/data/src/main/java/org/apache/iceberg/data/InternalRecordWrapper.java index a2d06243d933..4aea96d261b7 100644 --- a/data/src/main/java/org/apache/iceberg/data/InternalRecordWrapper.java +++ b/data/src/main/java/org/apache/iceberg/data/InternalRecordWrapper.java @@ -81,7 +81,13 @@ public int size() { @Override public T get(int pos, Class javaClass) { if (transforms[pos] != null) { - return javaClass.cast(transforms[pos].apply(wrapped.get(pos, Object.class))); + Object value = wrapped.get(pos, Object.class); + if (value == null) { + // transforms function don't allow to handle null values, so just return null here. + return null; + } else { + return javaClass.cast(transforms[pos].apply(value)); + } } return wrapped.get(pos, javaClass); } diff --git a/data/src/main/java/org/apache/iceberg/data/orc/GenericOrcReaders.java b/data/src/main/java/org/apache/iceberg/data/orc/GenericOrcReaders.java index 1ad87f5a25ff..bf721ec91be1 100644 --- a/data/src/main/java/org/apache/iceberg/data/orc/GenericOrcReaders.java +++ b/data/src/main/java/org/apache/iceberg/data/orc/GenericOrcReaders.java @@ -50,8 +50,6 @@ public class GenericOrcReaders { - private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC); - private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate(); private GenericOrcReaders() { } diff --git a/data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java b/data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java index a04d777df1d6..29426bc97566 100644 --- a/data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java +++ b/data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java @@ -129,7 +129,7 @@ public void write(Record value, VectorizedRowBatch output) { } @Override - public Stream metrics() { + public Stream> metrics() { return writer.metrics(); } @@ -160,7 +160,7 @@ public void nonNullWrite(int rowId, Record data, ColumnVector output) { } @Override - public Stream metrics() { + public Stream> metrics() { return writers.stream().flatMap(OrcValueWriter::metrics); } } diff --git a/data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriters.java b/data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriters.java index 658942400896..7efa1613de97 100644 --- a/data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriters.java +++ b/data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriters.java @@ -33,6 +33,7 @@ import java.util.Map; import java.util.UUID; import java.util.stream.Stream; +import org.apache.iceberg.DoubleFieldMetrics; import org.apache.iceberg.FieldMetrics; import org.apache.iceberg.FloatFieldMetrics; import org.apache.iceberg.orc.OrcValueWriter; @@ -219,12 +220,11 @@ public void nonNullWrite(int rowId, Long data, ColumnVector output) { } private static class FloatWriter implements OrcValueWriter { - private final int id; - private long nanCount; + private final FloatFieldMetrics.Builder floatFieldMetricsBuilder; + private long nullValueCount = 0; private FloatWriter(int id) { - this.id = id; - this.nanCount = 0; + this.floatFieldMetricsBuilder = new FloatFieldMetrics.Builder(id); } @Override @@ -235,24 +235,30 @@ public Class getJavaClass() { @Override public void nonNullWrite(int rowId, Float data, ColumnVector output) { ((DoubleColumnVector) output).vector[rowId] = data; - if (Float.isNaN(data)) { - nanCount++; - } + floatFieldMetricsBuilder.addValue(data); + } + + @Override + public void nullWrite() { + nullValueCount++; } @Override - public Stream metrics() { - return Stream.of(new FloatFieldMetrics(id, nanCount)); + public Stream> metrics() { + FieldMetrics metricsWithoutNullCount = floatFieldMetricsBuilder.build(); + return Stream.of(new FieldMetrics<>(metricsWithoutNullCount.id(), + metricsWithoutNullCount.valueCount() + nullValueCount, + nullValueCount, metricsWithoutNullCount.nanValueCount(), + metricsWithoutNullCount.lowerBound(), metricsWithoutNullCount.upperBound())); } } private static class DoubleWriter implements OrcValueWriter { - private final int id; - private long nanCount; + private final DoubleFieldMetrics.Builder doubleFieldMetricsBuilder; + private long nullValueCount = 0; private DoubleWriter(Integer id) { - this.id = id; - this.nanCount = 0; + this.doubleFieldMetricsBuilder = new DoubleFieldMetrics.Builder(id); } @Override @@ -263,14 +269,21 @@ public Class getJavaClass() { @Override public void nonNullWrite(int rowId, Double data, ColumnVector output) { ((DoubleColumnVector) output).vector[rowId] = data; - if (Double.isNaN(data)) { - nanCount++; - } + doubleFieldMetricsBuilder.addValue(data); + } + + @Override + public void nullWrite() { + nullValueCount++; } @Override - public Stream metrics() { - return Stream.of(new FloatFieldMetrics(id, nanCount)); + public Stream> metrics() { + FieldMetrics metricsWithoutNullCount = doubleFieldMetricsBuilder.build(); + return Stream.of(new FieldMetrics<>(metricsWithoutNullCount.id(), + metricsWithoutNullCount.valueCount() + nullValueCount, + nullValueCount, metricsWithoutNullCount.nanValueCount(), + metricsWithoutNullCount.lowerBound(), metricsWithoutNullCount.upperBound())); } } @@ -469,7 +482,7 @@ public void nonNullWrite(int rowId, List value, ColumnVector output) { } @Override - public Stream metrics() { + public Stream> metrics() { return element.metrics(); } } @@ -513,7 +526,7 @@ public void nonNullWrite(int rowId, Map map, ColumnVector output) { } @Override - public Stream metrics() { + public Stream> metrics() { return Stream.concat(keyWriter.metrics(), valueWriter.metrics()); } } diff --git a/data/src/test/java/org/apache/iceberg/RecordWrapperTest.java b/data/src/test/java/org/apache/iceberg/RecordWrapperTest.java new file mode 100644 index 000000000000..06814728869e --- /dev/null +++ b/data/src/test/java/org/apache/iceberg/RecordWrapperTest.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg; + +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.StructLikeWrapper; +import org.junit.Assert; +import org.junit.Test; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +public abstract class RecordWrapperTest { + + private static final Types.StructType PRIMITIVE_WITHOUT_TIME = Types.StructType.of( + required(100, "id", Types.LongType.get()), + optional(101, "data", Types.StringType.get()), + required(102, "b", Types.BooleanType.get()), + optional(103, "i", Types.IntegerType.get()), + required(104, "l", Types.LongType.get()), + optional(105, "f", Types.FloatType.get()), + required(106, "d", Types.DoubleType.get()), + optional(107, "date", Types.DateType.get()), + required(108, "ts_tz", Types.TimestampType.withZone()), + required(110, "s", Types.StringType.get()), + required(112, "fixed", Types.FixedType.ofLength(7)), + optional(113, "bytes", Types.BinaryType.get()), + required(114, "dec_9_0", Types.DecimalType.of(9, 0)), + required(115, "dec_11_2", Types.DecimalType.of(11, 2)), + required(116, "dec_38_10", Types.DecimalType.of(38, 10))// maximum precision + ); + + private static final Types.StructType TIMESTAMP_WITHOUT_ZONE = Types.StructType.of( + required(101, "ts0", Types.TimestampType.withoutZone()), + required(102, "ts1", Types.TimestampType.withoutZone()) + ); + + protected static final Types.StructType TIME = Types.StructType.of( + required(100, "time0", Types.TimeType.get()), + optional(101, "time1", Types.TimeType.get()) + ); + + @Test + public void testSimpleStructWithoutTime() { + generateAndValidate(new Schema(PRIMITIVE_WITHOUT_TIME.fields())); + } + + @Test + public void testTimestampWithoutZone() { + generateAndValidate(new Schema(TIMESTAMP_WITHOUT_ZONE.fields())); + } + + @Test + public void testTime() { + generateAndValidate(new Schema(TIME.fields())); + } + + @Test + public void testNestedSchema() { + Types.StructType structType = Types.StructType.of( + required(0, "id", Types.LongType.get()), + required(1, "level1", Types.StructType.of( + optional(2, "level2", Types.StructType.of( + required(3, "level3", Types.StructType.of( + optional(4, "level4", Types.StructType.of( + required(5, "level5", Types.StructType.of( + PRIMITIVE_WITHOUT_TIME.fields() + )) + )) + )) + )) + )) + ); + + generateAndValidate(new Schema(structType.fields())); + } + + private void generateAndValidate(Schema schema) { + generateAndValidate(schema, Assert::assertEquals); + } + + public interface AssertMethod { + void assertEquals(String message, StructLikeWrapper expected, StructLikeWrapper actual); + } + + protected abstract void generateAndValidate(Schema schema, AssertMethod assertMethod); +} diff --git a/data/src/test/java/org/apache/iceberg/TestMergingMetrics.java b/data/src/test/java/org/apache/iceberg/TestMergingMetrics.java index fa588302e8e2..d1d59b3ffd74 100644 --- a/data/src/test/java/org/apache/iceberg/TestMergingMetrics.java +++ b/data/src/test/java/org/apache/iceberg/TestMergingMetrics.java @@ -19,15 +19,26 @@ package org.apache.iceberg; +import java.nio.ByteBuffer; +import java.util.Collection; +import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.NaNUtil; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -105,74 +116,152 @@ public TestMergingMetrics(FileFormat fileFormat) { @Test public void verifySingleRecordMetric() throws Exception { Record record = GenericRecord.create(SCHEMA); - record.setField("id", 3); - record.setField("float", Float.NaN); // FLOAT_FIELD - 1 - record.setField("double", Double.NaN); // DOUBLE_FIELD - 1 - record.setField("floatlist", ImmutableList.of(3.3F, 2.8F, Float.NaN, -25.1F, Float.NaN)); // FLOAT_LIST - 2 - record.setField("map1", ImmutableMap.of(Float.NaN, "a", 0F, "b")); // MAP_FIELD_1 - 1 - record.setField("map2", ImmutableMap.of( + record.setField(ID_FIELD.name(), 3); + record.setField(FLOAT_FIELD.name(), Float.NaN); // FLOAT_FIELD - 1 + record.setField(DOUBLE_FIELD.name(), Double.NaN); // DOUBLE_FIELD - 1 + record.setField(FLOAT_LIST.name(), ImmutableList.of(3.3F, 2.8F, Float.NaN, -25.1F, Float.NaN)); // FLOAT_LIST - 2 + record.setField(MAP_FIELD_1.name(), ImmutableMap.of(Float.NaN, "a", 0F, "b")); // MAP_FIELD_1 - 1 + record.setField(MAP_FIELD_2.name(), ImmutableMap.of( 0, 0D, 1, Double.NaN, 2, 2D, 3, Double.NaN, 4, Double.NaN)); // MAP_FIELD_2 - 3 FileAppender appender = writeAndGetAppender(ImmutableList.of(record)); - Map nanValueCount = appender.metrics().nanValueCounts(); + Metrics metrics = appender.metrics(); + Map nanValueCount = metrics.nanValueCounts(); + Map upperBounds = metrics.upperBounds(); + Map lowerBounds = metrics.lowerBounds(); assertNaNCountMatch(1L, nanValueCount, FLOAT_FIELD); assertNaNCountMatch(1L, nanValueCount, DOUBLE_FIELD); assertNaNCountMatch(2L, nanValueCount, FLOAT_LIST); assertNaNCountMatch(1L, nanValueCount, MAP_FIELD_1); assertNaNCountMatch(3L, nanValueCount, MAP_FIELD_2); - } - private void assertNaNCountMatch(Long expected, Map nanValueCount, Types.NestedField field) { - Assert.assertEquals( - String.format("NaN count for field %s does not match expected", field.name()), - expected, nanValueCount.get(FIELDS_WITH_NAN_COUNT_TO_ID.get(field))); + assertBoundValueMatch(null, upperBounds, FLOAT_FIELD); + assertBoundValueMatch(null, upperBounds, DOUBLE_FIELD); + assertBoundValueMatch(3.3F, upperBounds, FLOAT_LIST); + assertBoundValueMatch(0F, upperBounds, MAP_FIELD_1); + assertBoundValueMatch(2D, upperBounds, MAP_FIELD_2); + + assertBoundValueMatch(null, lowerBounds, FLOAT_FIELD); + assertBoundValueMatch(null, lowerBounds, DOUBLE_FIELD); + assertBoundValueMatch(-25.1F, lowerBounds, FLOAT_LIST); + assertBoundValueMatch(0F, lowerBounds, MAP_FIELD_1); + assertBoundValueMatch(0D, lowerBounds, MAP_FIELD_2); } @Test public void verifyRandomlyGeneratedRecordsMetric() throws Exception { - List recordList = RandomGenericData.generate(SCHEMA, 50, 250L); - + // too big of the record count will more likely to make all upper/lower bounds +/-infinity, + // which makes the tests easier to pass + List recordList = RandomGenericData.generate(SCHEMA, 5, 250L); FileAppender appender = writeAndGetAppender(recordList); - Map nanValueCount = appender.metrics().nanValueCounts(); - FIELDS_WITH_NAN_COUNT_TO_ID.forEach((key, value) -> Assert.assertEquals( - String.format("NaN count for field %s does not match expected", key.name()), - getExpectedNaNCount(recordList, key), - nanValueCount.get(value))); + Map> expectedUpperBounds = new HashMap<>(); + Map> expectedLowerBounds = new HashMap<>(); + Map expectedNaNCount = new HashMap<>(); + + populateExpectedValues(recordList, expectedUpperBounds, expectedLowerBounds, expectedNaNCount); + + Metrics metrics = appender.metrics(); + expectedUpperBounds.forEach((key, value) -> assertBoundValueMatch(value.get(), metrics.upperBounds(), key)); + expectedLowerBounds.forEach((key, value) -> assertBoundValueMatch(value.get(), metrics.lowerBounds(), key)); + expectedNaNCount.forEach((key, value) -> assertNaNCountMatch(value.get(), metrics.nanValueCounts(), key)); SCHEMA.columns().stream() .filter(column -> !FIELDS_WITH_NAN_COUNT_TO_ID.containsKey(column)) .map(Types.NestedField::fieldId) - .forEach(id -> Assert.assertNull("NaN count for field %s should be null", nanValueCount.get(id))); + .forEach(id -> Assert.assertNull("NaN count for field %s should be null", + metrics.nanValueCounts().get(id))); + } + + private void assertNaNCountMatch(Long expected, Map nanValueCount, Types.NestedField field) { + Assert.assertEquals( + String.format("NaN count for field %s does not match expected", field.name()), + expected, nanValueCount.get(FIELDS_WITH_NAN_COUNT_TO_ID.get(field))); + } + + private void assertBoundValueMatch(Number expected, Map boundMap, Types.NestedField field) { + if (field.type().isNestedType() && fileFormat == FileFormat.ORC) { + // we don't update floating column bounds values within ORC nested columns + return; + } + + int actualFieldId = FIELDS_WITH_NAN_COUNT_TO_ID.get(field); + ByteBuffer byteBuffer = boundMap.get(actualFieldId); + Type type = SCHEMA.findType(actualFieldId); + Assert.assertEquals(String.format("Bound value for field %s must match", field.name()), + expected, byteBuffer == null ? null : Conversions.fromByteBuffer(type, byteBuffer)); + } + + private void populateExpectedValues(List records, + Map> upperBounds, + Map> lowerBounds, + Map expectedNaNCount) { + for (Types.NestedField field : FIELDS_WITH_NAN_COUNT_TO_ID.keySet()) { + expectedNaNCount.put(field, new AtomicLong(0)); + } + + for (Record record : records) { + updateExpectedValuePerRecord(upperBounds, lowerBounds, expectedNaNCount, + FLOAT_FIELD, (Float) record.getField(FLOAT_FIELD.name())); + updateExpectedValuePerRecord(upperBounds, lowerBounds, expectedNaNCount, + DOUBLE_FIELD, (Double) record.getField(DOUBLE_FIELD.name())); + + List floatList = (List) record.getField(FLOAT_LIST.name()); + if (floatList != null) { + updateExpectedValueFromRecords(upperBounds, lowerBounds, expectedNaNCount, FLOAT_LIST, floatList); + } + + Map map1 = (Map) record.getField(MAP_FIELD_1.name()); + if (map1 != null) { + updateExpectedValueFromRecords(upperBounds, lowerBounds, expectedNaNCount, MAP_FIELD_1, map1.keySet()); + } + + Map map2 = (Map) record.getField(MAP_FIELD_2.name()); + if (map2 != null) { + updateExpectedValueFromRecords(upperBounds, lowerBounds, expectedNaNCount, MAP_FIELD_2, map2.values()); + } + } + } + + private void updateExpectedValueFromRecords( + Map> upperBounds, + Map> lowerBounds, + Map expectedNaNCount, + Types.NestedField key, Collection vals) { + List nonNullNumbers = vals.stream().filter(v -> !NaNUtil.isNaN(v)).collect(Collectors.toList()); + Optional maxOptional = nonNullNumbers.stream().filter(Objects::nonNull) + .reduce((v1, v2) -> getMinOrMax(v1, v2, true)); + Optional minOptional = nonNullNumbers.stream().filter(Objects::nonNull) + .reduce((v1, v2) -> getMinOrMax(v1, v2, false)); + + expectedNaNCount.get(key).addAndGet(vals.size() - nonNullNumbers.size()); + maxOptional.ifPresent(max -> updateBound(key, max, upperBounds, true)); + minOptional.ifPresent(min -> updateBound(key, min, lowerBounds, false)); + } + + private void updateExpectedValuePerRecord(Map> upperBounds, + Map> lowerBounds, + Map expectedNaNCount, + Types.NestedField key, Number val) { + if (NaNUtil.isNaN(val)) { + expectedNaNCount.get(key).incrementAndGet(); + } else if (val != null) { + updateBound(key, val, upperBounds, true); + updateBound(key, val, lowerBounds, false); + } + } + + private void updateBound( + Types.NestedField key, Number val, Map> bounds, boolean isMax) { + bounds.computeIfAbsent(key, k -> new AtomicReference<>(val)).updateAndGet(old -> getMinOrMax(old, val, isMax)); } - private Long getExpectedNaNCount(List expectedRecords, Types.NestedField field) { - return expectedRecords.stream() - .mapToLong(e -> { - Object value = e.getField(field.name()); - if (value == null) { - return 0; - } - if (FLOAT_FIELD.equals(field)) { - return Float.isNaN((Float) value) ? 1 : 0; - } else if (DOUBLE_FIELD.equals(field)) { - return Double.isNaN((Double) value) ? 1 : 0; - } else if (FLOAT_LIST.equals(field)) { - return ((List) value).stream() - .filter(val -> val != null && Float.isNaN(val)) - .count(); - } else if (MAP_FIELD_1.equals(field)) { - return ((Map) value).keySet().stream() - .filter(key -> Float.isNaN(key)) - .count(); - } else if (MAP_FIELD_2.equals(field)) { - return ((Map) value).values().stream() - .filter(val -> val != null && Double.isNaN(val)) - .count(); - } else { - throw new RuntimeException("unknown field name for getting expected NaN count: " + field.name()); - } - }).sum(); + private Number getMinOrMax(Number val1, Number val2, boolean isMax) { + if (val1 instanceof Double) { + return isMax ? Double.max((Double) val1, (Double) val2) : Double.min((Double) val1, (Double) val2); + } else { + return isMax ? Float.max((Float) val1, (Float) val2) : Float.min((Float) val1, (Float) val2); + } } } diff --git a/data/src/test/java/org/apache/iceberg/data/DeleteReadTests.java b/data/src/test/java/org/apache/iceberg/data/DeleteReadTests.java index 4c58e562de7e..70ac77473c5d 100644 --- a/data/src/test/java/org/apache/iceberg/data/DeleteReadTests.java +++ b/data/src/test/java/org/apache/iceberg/data/DeleteReadTests.java @@ -33,6 +33,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.ArrayUtil; +import org.apache.iceberg.util.CharSequenceSet; import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.StructLikeSet; import org.apache.iceberg.util.StructProjection; @@ -193,7 +194,7 @@ public void testPositionDeletes() throws IOException { Pair.of(dataFile.path(), 6L) // id = 122 ); - Pair> posDeletes = FileHelpers.writeDeleteFile( + Pair posDeletes = FileHelpers.writeDeleteFile( table, Files.localOutput(temp.newFile()), Row.of(0), deletes); table.newRowDelta() @@ -225,7 +226,7 @@ public void testMixedPositionAndEqualityDeletes() throws IOException { Pair.of(dataFile.path(), 5L) // id = 121 ); - Pair> posDeletes = FileHelpers.writeDeleteFile( + Pair posDeletes = FileHelpers.writeDeleteFile( table, Files.localOutput(temp.newFile()), Row.of(0), deletes); table.newRowDelta() diff --git a/data/src/test/java/org/apache/iceberg/data/FileHelpers.java b/data/src/test/java/org/apache/iceberg/data/FileHelpers.java index 64488e78e65b..362f8463353d 100644 --- a/data/src/test/java/org/apache/iceberg/data/FileHelpers.java +++ b/data/src/test/java/org/apache/iceberg/data/FileHelpers.java @@ -22,7 +22,6 @@ import java.io.Closeable; import java.io.IOException; import java.util.List; -import java.util.Set; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.DeleteFile; @@ -37,19 +36,20 @@ import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.CharSequenceSet; import org.apache.iceberg.util.Pair; public class FileHelpers { private FileHelpers() { } - public static Pair> writeDeleteFile(Table table, OutputFile out, + public static Pair writeDeleteFile(Table table, OutputFile out, List> deletes) throws IOException { return writeDeleteFile(table, out, null, deletes); } - public static Pair> writeDeleteFile(Table table, OutputFile out, StructLike partition, + public static Pair writeDeleteFile(Table table, OutputFile out, StructLike partition, List> deletes) throws IOException { PositionDeleteWriter writer = Parquet.writeDeletes(out) diff --git a/data/src/test/java/org/apache/iceberg/data/TestDataFileIndexStatsFilters.java b/data/src/test/java/org/apache/iceberg/data/TestDataFileIndexStatsFilters.java index 8b1eeec5b104..9a3a04255dc7 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestDataFileIndexStatsFilters.java +++ b/data/src/test/java/org/apache/iceberg/data/TestDataFileIndexStatsFilters.java @@ -22,7 +22,6 @@ import java.io.File; import java.io.IOException; import java.util.List; -import java.util.Set; import java.util.stream.Collectors; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; @@ -35,6 +34,7 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.CharSequenceSet; import org.apache.iceberg.util.Pair; import org.junit.After; import org.junit.Assert; @@ -97,7 +97,7 @@ public void testPositionDeletePlanningPath() throws IOException { deletes.add(Pair.of(dataFile.path(), 0L)); deletes.add(Pair.of(dataFile.path(), 1L)); - Pair> posDeletes = FileHelpers.writeDeleteFile( + Pair posDeletes = FileHelpers.writeDeleteFile( table, Files.localOutput(temp.newFile()), deletes); table.newRowDelta() .addDeletes(posDeletes.first()) @@ -124,7 +124,7 @@ public void testPositionDeletePlanningPathFilter() throws IOException { deletes.add(Pair.of("some-other-file.parquet", 0L)); deletes.add(Pair.of("some-other-file.parquet", 1L)); - Pair> posDeletes = FileHelpers.writeDeleteFile( + Pair posDeletes = FileHelpers.writeDeleteFile( table, Files.localOutput(temp.newFile()), deletes); table.newRowDelta() .addDeletes(posDeletes.first()) diff --git a/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java b/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java index 406c9793021b..e95e2580361d 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java +++ b/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java @@ -51,6 +51,7 @@ *

  • warehouse - the warehouse path (Hadoop catalog only)
  • *
  • default-database - a database name to use as the default
  • *
  • base-namespace - a base namespace as the prefix for all databases (Hadoop catalog only)
  • + *
  • cache-enabled - whether to enable catalog cache
  • * *

    * To use a custom catalog that is not a Hive or Hadoop catalog, extend this class and override diff --git a/flink/src/main/java/org/apache/iceberg/flink/FlinkTableOptions.java b/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java similarity index 96% rename from flink/src/main/java/org/apache/iceberg/flink/FlinkTableOptions.java rename to flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java index 145190c9dedb..067abe8a6e41 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/FlinkTableOptions.java +++ b/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java @@ -23,9 +23,9 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; -public class FlinkTableOptions { +public class FlinkConfigOptions { - private FlinkTableOptions() { + private FlinkConfigOptions() { } public static final ConfigOption TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM = diff --git a/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java b/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java index 2cc9771c1969..88276d86d3df 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java +++ b/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java @@ -171,6 +171,7 @@ public Type visit(MapType mapType) { } @Override + @SuppressWarnings("ReferenceEquality") public Type visit(RowType rowType) { List newFields = Lists.newArrayListWithExpectedSize(rowType.getFieldCount()); boolean isRoot = root == rowType; diff --git a/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java b/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java index 9aff0c127449..81f9822815b6 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java +++ b/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java @@ -68,7 +68,7 @@ public void write(RowData row, VectorizedRowBatch output) { } @Override - public Stream metrics() { + public Stream> metrics() { return writer.metrics(); } diff --git a/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java b/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java index 758d73d87e9b..38a348995f00 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java +++ b/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java @@ -258,7 +258,7 @@ public void nonNullWrite(int rowId, ArrayData data, ColumnVector output) { } @Override - public Stream metrics() { + public Stream> metrics() { return elementWriter.metrics(); } @@ -306,7 +306,7 @@ public void nonNullWrite(int rowId, MapData data, ColumnVector output) { } @Override - public Stream metrics() { + public Stream> metrics() { return Stream.concat(keyWriter.metrics(), valueWriter.metrics()); } } @@ -344,7 +344,7 @@ public void nonNullWrite(int rowId, RowData data, ColumnVector output) { } @Override - public Stream metrics() { + public Stream> metrics() { return writers.stream().flatMap(OrcValueWriter::metrics); } } diff --git a/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java b/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java index 319568bb6e99..d7088b4700e5 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java +++ b/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java @@ -118,6 +118,9 @@ public ParquetValueReader struct(Types.StructType expected, GroupType s } else if (id == MetadataColumns.ROW_POSITION.fieldId()) { reorderedFields.add(ParquetValueReaders.position()); types.add(null); + } else if (id == MetadataColumns.IS_DELETED.fieldId()) { + reorderedFields.add(ParquetValueReaders.constant(false)); + types.add(null); } else { ParquetValueReader reader = readersById.get(id); if (reader != null) { diff --git a/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java index f679e2325dd0..ed6552809a52 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java @@ -29,6 +29,7 @@ import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.functions.sink.DiscardingSink; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; @@ -124,6 +125,7 @@ public static class Builder { private DistributionMode distributionMode = null; private Integer writeParallelism = null; private List equalityFieldColumns = null; + private String uidPrefix = null; private Builder() { } @@ -205,6 +207,30 @@ public Builder equalityFieldColumns(List columns) { return this; } + /** + * Set the uid prefix for FlinkSink operators. Note that FlinkSink internally consists of multiple operators (like + * writer, committer, dummy sink etc.) Actually operator uid will be appended with a suffix like "uid-writer". + *

    + * Flink auto generates operator uids if not set explicitly. It is a recommended + * + * best-practice to set uid for all operators before deploying to production. Flink has an option to {@code + * pipeline.auto-generate-uids=false} to disable auto-generation and force explicit setting of all operator uids. + *

    + * Be careful with setting this for an existing job, because now we are changing the opeartor uid from an + * auto-generated one to this new value. When deploying the change with a checkpoint, Flink won't be able to restore + * the previous Flink sink operator state (more specifically the committer operator state). You need to use {@code + * --allowNonRestoredState} to ignore the previous sink state. During restore Flink sink state is used to check if + * checkpointed files were actually committed or not. {@code --allowNonRestoredState} can lead to data loss if the + * Iceberg commit failed in the last completed checkpoints. + * + * @param newPrefix UID prefix for Flink sink operators + * @return {@link Builder} to connect the iceberg table. + */ + public Builder uidPrefix(String newPrefix) { + this.uidPrefix = newPrefix; + return this; + } + @SuppressWarnings("unchecked") public DataStreamSink build() { Preconditions.checkArgument(rowDataInput != null, @@ -243,16 +269,29 @@ public DataStreamSink build() { this.writeParallelism = writeParallelism == null ? rowDataInput.getParallelism() : writeParallelism; - DataStream returnStream = rowDataInput + SingleOutputStreamOperator writerStream = rowDataInput .transform(ICEBERG_STREAM_WRITER_NAME, TypeInformation.of(WriteResult.class), streamWriter) - .setParallelism(writeParallelism) + .setParallelism(writeParallelism); + if (uidPrefix != null) { + writerStream = writerStream.uid(uidPrefix + "-writer"); + } + + SingleOutputStreamOperator committerStream = writerStream .transform(ICEBERG_FILES_COMMITTER_NAME, Types.VOID, filesCommitter) .setParallelism(1) .setMaxParallelism(1); + if (uidPrefix != null) { + committerStream = committerStream.uid(uidPrefix + "-committer"); + } - return returnStream.addSink(new DiscardingSink()) + DataStreamSink resultStream = committerStream + .addSink(new DiscardingSink()) .name(String.format("IcebergSink %s", table.name())) .setParallelism(1); + if (uidPrefix != null) { + resultStream = resultStream.uid(uidPrefix + "-dummysink"); + } + return resultStream; } private DataStream distributeDataStream(DataStream input, diff --git a/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java b/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java index f44ea4ac5a90..f74a8968fab8 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java +++ b/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.nio.ByteBuffer; -import java.util.Collections; import java.util.Iterator; import java.util.Map; import java.util.stream.Stream; @@ -35,6 +34,7 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; /** @@ -64,7 +64,7 @@ abstract class DataIterator implements CloseableIterator { Map files = Maps.newHashMapWithExpectedSize(task.files().size()); decryptedFiles.forEach(decrypted -> files.putIfAbsent(decrypted.location(), decrypted)); - this.inputFiles = Collections.unmodifiableMap(files); + this.inputFiles = ImmutableMap.copyOf(files); this.currentIterator = CloseableIterator.empty(); } diff --git a/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java b/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java index 84507c411bcc..a3263d284c0c 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java +++ b/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java @@ -36,8 +36,8 @@ import org.apache.iceberg.TableScan; import org.apache.iceberg.encryption.EncryptionManager; import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkConfigOptions; import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.FlinkTableOptions; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; import org.apache.iceberg.io.FileIO; @@ -48,11 +48,11 @@ private FlinkSource() { } /** - * Initialize a {@link Builder} to read the data from iceberg table. Equivalent to {@link TableScan}. - * See more options in {@link ScanContext}. + * Initialize a {@link Builder} to read the data from iceberg table. Equivalent to {@link TableScan}. See more options + * in {@link ScanContext}. *

    - * The Source can be read static data in bounded mode. It can also continuously check the arrival of new data and - * read records incrementally. + * The Source can be read static data in bounded mode. It can also continuously check the arrival of new data and read + * records incrementally. *

      *
    • Without startSnapshotId: Bounded
    • *
    • With startSnapshotId and with endSnapshotId: Bounded
    • @@ -222,10 +222,11 @@ public DataStream build() { int inferParallelism(FlinkInputFormat format, ScanContext context) { int parallelism = readableConfig.get(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM); - if (readableConfig.get(FlinkTableOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM)) { - int maxInferParallelism = readableConfig.get(FlinkTableOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX); + if (readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM)) { + int maxInferParallelism = readableConfig.get(FlinkConfigOptions + .TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX); Preconditions.checkState(maxInferParallelism >= 1, - FlinkTableOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX.key() + " cannot be less than 1"); + FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX.key() + " cannot be less than 1"); int splitNum; try { FlinkInputSplit[] splits = format.createInputSplits(0); diff --git a/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java b/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java index 28103263172e..1ee000c82360 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java +++ b/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java @@ -79,7 +79,7 @@ protected TableEnvironment getTableEnv() { .build(); TableEnvironment env = TableEnvironment.create(settings); - env.getConfig().getConfiguration().set(FlinkTableOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); + env.getConfig().getConfiguration().set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); tEnv = env; } } diff --git a/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java index 5e1372da925d..d8c91b5a64ab 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java +++ b/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java @@ -53,7 +53,6 @@ import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.FileAppenderFactory; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.HashMultiset; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; @@ -191,9 +190,19 @@ public static void assertTableRows(Table table, List expected) throws I public static void assertTableRecords(Table table, List expected) throws IOException { table.refresh(); + + Types.StructType type = table.schema().asStruct(); + StructLikeSet expectedSet = StructLikeSet.create(type); + expectedSet.addAll(expected); + try (CloseableIterable iterable = IcebergGenerics.read(table).build()) { - Assert.assertEquals("Should produce the expected record", - HashMultiset.create(expected), HashMultiset.create(iterable)); + StructLikeSet actualSet = StructLikeSet.create(type); + + for (Record record : iterable) { + actualSet.add(record); + } + + Assert.assertEquals("Should produce the expected record", expectedSet, actualSet); } } diff --git a/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java b/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java new file mode 100644 index 000000000000..9012fc564bd1 --- /dev/null +++ b/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.flink; + +import java.util.Iterator; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.RecordWrapperTest; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.data.InternalRecordWrapper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.data.RandomRowData; +import org.apache.iceberg.util.StructLikeWrapper; +import org.junit.Assert; + +public class TestRowDataWrapper extends RecordWrapperTest { + + /** + * Flink's time type has been truncated to millis seconds, so we need a customized assert method to check the + * values. + */ + @Override + public void testTime() { + generateAndValidate(new Schema(TIME.fields()), (message, expectedWrapper, actualWrapper) -> { + for (int pos = 0; pos < TIME.fields().size(); pos++) { + Object expected = expectedWrapper.get().get(pos, Object.class); + Object actual = actualWrapper.get().get(pos, Object.class); + if (expected == actual) { + return; + } + + if (expected == null || actual == null) { + Assert.fail(String.format("The expected value is %s but actual value is %s", expected, actual)); + } + + int expectedMilliseconds = (int) ((long) expected / 1000_000); + int actualMilliseconds = (int) ((long) actual / 1000_000); + Assert.assertEquals(message, expectedMilliseconds, actualMilliseconds); + } + }); + } + + @Override + protected void generateAndValidate(Schema schema, RecordWrapperTest.AssertMethod assertMethod) { + int numRecords = 100; + Iterable recordList = RandomGenericData.generate(schema, numRecords, 101L); + Iterable rowDataList = RandomRowData.generate(schema, numRecords, 101L); + + InternalRecordWrapper recordWrapper = new InternalRecordWrapper(schema.asStruct()); + RowDataWrapper rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + + Iterator actual = recordList.iterator(); + Iterator expected = rowDataList.iterator(); + + StructLikeWrapper actualWrapper = StructLikeWrapper.forType(schema.asStruct()); + StructLikeWrapper expectedWrapper = StructLikeWrapper.forType(schema.asStruct()); + for (int i = 0; i < numRecords; i++) { + Assert.assertTrue("Should have more records", actual.hasNext()); + Assert.assertTrue("Should have more RowData", expected.hasNext()); + + StructLike recordStructLike = recordWrapper.wrap(actual.next()); + StructLike rowDataStructLike = rowDataWrapper.wrap(expected.next()); + + assertMethod.assertEquals("Should have expected StructLike values", + actualWrapper.set(recordStructLike), expectedWrapper.set(rowDataStructLike)); + } + + Assert.assertFalse("Shouldn't have more record", actual.hasNext()); + Assert.assertFalse("Shouldn't have more RowData", expected.hasNext()); + } +} diff --git a/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java index 4ada31a619ca..34785cfb6a34 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ b/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -815,7 +815,7 @@ private FileAppenderFactory createDeletableAppenderFactory() { private ManifestFile createTestingManifestFile(Path manifestPath) { return new GenericManifestFile(manifestPath.toAbsolutePath().toString(), manifestPath.toFile().length(), 0, - ManifestContent.DATA, 0, 0, 0L, 0, 0, 0, 0, 0, 0, null); + ManifestContent.DATA, 0, 0, 0L, 0, 0, 0, 0, 0, 0, null, null); } private List assertFlinkManifests(int expectedCount) throws IOException { diff --git a/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java b/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java index f99e4b682a86..9af1b7c65331 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java +++ b/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java @@ -38,7 +38,7 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.flink.FlinkTableOptions; +import org.apache.iceberg.flink.FlinkConfigOptions; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -158,7 +158,7 @@ public void testInferedParallelism() throws IOException { // 2 splits and max infer parallelism is 1 (max < splits num), the parallelism is 1 Configuration configuration = new Configuration(); - configuration.setInteger(FlinkTableOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX, 1); + configuration.setInteger(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX, 1); parallelism = FlinkSource.forRowData() .flinkConf(configuration) .inferParallelism(flinkInputFormat, ScanContext.builder().build()); @@ -171,7 +171,7 @@ public void testInferedParallelism() throws IOException { Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); // 2 splits, infer parallelism is disabled, the parallelism is flink default parallelism 1 - configuration.setBoolean(FlinkTableOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); + configuration.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); parallelism = FlinkSource.forRowData() .flinkConf(configuration) .inferParallelism(flinkInputFormat, ScanContext.builder().limit(3).build()); diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/CachedClientPool.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/CachedClientPool.java index 0079621a26c8..33f026fbd952 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/CachedClientPool.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/CachedClientPool.java @@ -58,7 +58,6 @@ HiveClientPool clientPool() { return clientPoolCache.get(metastoreUri, k -> new HiveClientPool(clientPoolSize, conf)); } - private synchronized void init() { if (clientPoolCache == null) { clientPoolCache = Caffeine.newBuilder().expireAfterAccess(evictionInterval, TimeUnit.MILLISECONDS) diff --git a/jmh.gradle b/jmh.gradle index 3fd4766494c5..031618ff6c3a 100644 --- a/jmh.gradle +++ b/jmh.gradle @@ -17,7 +17,14 @@ * under the License. */ -def jmhProjects = [ project("iceberg-spark2") ] +def jmhProjects = [] +if (JavaVersion.current() == JavaVersion.VERSION_1_8) { + jmhProjects = [ project("iceberg-spark2"), project("iceberg-spark3") ] +} else if (JavaVersion.current() == JavaVersion.VERSION_11) { + jmhProjects = [ project("iceberg-spark3") ] +} else { + throw new GradleException("The JMH benchamrks must be run with JDK 8 or JDK 11") +} configure(jmhProjects) { apply plugin: 'me.champeau.gradle.jmh' @@ -31,10 +38,17 @@ configure(jmhProjects) { include = [jmhIncludeRegex] } + // Path is relative to either spark2 or spark3 folder, depending on project being tested + sourceSets { + jmh { + java.srcDirs = ['src/jmh/java', '../spark/src/jmh/java'] + compileClasspath += sourceSets.main.runtimeClasspath + } + } + jmhCompileGeneratedClasses { pluginManager.withPlugin('com.palantir.baseline-error-prone') { options.errorprone.enabled = false } } } - diff --git a/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergSplit.java b/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergSplit.java index 43b78d661a0d..632224eea876 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergSplit.java +++ b/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergSplit.java @@ -27,8 +27,6 @@ import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.hadoop.HadoopFileIO; -import org.apache.iceberg.hadoop.SerializableConfiguration; import org.apache.iceberg.hadoop.Util; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.mr.InputFormatConfig; @@ -92,13 +90,7 @@ public void write(DataOutput out) throws IOException { out.writeInt(data.length); out.write(data); - byte[] ioData; - if (io instanceof HadoopFileIO) { - SerializableConfiguration serializableConf = new SerializableConfiguration(((HadoopFileIO) io).conf()); - ioData = SerializationUtil.serializeToBytes(new HadoopFileIO(serializableConf::get)); - } else { - ioData = SerializationUtil.serializeToBytes(io); - } + byte[] ioData = SerializationUtil.serializeToBytes(io); out.writeInt(ioData.length); out.write(ioData); diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerTestUtils.java b/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerTestUtils.java index b08673ca89ca..649998fa718c 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerTestUtils.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerTestUtils.java @@ -96,13 +96,6 @@ static void init(TestHiveShell shell, TestTables testTables, TemporaryFolder tem shell.setHiveSessionValue("hive.execution.engine", engine); shell.setHiveSessionValue("hive.jar.directory", temp.getRoot().getAbsolutePath()); shell.setHiveSessionValue("tez.staging-dir", temp.getRoot().getAbsolutePath()); - - // temporarily disabling vectorization in Tez, since it doesn't work with projection pruning (fix: TEZ-4248) - // TODO: remove this once TEZ-4248 has been released and the Tez dependencies updated here - if (engine.equals("tez")) { - shell.setHiveSessionValue("hive.vectorized.execution.enabled", "false"); - } - } static void close(TestHiveShell shell) throws Exception { diff --git a/nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java b/nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java index d789899b3bf6..88c442185429 100644 --- a/nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java +++ b/nessie/src/main/java/org/apache/iceberg/nessie/NessieCatalog.java @@ -65,9 +65,9 @@ * Nessie implementation of Iceberg Catalog. * *

      - * A note on namespaces: Nessie namespaces are implicit and do not need to be explicitly created or deleted. - * The create and delete namespace methods are no-ops for the NessieCatalog. One can still list namespaces that have - * objects stored in them to assist with namespace-centric catalog exploration. + * A note on namespaces: Nessie namespaces are implicit and do not need to be explicitly created or deleted. + * The create and delete namespace methods are no-ops for the NessieCatalog. One can still list namespaces that have + * objects stored in them to assist with namespace-centric catalog exploration. *

      */ public class NessieCatalog extends BaseMetastoreCatalog implements AutoCloseable, SupportsNamespaces, Configurable { @@ -177,14 +177,14 @@ public boolean dropTable(TableIdentifier identifier, boolean purge) { boolean threw = true; try { Tasks.foreach(contents) - .retry(5) - .stopRetryOn(NessieNotFoundException.class) - .throwFailureWhenFinished() - .onFailure((c, exception) -> refresh()) - .run(c -> { - client.getTreeApi().commitMultipleOperations(reference.getAsBranch().getName(), reference.getHash(), c); - refresh(); // note: updated to reference.updateReference() with Nessie 0.6 - }, BaseNessieClientServerException.class); + .retry(5) + .stopRetryOn(NessieNotFoundException.class) + .throwFailureWhenFinished() + .onFailure((c, exception) -> refresh()) + .run(c -> { + client.getTreeApi().commitMultipleOperations(reference.getAsBranch().getName(), reference.getHash(), c); + refresh(); // note: updated to reference.updateReference() with Nessie 0.6 + }, BaseNessieClientServerException.class); threw = false; } catch (NessieConflictException e) { logger.error("Cannot drop table: failed after retry (update ref and retry)", e); @@ -212,7 +212,8 @@ public void renameTable(TableIdentifier from, TableIdentifier toOriginal) { } Operations contents = ImmutableOperations.builder() - .addOperations(ImmutablePut.builder().key(NessieUtil.toKey(to)).contents(existingFromTable).build(), + .addOperations( + ImmutablePut.builder().key(NessieUtil.toKey(to)).contents(existingFromTable).build(), ImmutableDelete.builder().key(NessieUtil.toKey(from)).build()) .commitMeta(NessieUtil.buildCommitMetadata("iceberg rename table", catalogOptions)) .build(); @@ -227,7 +228,6 @@ public void renameTable(TableIdentifier from, TableIdentifier toOriginal) { client.getTreeApi().commitMultipleOperations(reference.getAsBranch().getName(), reference.getHash(), c); refresh(); }, BaseNessieClientServerException.class); - } catch (NessieNotFoundException e) { // important note: the NotFoundException refers to the ref only. If a table was not found it would imply that the // another commit has deleted the table from underneath us. This would arise as a Conflict exception as opposed to @@ -247,7 +247,7 @@ public void renameTable(TableIdentifier from, TableIdentifier toOriginal) { * creating namespaces in nessie is implicit, therefore this is a no-op. Metadata is ignored. * * @param namespace a multi-part namespace - * @param metadata a string Map of properties for the given namespace + * @param metadata a string Map of properties for the given namespace */ @Override public void createNamespace(Namespace namespace, Map metadata) { @@ -338,11 +338,13 @@ private UpdateableReference loadReference(String requestedRef) { return new UpdateableReference(ref, client.getTreeApi()); } catch (NessieNotFoundException ex) { if (requestedRef != null) { - throw new IllegalArgumentException(String.format("Nessie ref '%s' does not exist. " + - "This ref must exist before creating a NessieCatalog.", requestedRef), ex); + throw new IllegalArgumentException(String.format( + "Nessie ref '%s' does not exist. This ref must exist before creating a NessieCatalog.", + requestedRef), ex); } - throw new IllegalArgumentException(String.format("Nessie does not have an existing default branch." + + throw new IllegalArgumentException(String.format( + "Nessie does not have an existing default branch." + "Either configure an alternative ref via %s or create the default branch on the server.", NessieConfigConstants.CONF_NESSIE_REF), ex); } diff --git a/nessie/src/main/java/org/apache/iceberg/nessie/TableReference.java b/nessie/src/main/java/org/apache/iceberg/nessie/TableReference.java index 9f613e60dbe8..41064968fbc8 100644 --- a/nessie/src/main/java/org/apache/iceberg/nessie/TableReference.java +++ b/nessie/src/main/java/org/apache/iceberg/nessie/TableReference.java @@ -57,7 +57,8 @@ public String reference() { */ public static TableReference parse(TableIdentifier path) { TableReference pti = parse(path.name()); - return new TableReference(TableIdentifier.of(path.namespace(), pti.tableIdentifier().name()), + return new TableReference( + TableIdentifier.of(path.namespace(), pti.tableIdentifier().name()), pti.timestamp(), pti.reference()); } diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java b/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java index db45cf040a8b..a73a768fe62f 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java @@ -106,7 +106,7 @@ NessieCatalog initCatalog(String ref) { CatalogProperties.URI, uri, "auth_type", "NONE", CatalogProperties.WAREHOUSE_LOCATION, temp.getRoot().toURI().toString() - )); + )); return newCatalog; } @@ -120,8 +120,7 @@ protected Table createTable(TableIdentifier tableIdentifier, int count) { } protected void createTable(TableIdentifier tableIdentifier) { - Schema schema = new Schema(StructType.of(required(1, "id", LongType.get())) - .fields()); + Schema schema = new Schema(StructType.of(required(1, "id", LongType.get())).fields()); catalog.createTable(tableIdentifier, schema).location(); } @@ -153,5 +152,4 @@ static String metadataLocation(NessieCatalog catalog, TableIdentifier tableIdent NessieTableOperations icebergOps = (NessieTableOperations) ops; return icebergOps.currentMetadataLocation(); } - } diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/NessieUtilTest.java b/nessie/src/test/java/org/apache/iceberg/nessie/NessieUtilTest.java index b5f7b27ab94a..6c021241d2e1 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/NessieUtilTest.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/NessieUtilTest.java @@ -21,15 +21,17 @@ import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; +import org.assertj.core.api.Assertions; import org.junit.Test; import org.projectnessie.model.CommitMeta; public class NessieUtilTest { - @Test(expected = IllegalArgumentException.class) + @Test public void testBuildingCommitMetadataWithNullCatalogOptions() { - NessieUtil.buildCommitMetadata("msg", null); + Assertions.assertThatThrownBy(() -> NessieUtil.buildCommitMetadata("msg", null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("catalogOptions must not be null"); } @Test @@ -40,21 +42,21 @@ public void testSparkAppIdAndUserIsSetOnCommitMetadata() { CommitMeta commitMeta = NessieUtil.buildCommitMetadata( commitMsg, ImmutableMap.of(CatalogProperties.APP_ID, appId, CatalogProperties.USER, user)); - Assert.assertEquals(commitMsg, commitMeta.getMessage()); - Assert.assertEquals(user, commitMeta.getAuthor()); - Assert.assertEquals(2, commitMeta.getProperties().size()); - Assert.assertEquals("iceberg", commitMeta.getProperties().get(NessieUtil.APPLICATION_TYPE)); - Assert.assertEquals(appId, commitMeta.getProperties().get(CatalogProperties.APP_ID)); + Assertions.assertThat(commitMeta.getMessage()).isEqualTo(commitMsg); + Assertions.assertThat(commitMeta.getAuthor()).isEqualTo(user); + Assertions.assertThat(commitMeta.getProperties()).hasSize(2); + Assertions.assertThat(commitMeta.getProperties().get(NessieUtil.APPLICATION_TYPE)).isEqualTo("iceberg"); + Assertions.assertThat(commitMeta.getProperties().get(CatalogProperties.APP_ID)).isEqualTo(appId); } @Test public void testAuthorIsSetOnCommitMetadata() { String commitMsg = "commit msg"; CommitMeta commitMeta = NessieUtil.buildCommitMetadata(commitMsg, ImmutableMap.of()); - Assert.assertEquals(commitMsg, commitMeta.getMessage()); - Assert.assertEquals(System.getProperty("user.name"), commitMeta.getAuthor()); - Assert.assertEquals(1, commitMeta.getProperties().size()); - Assert.assertEquals("iceberg", commitMeta.getProperties().get(NessieUtil.APPLICATION_TYPE)); + Assertions.assertThat(commitMeta.getMessage()).isEqualTo(commitMsg); + Assertions.assertThat(commitMeta.getAuthor()).isEqualTo(System.getProperty("user.name")); + Assertions.assertThat(commitMeta.getProperties()).hasSize(1); + Assertions.assertThat(commitMeta.getProperties().get(NessieUtil.APPLICATION_TYPE)).isEqualTo("iceberg"); } @Test @@ -63,7 +65,7 @@ public void testAuthorIsNullWithoutJvmUser() { try { System.clearProperty("user.name"); CommitMeta commitMeta = NessieUtil.buildCommitMetadata("commit msg", ImmutableMap.of()); - Assert.assertNull(commitMeta.getAuthor()); + Assertions.assertThat(commitMeta.getAuthor()).isNull(); } finally { System.setProperty("user.name", jvmUserName); } diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestBranchVisibility.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestBranchVisibility.java index 07dfa30c7e49..43bbc31e264b 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestBranchVisibility.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestBranchVisibility.java @@ -21,8 +21,8 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; import org.junit.After; -import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.projectnessie.error.NessieConflictException; @@ -30,7 +30,6 @@ public class TestBranchVisibility extends BaseTestIceberg { - private final TableIdentifier tableIdentifier1 = TableIdentifier.of("test-ns", "table1"); private final TableIdentifier tableIdentifier2 = TableIdentifier.of("test-ns", "table2"); private NessieCatalog testCatalog; @@ -40,7 +39,6 @@ public TestBranchVisibility() { super("main"); } - @Before public void before() throws NessieNotFoundException, NessieConflictException { createTable(tableIdentifier1, 1); // table 1 @@ -55,8 +53,7 @@ public void after() throws NessieNotFoundException, NessieConflictException { catalog.dropTable(tableIdentifier1); catalog.dropTable(tableIdentifier2); catalog.refresh(); - catalog.getTreeApi().deleteBranch("test", - catalog.getTreeApi().getReferenceByName("test").getHash()); + catalog.getTreeApi().deleteBranch("test", catalog.getTreeApi().getReferenceByName("test").getHash()); testCatalog = null; } @@ -78,7 +75,7 @@ public void testUpdateCatalogs() { testCatalogEquality(catalog, testCatalog, false, false); // points to the previous metadata location - Assert.assertEquals(initialMetadataLocation, metadataLocation(catalog, tableIdentifier2)); + Assertions.assertThat(initialMetadataLocation).isEqualTo(metadataLocation(catalog, tableIdentifier2)); } @Test @@ -102,16 +99,16 @@ public void testCatalogWithTableNames() throws NessieNotFoundException { String mainHash = tree.getReferenceByName("main").getHash(); // asking for table@branch gives expected regardless of catalog - Assert.assertEquals(metadataLocation(catalog, TableIdentifier.of("test-ns", "table1@test")), - metadataLocation(testCatalog, tableIdentifier1)); + Assertions.assertThat(metadataLocation(catalog, TableIdentifier.of("test-ns", "table1@test"))) + .isEqualTo(metadataLocation(testCatalog, tableIdentifier1)); // asking for table@branch#hash gives expected regardless of catalog - Assert.assertEquals(metadataLocation(catalog, TableIdentifier.of("test-ns", "table1@" + mainHash)), - metadataLocation(testCatalog, tableIdentifier1)); + Assertions.assertThat(metadataLocation(catalog, TableIdentifier.of("test-ns", "table1@" + mainHash))) + .isEqualTo(metadataLocation(testCatalog, tableIdentifier1)); } @Test - public void testConcurrentChanges() throws NessieNotFoundException { + public void testConcurrentChanges() { NessieCatalog emptyTestCatalog = initCatalog("test"); updateSchema(testCatalog, tableIdentifier1); // Updating table with out of date hash. We expect this to succeed because of retry despite the conflict. @@ -122,21 +119,31 @@ private void updateSchema(NessieCatalog catalog, TableIdentifier identifier) { catalog.loadTable(identifier).updateSchema().addColumn("id" + schemaCounter++, Types.LongType.get()).commit(); } - private void testCatalogEquality(NessieCatalog catalog, - NessieCatalog compareCatalog, - boolean table1Equal, - boolean table2Equal) { + private void testCatalogEquality( + NessieCatalog catalog, NessieCatalog compareCatalog, boolean table1Equal, boolean table2Equal) { String testTable1 = metadataLocation(compareCatalog, tableIdentifier1); String table1 = metadataLocation(catalog, tableIdentifier1); String testTable2 = metadataLocation(compareCatalog, tableIdentifier2); String table2 = metadataLocation(catalog, tableIdentifier2); - String msg1 = String.format("Table %s on ref %s should%s equal table %s on ref %s", tableIdentifier1.name(), - tableIdentifier2.name(), table1Equal ? "" : " not", catalog.currentRefName(), testCatalog.currentRefName()); - Assert.assertEquals(msg1, table1Equal, table1.equals(testTable1)); - String msg2 = String.format("Table %s on ref %s should%s equal table %s on ref %s", tableIdentifier1.name(), - tableIdentifier2.name(), table1Equal ? "" : " not", catalog.currentRefName(), testCatalog.currentRefName()); - Assert.assertEquals(msg2, table2Equal, table2.equals(testTable2)); + Assertions.assertThat(table1.equals(testTable1)) + .withFailMessage(() -> String.format( + "Table %s on ref %s should%s equal table %s on ref %s", + tableIdentifier1.name(), + tableIdentifier2.name(), + table1Equal ? "" : " not", + catalog.currentRefName(), + testCatalog.currentRefName())) + .isEqualTo(table1Equal); + + Assertions.assertThat(table2.equals(testTable2)) + .withFailMessage(() -> String.format( + "Table %s on ref %s should%s equal table %s on ref %s", + tableIdentifier1.name(), + tableIdentifier2.name(), + table1Equal ? "" : " not", + catalog.currentRefName(), + testCatalog.currentRefName())) + .isEqualTo(table2Equal); } - } diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNamespace.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNamespace.java index f4b136dd2ee9..0b0637ed4965 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNamespace.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNamespace.java @@ -22,10 +22,9 @@ import java.util.List; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; +import org.assertj.core.api.Assertions; import org.junit.Test; -import static org.junit.Assert.assertEquals; - public class TestNamespace extends BaseTestIceberg { private static final String BRANCH = "test-namespace"; @@ -43,21 +42,21 @@ public void testListNamespaces() { createTable(TableIdentifier.parse("t6")); List tables = catalog.listTables(Namespace.of("a", "b", "c")); - assertEquals(1, tables.size()); + Assertions.assertThat(tables).isNotNull().hasSize(1); tables = catalog.listTables(Namespace.of("a", "b")); - assertEquals(2, tables.size()); + Assertions.assertThat(tables).isNotNull().hasSize(2); tables = catalog.listTables(Namespace.of("a")); - assertEquals(3, tables.size()); + Assertions.assertThat(tables).isNotNull().hasSize(3); tables = catalog.listTables(null); - assertEquals(6, tables.size()); + Assertions.assertThat(tables).isNotNull().hasSize(6); List namespaces = catalog.listNamespaces(); - assertEquals(5, namespaces.size()); + Assertions.assertThat(namespaces).isNotNull().hasSize(5); namespaces = catalog.listNamespaces(Namespace.of("a")); - assertEquals(3, namespaces.size()); + Assertions.assertThat(namespaces).isNotNull().hasSize(3); namespaces = catalog.listNamespaces(Namespace.of("a", "b")); - assertEquals(2, namespaces.size()); + Assertions.assertThat(namespaces).isNotNull().hasSize(2); namespaces = catalog.listNamespaces(Namespace.of("b")); - assertEquals(2, namespaces.size()); + Assertions.assertThat(namespaces).isNotNull().hasSize(2); } } diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java index 47178d4f61d5..1596361ca5ad 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java @@ -19,7 +19,6 @@ package org.apache.iceberg.nessie; - import java.io.File; import java.io.IOException; import java.nio.file.Paths; @@ -30,7 +29,6 @@ import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecordBuilder; import org.apache.hadoop.fs.Path; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.Files; @@ -45,8 +43,8 @@ import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; import org.junit.After; -import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.projectnessie.error.NessieConflictException; @@ -60,7 +58,6 @@ import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; - public class TestNessieTable extends BaseTestIceberg { private static final String BRANCH = "iceberg-table-test"; @@ -115,14 +112,12 @@ public void testCreate() throws NessieNotFoundException, IOException { icebergTable.updateSchema().addColumn("mother", Types.LongType.get()).commit(); IcebergTable table = getTable(KEY); // check parameters are in expected state - Assert.assertEquals(getTableLocation(tableName), - (temp.getRoot().toURI().toString() + DB_NAME + "/" + - tableName).replace("//", - "/")); + String expected = (temp.getRoot().toURI() + DB_NAME + "/" + tableName).replace("//", "/"); + Assertions.assertThat(getTableLocation(tableName)).isEqualTo(expected); // Only 1 snapshotFile Should exist and no manifests should exist - Assert.assertEquals(2, metadataVersionFiles(tableName).size()); - Assert.assertEquals(0, manifestFiles(tableName).size()); + Assertions.assertThat(metadataVersionFiles(tableName)).isNotNull().hasSize(2); + Assertions.assertThat(manifestFiles(tableName)).isNotNull().isEmpty(); verifyCommitMetadata(); } @@ -130,23 +125,24 @@ public void testCreate() throws NessieNotFoundException, IOException { @Test public void testRename() throws NessieNotFoundException { String renamedTableName = "rename_table_name"; - TableIdentifier renameTableIdentifier = TableIdentifier.of(TABLE_IDENTIFIER.namespace(), + TableIdentifier renameTableIdentifier = TableIdentifier.of( + TABLE_IDENTIFIER.namespace(), renamedTableName); Table original = catalog.loadTable(TABLE_IDENTIFIER); catalog.renameTable(TABLE_IDENTIFIER, renameTableIdentifier); - Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER)); - Assert.assertTrue(catalog.tableExists(renameTableIdentifier)); + Assertions.assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isFalse(); + Assertions.assertThat(catalog.tableExists(renameTableIdentifier)).isTrue(); Table renamed = catalog.loadTable(renameTableIdentifier); - Assert.assertEquals(original.schema().asStruct(), renamed.schema().asStruct()); - Assert.assertEquals(original.spec(), renamed.spec()); - Assert.assertEquals(original.location(), renamed.location()); - Assert.assertEquals(original.currentSnapshot(), renamed.currentSnapshot()); + Assertions.assertThat(original.schema().asStruct()).isEqualTo(renamed.schema().asStruct()); + Assertions.assertThat(original.spec()).isEqualTo(renamed.spec()); + Assertions.assertThat(original.location()).isEqualTo(renamed.location()); + Assertions.assertThat(original.currentSnapshot()).isEqualTo(renamed.currentSnapshot()); - Assert.assertTrue(catalog.dropTable(renameTableIdentifier)); + Assertions.assertThat(catalog.dropTable(renameTableIdentifier)).isTrue(); verifyCommitMetadata(); } @@ -154,19 +150,19 @@ public void testRename() throws NessieNotFoundException { private void verifyCommitMetadata() throws NessieNotFoundException { // check that the author is properly set List log = tree.getCommitLog(BRANCH, null, null).getOperations(); - Assert.assertFalse(log.isEmpty()); - log.forEach(x -> { - Assert.assertNotEquals("", x.getAuthor()); - Assert.assertEquals(System.getProperty("user.name"), x.getAuthor()); - Assert.assertEquals("iceberg", x.getProperties().get(NessieUtil.APPLICATION_TYPE)); + Assertions.assertThat(log).isNotNull().isNotEmpty(); + log.forEach(x -> { + Assertions.assertThat(x.getAuthor()).isNotNull().isNotEmpty(); + Assertions.assertThat(System.getProperty("user.name")).isEqualTo(x.getAuthor()); + Assertions.assertThat("iceberg").isEqualTo(x.getProperties().get(NessieUtil.APPLICATION_TYPE)); }); } @Test public void testDrop() throws NessieNotFoundException { - Assert.assertTrue(catalog.tableExists(TABLE_IDENTIFIER)); - Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER)); - Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER)); + Assertions.assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isTrue(); + Assertions.assertThat(catalog.dropTable(TABLE_IDENTIFIER)).isTrue(); + Assertions.assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isFalse(); verifyCommitMetadata(); } @@ -174,7 +170,6 @@ public void testDrop() throws NessieNotFoundException { public void testDropWithoutPurgeLeavesTableData() throws IOException { Table table = catalog.loadTable(TABLE_IDENTIFIER); - String fileLocation = addRecordsToFile(table, "file"); DataFile file = DataFiles.builder(table.spec()) @@ -188,11 +183,11 @@ public void testDropWithoutPurgeLeavesTableData() throws IOException { String manifestListLocation = table.currentSnapshot().manifestListLocation().replace("file:", ""); - Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER, false)); - Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER)); + Assertions.assertThat(catalog.dropTable(TABLE_IDENTIFIER, false)).isTrue(); + Assertions.assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isFalse(); - Assert.assertTrue(new File(fileLocation).exists()); - Assert.assertTrue(new File(manifestListLocation).exists()); + Assertions.assertThat(new File(fileLocation)).exists(); + Assertions.assertThat(new File(manifestListLocation)).exists(); } @Test @@ -232,21 +227,21 @@ public void testDropTable() throws IOException { List manifests = table.currentSnapshot().allManifests(); - Assert.assertTrue(catalog.dropTable(TABLE_IDENTIFIER)); - Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER)); + Assertions.assertThat(catalog.dropTable(TABLE_IDENTIFIER)).isTrue(); + Assertions.assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isFalse(); - Assert.assertTrue(new File(location1).exists()); - Assert.assertTrue(new File(location2).exists()); - Assert.assertTrue(new File(manifestListLocation).exists()); + Assertions.assertThat(new File(location1)).exists(); + Assertions.assertThat(new File(location2)).exists(); + Assertions.assertThat(new File(manifestListLocation)).exists(); for (ManifestFile manifest : manifests) { - Assert.assertTrue(new File(manifest.path().replace("file:", "")).exists()); + Assertions.assertThat(new File(manifest.path().replace("file:", ""))).exists(); } - Assert.assertTrue(new File( + Assertions.assertThat(new File( ((HasTableOperations) table).operations() .current() .metadataFileLocation() - .replace("file:", "")) - .exists()); + .replace("file:", ""))) + .exists(); verifyCommitMetadata(); } @@ -260,9 +255,9 @@ public void testExistingTableUpdate() { icebergTable = catalog.loadTable(TABLE_IDENTIFIER); // Only 2 snapshotFile Should exist and no manifests should exist - Assert.assertEquals(2, metadataVersionFiles(TABLE_NAME).size()); - Assert.assertEquals(0, manifestFiles(TABLE_NAME).size()); - Assert.assertEquals(altered.asStruct(), icebergTable.schema().asStruct()); + Assertions.assertThat(metadataVersionFiles(TABLE_NAME)).isNotNull().hasSize(2); + Assertions.assertThat(manifestFiles(TABLE_NAME)).isNotNull().isEmpty(); + Assertions.assertThat(altered.asStruct()).isEqualTo(icebergTable.schema().asStruct()); } @Test @@ -275,9 +270,10 @@ public void testFailure() throws NessieNotFoundException, NessieConflictExceptio client.getContentsApi().setContents(KEY, branch.getName(), branch.getHash(), "", IcebergTable.of("dummytable.metadata.json")); - AssertHelpers.assertThrows("Update schema fails with conflict exception, ref not up to date", - CommitFailedException.class, - () -> icebergTable.updateSchema().addColumn("data", Types.LongType.get()).commit()); + Assertions.assertThatThrownBy(() -> icebergTable.updateSchema().addColumn("data", Types.LongType.get()).commit()) + .isInstanceOf(CommitFailedException.class) + .hasMessage( + "Commit failed: Reference hash is out of date. Update the reference iceberg-table-test and try again"); } @Test @@ -290,8 +286,8 @@ public void testListTables() { t.name().equals(TABLE_NAME)) .collect(Collectors.toList()); - Assert.assertEquals(1, expectedIdents.size()); - Assert.assertTrue(catalog.tableExists(TABLE_IDENTIFIER)); + Assertions.assertThat(expectedIdents).hasSize(1); + Assertions.assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isTrue(); } private String getTableBasePath(String tableName) { diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestTableReference.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestTableReference.java index 5d0ab61134d6..96785fd64437 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestTableReference.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestTableReference.java @@ -19,65 +19,59 @@ package org.apache.iceberg.nessie; -import org.apache.iceberg.AssertHelpers; -import org.junit.Assert; +import org.assertj.core.api.Assertions; import org.junit.Test; public class TestTableReference { - @Test public void noMarkings() { String path = "foo"; TableReference pti = TableReference.parse(path); - Assert.assertEquals(path, pti.tableIdentifier().name()); - Assert.assertNull(pti.reference()); - Assert.assertNull(pti.timestamp()); + Assertions.assertThat(path).isEqualTo(pti.tableIdentifier().name()); + Assertions.assertThat(pti.reference()).isNull(); + Assertions.assertThat(pti.timestamp()).isNull(); } @Test public void branchOnly() { String path = "foo@bar"; TableReference pti = TableReference.parse(path); - Assert.assertEquals("foo", pti.tableIdentifier().name()); - Assert.assertEquals("bar", pti.reference()); - Assert.assertNull(pti.timestamp()); + Assertions.assertThat("foo").isEqualTo(pti.tableIdentifier().name()); + Assertions.assertThat("bar").isEqualTo(pti.reference()); + Assertions.assertThat(pti.timestamp()).isNull(); } @Test public void timestampOnly() { String path = "foo#baz"; - AssertHelpers.assertThrows("TableIdentifier is not parsable", - IllegalArgumentException.class, - "Invalid table name: # is not allowed (reference by timestamp is not supported)", () -> - TableReference.parse(path)); + Assertions.assertThatThrownBy(() -> TableReference.parse(path)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid table name: # is not allowed (reference by timestamp is not supported)"); } @Test public void branchAndTimestamp() { String path = "foo@bar#baz"; - AssertHelpers.assertThrows("TableIdentifier is not parsable", - IllegalArgumentException.class, - "Invalid table name: # is not allowed (reference by timestamp is not supported)", () -> - TableReference.parse(path)); + Assertions.assertThatThrownBy(() -> TableReference.parse(path)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid table name: # is not allowed (reference by timestamp is not supported)"); } @Test public void twoBranches() { String path = "foo@bar@boo"; - AssertHelpers.assertThrows("TableIdentifier is not parsable", - IllegalArgumentException.class, - "Can only reference one branch in", () -> - TableReference.parse(path)); + Assertions.assertThatThrownBy(() -> TableReference.parse(path)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Can only reference one branch in"); } @Test public void twoTimestamps() { String path = "foo#baz#baa"; - AssertHelpers.assertThrows("TableIdentifier is not parsable", - IllegalArgumentException.class, - "Can only reference one timestamp in", () -> - TableReference.parse(path)); + Assertions.assertThatThrownBy(() -> TableReference.parse(path)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Can only reference one timestamp in"); } @Test @@ -85,22 +79,22 @@ public void strangeCharacters() { String branch = "bar"; String path = "/%"; TableReference pti = TableReference.parse(path); - Assert.assertEquals(path, pti.tableIdentifier().name()); - Assert.assertNull(pti.reference()); - Assert.assertNull(pti.timestamp()); + Assertions.assertThat(path).isEqualTo(pti.tableIdentifier().name()); + Assertions.assertThat(pti.reference()).isNull(); + Assertions.assertThat(pti.timestamp()).isNull(); pti = TableReference.parse(path + "@" + branch); - Assert.assertEquals(path, pti.tableIdentifier().name()); - Assert.assertEquals(branch, pti.reference()); - Assert.assertNull(pti.timestamp()); + Assertions.assertThat(path).isEqualTo(pti.tableIdentifier().name()); + Assertions.assertThat(branch).isEqualTo(pti.reference()); + Assertions.assertThat(pti.timestamp()).isNull(); path = "&&"; pti = TableReference.parse(path); - Assert.assertEquals(path, pti.tableIdentifier().name()); - Assert.assertNull(pti.reference()); - Assert.assertNull(pti.timestamp()); + Assertions.assertThat(path).isEqualTo(pti.tableIdentifier().name()); + Assertions.assertThat(pti.reference()).isNull(); + Assertions.assertThat(pti.timestamp()).isNull(); pti = TableReference.parse(path + "@" + branch); - Assert.assertEquals(path, pti.tableIdentifier().name()); - Assert.assertEquals(branch, pti.reference()); - Assert.assertNull(pti.timestamp()); + Assertions.assertThat(path).isEqualTo(pti.tableIdentifier().name()); + Assertions.assertThat(branch).isEqualTo(pti.reference()); + Assertions.assertThat(pti.timestamp()).isNull(); } @Test @@ -108,22 +102,22 @@ public void doubleByte() { String branch = "bar"; String path = "/%国"; TableReference pti = TableReference.parse(path); - Assert.assertEquals(path, pti.tableIdentifier().name()); - Assert.assertNull(pti.reference()); - Assert.assertNull(pti.timestamp()); + Assertions.assertThat(path).isEqualTo(pti.tableIdentifier().name()); + Assertions.assertThat(pti.reference()).isNull(); + Assertions.assertThat(pti.timestamp()).isNull(); pti = TableReference.parse(path + "@" + branch); - Assert.assertEquals(path, pti.tableIdentifier().name()); - Assert.assertEquals(branch, pti.reference()); - Assert.assertNull(pti.timestamp()); + Assertions.assertThat(path).isEqualTo(pti.tableIdentifier().name()); + Assertions.assertThat(branch).isEqualTo(pti.reference()); + Assertions.assertThat(pti.timestamp()).isNull(); path = "国.国"; pti = TableReference.parse(path); - Assert.assertEquals(path, pti.tableIdentifier().toString()); - Assert.assertNull(pti.reference()); - Assert.assertNull(pti.timestamp()); + Assertions.assertThat(path).isEqualTo(pti.tableIdentifier().toString()); + Assertions.assertThat(pti.reference()).isNull(); + Assertions.assertThat(pti.timestamp()).isNull(); pti = TableReference.parse(path + "@" + branch); - Assert.assertEquals(path, pti.tableIdentifier().toString()); - Assert.assertEquals(branch, pti.reference()); - Assert.assertNull(pti.timestamp()); + Assertions.assertThat(path).isEqualTo(pti.tableIdentifier().toString()); + Assertions.assertThat(branch).isEqualTo(pti.reference()); + Assertions.assertThat(pti.timestamp()).isNull(); } @Test @@ -131,12 +125,12 @@ public void whitespace() { String branch = "bar "; String path = "foo "; TableReference pti = TableReference.parse(path); - Assert.assertEquals(path, pti.tableIdentifier().name()); - Assert.assertNull(pti.reference()); - Assert.assertNull(pti.timestamp()); + Assertions.assertThat(path).isEqualTo(pti.tableIdentifier().name()); + Assertions.assertThat(pti.reference()).isNull(); + Assertions.assertThat(pti.timestamp()).isNull(); pti = TableReference.parse(path + "@" + branch); - Assert.assertEquals(path, pti.tableIdentifier().name()); - Assert.assertEquals(branch, pti.reference()); - Assert.assertNull(pti.timestamp()); + Assertions.assertThat(path).isEqualTo(pti.tableIdentifier().name()); + Assertions.assertThat(branch).isEqualTo(pti.reference()); + Assertions.assertThat(pti.timestamp()).isNull(); } } diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcMetrics.java b/orc/src/main/java/org/apache/iceberg/orc/OrcMetrics.java index 67b608441399..0fad4dae139e 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/OrcMetrics.java +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcMetrics.java @@ -22,11 +22,14 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.sql.Timestamp; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.FieldMetrics; @@ -41,6 +44,7 @@ import org.apache.iceberg.hadoop.HadoopInputFile; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Conversions; @@ -92,7 +96,7 @@ static Metrics fromInputFile(InputFile file, Configuration config, MetricsConfig } } - static Metrics fromWriter(Writer writer, Stream fieldMetricsStream, MetricsConfig metricsConfig) { + static Metrics fromWriter(Writer writer, Stream> fieldMetricsStream, MetricsConfig metricsConfig) { try { return buildOrcMetrics(writer.getNumberOfRows(), writer.getSchema(), writer.getStatistics(), fieldMetricsStream, metricsConfig, null); @@ -103,7 +107,7 @@ static Metrics fromWriter(Writer writer, Stream fieldMetricsStream private static Metrics buildOrcMetrics(final long numOfRows, final TypeDescription orcSchema, final ColumnStatistics[] colStats, - final Stream fieldMetricsStream, + final Stream> fieldMetricsStream, final MetricsConfig metricsConfig, final NameMapping mapping) { final TypeDescription orcSchemaWithIds = (!ORCSchemaUtil.hasIds(orcSchema) && mapping != null) ? @@ -129,6 +133,10 @@ private static Metrics buildOrcMetrics(final long numOfRows, final TypeDescripti Map lowerBounds = Maps.newHashMap(); Map upperBounds = Maps.newHashMap(); + Map> fieldMetricsMap = Optional.ofNullable(fieldMetricsStream) + .map(stream -> stream.collect(Collectors.toMap(FieldMetrics::id, Function.identity()))) + .orElseGet(HashMap::new); + for (int i = 0; i < colStats.length; i++) { final ColumnStatistics colStat = colStats[i]; final TypeDescription orcCol = orcSchemaWithIds.findSubtype(i); @@ -160,10 +168,10 @@ private static Metrics buildOrcMetrics(final long numOfRows, final TypeDescripti if (metricsMode != MetricsModes.Counts.get()) { Optional orcMin = (colStat.getNumberOfValues() > 0) ? - fromOrcMin(icebergCol.type(), colStat, metricsMode) : Optional.empty(); + fromOrcMin(icebergCol.type(), colStat, metricsMode, fieldMetricsMap.get(fieldId)) : Optional.empty(); orcMin.ifPresent(byteBuffer -> lowerBounds.put(icebergCol.fieldId(), byteBuffer)); Optional orcMax = (colStat.getNumberOfValues() > 0) ? - fromOrcMax(icebergCol.type(), colStat, metricsMode) : Optional.empty(); + fromOrcMax(icebergCol.type(), colStat, metricsMode, fieldMetricsMap.get(fieldId)) : Optional.empty(); orcMax.ifPresent(byteBuffer -> upperBounds.put(icebergCol.fieldId(), byteBuffer)); } } @@ -174,13 +182,13 @@ private static Metrics buildOrcMetrics(final long numOfRows, final TypeDescripti columnSizes, valueCounts, nullCounts, - MetricsUtil.createNanValueCounts(fieldMetricsStream, effectiveMetricsConfig, schema), + MetricsUtil.createNanValueCounts(fieldMetricsMap.values().stream(), effectiveMetricsConfig, schema), lowerBounds, upperBounds); } private static Optional fromOrcMin(Type type, ColumnStatistics columnStats, - MetricsMode metricsMode) { + MetricsMode metricsMode, FieldMetrics fieldMetrics) { Object min = null; if (columnStats instanceof IntegerColumnStatistics) { min = ((IntegerColumnStatistics) columnStats).getMinimum(); @@ -188,10 +196,11 @@ private static Optional fromOrcMin(Type type, ColumnStatistics colum min = Math.toIntExact((long) min); } } else if (columnStats instanceof DoubleColumnStatistics) { - min = ((DoubleColumnStatistics) columnStats).getMinimum(); - if (type.typeId() == Type.TypeID.FLOAT) { - min = ((Double) min).floatValue(); - } + // since Orc includes NaN for upper/lower bounds of floating point columns, and we don't want this behavior, + // we have tracked metrics for such columns ourselves and thus do not need to rely on Orc's column statistics. + Preconditions.checkNotNull(fieldMetrics, + "[BUG] Float or double type columns should have metrics being tracked by Iceberg Orc writers"); + min = fieldMetrics.lowerBound(); } else if (columnStats instanceof StringColumnStatistics) { min = ((StringColumnStatistics) columnStats).getMinimum(); } else if (columnStats instanceof DecimalColumnStatistics) { @@ -217,7 +226,7 @@ private static Optional fromOrcMin(Type type, ColumnStatistics colum } private static Optional fromOrcMax(Type type, ColumnStatistics columnStats, - MetricsMode metricsMode) { + MetricsMode metricsMode, FieldMetrics fieldMetrics) { Object max = null; if (columnStats instanceof IntegerColumnStatistics) { max = ((IntegerColumnStatistics) columnStats).getMaximum(); @@ -225,10 +234,11 @@ private static Optional fromOrcMax(Type type, ColumnStatistics colum max = Math.toIntExact((long) max); } } else if (columnStats instanceof DoubleColumnStatistics) { - max = ((DoubleColumnStatistics) columnStats).getMaximum(); - if (type.typeId() == Type.TypeID.FLOAT) { - max = ((Double) max).floatValue(); - } + // since Orc includes NaN for upper/lower bounds of floating point columns, and we don't want this behavior, + // we have tracked metrics for such columns ourselves and thus do not need to rely on Orc's column statistics. + Preconditions.checkNotNull(fieldMetrics, + "[BUG] Float or double type columns should have metrics being tracked by Iceberg Orc writers"); + max = fieldMetrics.upperBound(); } else if (columnStats instanceof StringColumnStatistics) { max = ((StringColumnStatistics) columnStats).getMaximum(); } else if (columnStats instanceof DecimalColumnStatistics) { diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcRowWriter.java b/orc/src/main/java/org/apache/iceberg/orc/OrcRowWriter.java index 413634e3e100..1f0ea13dc685 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/OrcRowWriter.java +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcRowWriter.java @@ -41,7 +41,7 @@ public interface OrcRowWriter { /** * Returns a stream of {@link FieldMetrics} that this OrcRowWriter keeps track of. */ - default Stream metrics() { + default Stream> metrics() { return Stream.empty(); } } diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcValueReaders.java b/orc/src/main/java/org/apache/iceberg/orc/OrcValueReaders.java index 514e00977580..1be6889dbee1 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/OrcValueReaders.java +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcValueReaders.java @@ -154,6 +154,9 @@ protected StructReader(List> readers, Types.StructType struct, } else if (field.equals(MetadataColumns.ROW_POSITION)) { this.isConstantOrMetadataField[pos] = true; this.readers[pos] = new RowPositionReader(); + } else if (field.equals(MetadataColumns.IS_DELETED)) { + this.isConstantOrMetadataField[pos] = true; + this.readers[pos] = constants(false); } else { this.readers[pos] = readers.get(readerIndex); readerIndex++; diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcValueWriter.java b/orc/src/main/java/org/apache/iceberg/orc/OrcValueWriter.java index 2f72fc20e053..b6030abb7a78 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/OrcValueWriter.java +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcValueWriter.java @@ -38,6 +38,7 @@ default void write(int rowId, T data, ColumnVector output) { if (data == null) { output.noNulls = false; output.isNull[rowId] = true; + nullWrite(); } else { output.isNull[rowId] = false; nonNullWrite(rowId, data, output); @@ -46,10 +47,14 @@ default void write(int rowId, T data, ColumnVector output) { void nonNullWrite(int rowId, T data, ColumnVector output); + default void nullWrite() { + // no op + } + /** * Returns a stream of {@link FieldMetrics} that this OrcValueWriter keeps track of. */ - default Stream metrics() { + default Stream> metrics() { return Stream.empty(); } } diff --git a/parquet/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java b/parquet/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java index 710c771036d4..e0377050ba10 100644 --- a/parquet/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java +++ b/parquet/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java @@ -158,6 +158,9 @@ public ParquetValueReader struct(Types.StructType expected, GroupType struct, } else if (id == MetadataColumns.ROW_POSITION.fieldId()) { reorderedFields.add(ParquetValueReaders.position()); types.add(null); + } else if (id == MetadataColumns.IS_DELETED.fieldId()) { + reorderedFields.add(ParquetValueReaders.constant(false)); + types.add(null); } else { ParquetValueReader reader = readersById.get(id); if (reader != null) { diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/BasePageIterator.java b/parquet/src/main/java/org/apache/iceberg/parquet/BasePageIterator.java index cf79418ed866..7e5abfbc20d0 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/BasePageIterator.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/BasePageIterator.java @@ -77,7 +77,8 @@ protected void reset() { protected abstract void initDefinitionLevelsReader(DataPageV1 dataPageV1, ColumnDescriptor descriptor, ByteBufferInputStream in, int count) throws IOException; - protected abstract void initDefinitionLevelsReader(DataPageV2 dataPageV2, ColumnDescriptor descriptor); + protected abstract void initDefinitionLevelsReader(DataPageV2 dataPageV2, ColumnDescriptor descriptor) + throws IOException; public int currentPageCount() { return triplesCount; diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java index 099dbcfa0823..d57ef5af8312 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -170,6 +170,11 @@ public WriteBuilder overwrite(boolean enabled) { return this; } + public WriteBuilder writerVersion(WriterVersion version) { + this.writerVersion = version; + return this; + } + @SuppressWarnings("unchecked") private WriteSupport getWriteSupport(MessageType type) { if (writeSupport != null) { diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvro.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvro.java index 669269a2a99f..faa2957645e7 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvro.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvro.java @@ -20,8 +20,10 @@ package org.apache.iceberg.parquet; import java.math.BigDecimal; +import java.math.BigInteger; import java.util.List; import java.util.Map; +import java.util.WeakHashMap; import org.apache.avro.Conversion; import org.apache.avro.Conversions; import org.apache.avro.LogicalType; @@ -30,8 +32,10 @@ import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericFixed; import org.apache.avro.specific.SpecificData; +import org.apache.commons.math3.util.Pair; import org.apache.iceberg.avro.AvroSchemaVisitor; import org.apache.iceberg.avro.UUIDConversion; +import org.apache.iceberg.relocated.com.google.common.base.Objects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.TypeUtil; @@ -48,8 +52,8 @@ static Schema parquetAvroSchema(Schema avroSchema) { static class ParquetDecimal extends LogicalType { private static final String NAME = "parquet-decimal"; - private int precision; - private int scale; + private final int precision; + private final int scale; ParquetDecimal(int precision, int scale) { super(NAME); @@ -154,12 +158,10 @@ public Long toLong(BigDecimal value, org.apache.avro.Schema schema, LogicalType } private static class FixedDecimalConversion extends Conversions.DecimalConversion { - private final LogicalType[] decimalsByScale = new LogicalType[39]; + private final WeakHashMap, LogicalType> decimalsByScale; private FixedDecimalConversion() { - for (int i = 0; i < decimalsByScale.length; i += 1) { - decimalsByScale[i] = LogicalTypes.decimal(i, i); - } + this.decimalsByScale = new WeakHashMap<>(); } @Override @@ -169,12 +171,16 @@ public String getLogicalTypeName() { @Override public BigDecimal fromFixed(GenericFixed value, Schema schema, LogicalType type) { - return super.fromFixed(value, schema, decimalsByScale[((ParquetDecimal) type).scale()]); + ParquetDecimal dec = (ParquetDecimal) type; + return new BigDecimal(new BigInteger(value.bytes()), dec.scale()); } @Override public GenericFixed toFixed(BigDecimal value, Schema schema, LogicalType type) { - return super.toFixed(value, schema, decimalsByScale[((ParquetDecimal) type).scale()]); + ParquetDecimal dec = (ParquetDecimal) type; + Pair key = new Pair<>(dec.precision(), dec.scale()); + return super.toFixed(value, schema, + decimalsByScale.computeIfAbsent(key, k -> LogicalTypes.decimal(k.getFirst(), k.getSecond()))); } } @@ -252,7 +258,7 @@ public Schema record(Schema record, List names, List types) { newFields.add(copyField(field, type)); - if (field.schema() != type) { + if (!Objects.equal(field.schema(), type)) { hasChange = true; } } @@ -274,7 +280,7 @@ public Schema union(Schema union, List options) { @Override public Schema array(Schema array, Schema element) { - if (array.getElementType() != element) { + if (!Objects.equal(array.getElementType(), element)) { return Schema.createArray(element); } return array; @@ -282,7 +288,7 @@ public Schema array(Schema array, Schema element) { @Override public Schema map(Schema map, Schema value) { - if (map.getValueType() != value) { + if (!Objects.equal(map.getValueType(), value)) { return Schema.createMap(value); } return map; @@ -318,7 +324,7 @@ private boolean isIdentical(List types, List replacements) { int length = types.size(); for (int i = 0; i < length; i += 1) { - if (types.get(i) != replacements.get(i)) { + if (!Objects.equal(types.get(i), replacements.get(i))) { return false; } } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java index fb587b177a15..b7dae8a6a0bd 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java @@ -28,6 +28,8 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.iceberg.FieldMetrics; import org.apache.iceberg.Metrics; @@ -40,6 +42,7 @@ import org.apache.iceberg.expressions.Literal; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Conversions; @@ -80,13 +83,16 @@ public static Metrics fileMetrics(InputFile file, MetricsConfig metricsConfig, N } } - public static Metrics footerMetrics(ParquetMetadata metadata, Stream fieldMetrics, + public static Metrics footerMetrics(ParquetMetadata metadata, Stream> fieldMetrics, MetricsConfig metricsConfig) { return footerMetrics(metadata, fieldMetrics, metricsConfig, null); } - public static Metrics footerMetrics(ParquetMetadata metadata, Stream fieldMetrics, + @SuppressWarnings("checkstyle:CyclomaticComplexity") + public static Metrics footerMetrics(ParquetMetadata metadata, Stream> fieldMetrics, MetricsConfig metricsConfig, NameMapping nameMapping) { + Preconditions.checkNotNull(fieldMetrics, "fieldMetrics should not be null"); + long rowCount = 0; Map columnSizes = Maps.newHashMap(); Map valueCounts = Maps.newHashMap(); @@ -99,6 +105,9 @@ public static Metrics footerMetrics(ParquetMetadata metadata, Stream> fieldMetricsMap = fieldMetrics.collect( + Collectors.toMap(FieldMetrics::id, Function.identity())); + List blocks = metadata.getBlocks(); for (BlockMetaData block : blocks) { rowCount += block.getRowCount(); @@ -125,7 +134,9 @@ public static Metrics footerMetrics(ParquetMetadata metadata, Stream min = ParquetConversions.fromParquetPrimitive( @@ -147,9 +158,38 @@ public static Metrics footerMetrics(ParquetMetadata metadata, Stream> idToFieldMetricsMap, MetricsConfig metricsConfig, Schema schema, + Map> lowerBounds, Map> upperBounds) { + idToFieldMetricsMap.entrySet().forEach(entry -> { + int fieldId = entry.getKey(); + FieldMetrics metrics = entry.getValue(); + MetricsMode metricsMode = MetricsUtil.metricsMode(schema, metricsConfig, fieldId); + + // only check for MetricsModes.None, since we don't truncate float/double values. + if (metricsMode != MetricsModes.None.get()) { + if (!metrics.hasBounds()) { + lowerBounds.remove(fieldId); + upperBounds.remove(fieldId); + } else if (metrics.upperBound() instanceof Float) { + lowerBounds.put(fieldId, Literal.of((Float) metrics.lowerBound())); + upperBounds.put(fieldId, Literal.of((Float) metrics.upperBound())); + } else if (metrics.upperBound() instanceof Double) { + lowerBounds.put(fieldId, Literal.of((Double) metrics.lowerBound())); + upperBounds.put(fieldId, Literal.of((Double) metrics.upperBound())); + } else { + throw new UnsupportedOperationException("Expected only float or double column metrics"); + } + } + }); } private static MessageType getParquetTypeWithIds(ParquetMetadata metadata, NameMapping nameMapping) { diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriter.java index 7692ee58028d..fa9dcb7d0237 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriter.java @@ -34,7 +34,7 @@ public interface ParquetValueWriter { /** * Returns a stream of {@link FieldMetrics} that this ParquetValueWriter keeps track of. */ - default Stream metrics() { + default Stream> metrics() { return Stream.empty(); } } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java index 5413b3379387..1911d40467c6 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java @@ -28,8 +28,10 @@ import java.util.List; import java.util.Map; import java.util.function.Function; +import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.avro.util.Utf8; +import org.apache.iceberg.DoubleFieldMetrics; import org.apache.iceberg.FieldMetrics; import org.apache.iceberg.FloatFieldMetrics; import org.apache.iceberg.deletes.PositionDelete; @@ -170,50 +172,44 @@ public void writeDouble(int repetitionLevel, double value) { } private static class FloatWriter extends UnboxedWriter { - private final int id; - private long nanCount; + private final FloatFieldMetrics.Builder floatFieldMetricsBuilder; private FloatWriter(ColumnDescriptor desc) { super(desc); - this.id = desc.getPrimitiveType().getId().intValue(); - this.nanCount = 0; + int id = desc.getPrimitiveType().getId().intValue(); + this.floatFieldMetricsBuilder = new FloatFieldMetrics.Builder(id); } @Override public void write(int repetitionLevel, Float value) { writeFloat(repetitionLevel, value); - if (Float.isNaN(value)) { - nanCount++; - } + floatFieldMetricsBuilder.addValue(value); } @Override - public Stream metrics() { - return Stream.of(new FloatFieldMetrics(id, nanCount)); + public Stream> metrics() { + return Stream.of(floatFieldMetricsBuilder.build()); } } private static class DoubleWriter extends UnboxedWriter { - private final int id; - private long nanCount; + private final DoubleFieldMetrics.Builder doubleFieldMetricsBuilder; private DoubleWriter(ColumnDescriptor desc) { super(desc); - this.id = desc.getPrimitiveType().getId().intValue(); - this.nanCount = 0; + int id = desc.getPrimitiveType().getId().intValue(); + this.doubleFieldMetricsBuilder = new DoubleFieldMetrics.Builder(id); } @Override public void write(int repetitionLevel, Double value) { writeDouble(repetitionLevel, value); - if (Double.isNaN(value)) { - nanCount++; - } + doubleFieldMetricsBuilder.addValue(value); } @Override - public Stream metrics() { - return Stream.of(new FloatFieldMetrics(id, nanCount)); + public Stream> metrics() { + return Stream.of(doubleFieldMetricsBuilder.build()); } } @@ -332,6 +328,7 @@ static class OptionWriter implements ParquetValueWriter { private final int definitionLevel; private final ParquetValueWriter writer; private final List> children; + private long nullValueCount = 0; OptionWriter(int definitionLevel, ParquetValueWriter writer) { this.definitionLevel = definitionLevel; @@ -345,6 +342,7 @@ public void write(int repetitionLevel, T value) { writer.write(repetitionLevel, value); } else { + nullValueCount++; for (TripleWriter column : children) { column.writeNull(repetitionLevel, definitionLevel - 1); } @@ -362,7 +360,29 @@ public void setColumnStore(ColumnWriteStore columnStore) { } @Override - public Stream metrics() { + public Stream> metrics() { + if (writer instanceof PrimitiveWriter) { + List> fieldMetricsFromWriter = writer.metrics().collect(Collectors.toList()); + + if (fieldMetricsFromWriter.size() == 0) { + // we are not tracking field metrics for this type ourselves + return Stream.empty(); + } else if (fieldMetricsFromWriter.size() == 1) { + FieldMetrics metrics = fieldMetricsFromWriter.get(0); + return Stream.of( + new FieldMetrics<>(metrics.id(), + metrics.valueCount() + nullValueCount, nullValueCount, + metrics.nanValueCount(), metrics.lowerBound(), metrics.upperBound()) + ); + } else { + throw new IllegalStateException(String.format( + "OptionWriter should only expect at most one field metric from a primitive writer." + + "Current number of fields: %s, primitive writer type: %s", + fieldMetricsFromWriter.size(), writer.getClass().getSimpleName())); + } + } + + // skipping updating null stats for non-primitive types since we don't use them today, to avoid unnecessary work return writer.metrics(); } } @@ -421,7 +441,7 @@ public void setColumnStore(ColumnWriteStore columnStore) { protected abstract Iterator elements(L value); @Override - public Stream metrics() { + public Stream> metrics() { return writer.metrics(); } } @@ -499,7 +519,7 @@ public void setColumnStore(ColumnWriteStore columnStore) { protected abstract Iterator> pairs(M value); @Override - public Stream metrics() { + public Stream> metrics() { return Stream.concat(keyWriter.metrics(), valueWriter.metrics()); } } @@ -558,7 +578,7 @@ public void setColumnStore(ColumnWriteStore columnStore) { protected abstract Object get(S struct, int index); @Override - public Stream metrics() { + public Stream> metrics() { return Arrays.stream(writers).flatMap(ParquetValueWriter::metrics); } } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java b/parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java index 42598acf442e..acdda78c680b 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java @@ -21,6 +21,7 @@ import java.util.List; import java.util.Set; +import org.apache.iceberg.relocated.com.google.common.base.Objects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.parquet.schema.GroupType; @@ -102,7 +103,7 @@ public Type list(GroupType list, Type element) { if (elementId != null && selectedIds.contains(elementId)) { return list; } else if (element != null) { - if (element != originalElement) { + if (!Objects.equal(element, originalElement)) { Integer listId = getId(list); // the element type was projected Type listType = Types.list(list.getRepetition()) @@ -129,8 +130,8 @@ public Type map(GroupType map, Type key, Type value) { return map; } else if (value != null) { Integer mapId = getId(map); - if (value != originalValue) { - Type mapType = Types.map(map.getRepetition()) + if (!Objects.equal(value, originalValue)) { + Type mapType = Types.map(map.getRepetition()) .key(originalKey) .value(value) .named(map.getName()); diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java b/parquet/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java index 6dd76d722074..3a9b439cb300 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/TypeWithSchemaVisitor.java @@ -19,7 +19,7 @@ package org.apache.iceberg.parquet; -import java.util.LinkedList; +import java.util.ArrayDeque; import java.util.List; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -36,8 +36,8 @@ * @param the Java class returned by the visitor */ public class TypeWithSchemaVisitor { - @SuppressWarnings({"checkstyle:VisibilityModifier", "checkstyle:IllegalType"}) - protected LinkedList fieldNames = Lists.newLinkedList(); + @SuppressWarnings("checkstyle:VisibilityModifier") + protected ArrayDeque fieldNames = new ArrayDeque<>(); @SuppressWarnings("checkstyle:CyclomaticComplexity") public static T visit(org.apache.iceberg.types.Type iType, Type type, TypeWithSchemaVisitor visitor) { diff --git a/parquet/src/test/java/org/apache/iceberg/TestHelpers.java b/parquet/src/test/java/org/apache/iceberg/TestHelpers.java index 9b7b8decfef0..13fdb10d7c07 100644 --- a/parquet/src/test/java/org/apache/iceberg/TestHelpers.java +++ b/parquet/src/test/java/org/apache/iceberg/TestHelpers.java @@ -20,6 +20,8 @@ package org.apache.iceberg; import java.util.concurrent.Callable; +import org.apache.avro.AvroRuntimeException; +import org.apache.avro.generic.GenericRecord; import org.junit.Assert; public class TestHelpers { @@ -85,4 +87,17 @@ private static void handleException(String message, throw e; } } + + /** + * A convenience method to assert if an Avro field is empty + * @param record The record to read from + * @param field The name of the field + */ + public static void assertEmptyAvroField(GenericRecord record, String field) { + TestHelpers.assertThrows( + "Not a valid schema field: " + field, + AvroRuntimeException.class, + "Not a valid schema field: " + field, + () -> record.get(field)); + } } diff --git a/parquet/src/test/java/org/apache/iceberg/avro/TestReadProjection.java b/parquet/src/test/java/org/apache/iceberg/avro/TestReadProjection.java index 9243a809ff1e..685d4ee88f19 100644 --- a/parquet/src/test/java/org/apache/iceberg/avro/TestReadProjection.java +++ b/parquet/src/test/java/org/apache/iceberg/avro/TestReadProjection.java @@ -24,6 +24,7 @@ import java.util.Map; import org.apache.avro.generic.GenericData.Record; import org.apache.iceberg.Schema; +import org.apache.iceberg.TestHelpers; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -148,7 +149,7 @@ public void testBasicProjection() throws Exception { ); Record projected = writeAndRead("basic_projection_id", writeSchema, idOnly, record); - Assert.assertNull("Should not project data", projected.get("data")); + TestHelpers.assertEmptyAvroField(projected, "data"); Assert.assertEquals("Should contain the correct id value", 34L, (long) projected.get("id")); Schema dataOnly = new Schema( @@ -157,10 +158,10 @@ public void testBasicProjection() throws Exception { projected = writeAndRead("basic_projection_data", writeSchema, dataOnly, record); - Assert.assertNull("Should not project id", projected.get("id")); + TestHelpers.assertEmptyAvroField(projected, "id"); int cmp = Comparators.charSequences() .compare("test", (CharSequence) projected.get("data")); - Assert.assertTrue("Should contain the correct data value", cmp == 0); + Assert.assertEquals("Should contain the correct data value", 0, cmp); } @Test @@ -210,9 +211,8 @@ public void testNestedStructProjection() throws Exception { ); Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - Record projectedLocation = (Record) projected.get("location"); + TestHelpers.assertEmptyAvroField(projected, "location"); Assert.assertEquals("Should contain the correct id value", 34L, (long) projected.get("id")); - Assert.assertNull("Should not project location", projectedLocation); Schema latOnly = new Schema( Types.NestedField.optional(3, "location", Types.StructType.of( @@ -221,10 +221,10 @@ public void testNestedStructProjection() throws Exception { ); projected = writeAndRead("latitude_only", writeSchema, latOnly, record); - projectedLocation = (Record) projected.get("location"); - Assert.assertNull("Should not project id", projected.get("id")); + Record projectedLocation = (Record) projected.get("location"); + TestHelpers.assertEmptyAvroField(projected, "id"); Assert.assertNotNull("Should project location", projected.get("location")); - Assert.assertNull("Should not project longitude", projectedLocation.get("long")); + TestHelpers.assertEmptyAvroField(projectedLocation, "long"); Assert.assertEquals("Should project latitude", 52.995143f, (float) projectedLocation.get("lat"), 0.000001f); @@ -236,16 +236,16 @@ public void testNestedStructProjection() throws Exception { projected = writeAndRead("longitude_only", writeSchema, longOnly, record); projectedLocation = (Record) projected.get("location"); - Assert.assertNull("Should not project id", projected.get("id")); + TestHelpers.assertEmptyAvroField(projected, "id"); Assert.assertNotNull("Should project location", projected.get("location")); - Assert.assertNull("Should not project latitutde", projectedLocation.get("lat")); + TestHelpers.assertEmptyAvroField(projectedLocation, "lat"); Assert.assertEquals("Should project longitude", -1.539054f, (float) projectedLocation.get("long"), 0.000001f); Schema locationOnly = writeSchema.select("location"); projected = writeAndRead("location_only", writeSchema, locationOnly, record); projectedLocation = (Record) projected.get("location"); - Assert.assertNull("Should not project id", projected.get("id")); + TestHelpers.assertEmptyAvroField(projected, "id"); Assert.assertNotNull("Should project location", projected.get("location")); Assert.assertEquals("Should project latitude", 52.995143f, (float) projectedLocation.get("lat"), 0.000001f); @@ -273,23 +273,23 @@ public void testMapProjection() throws IOException { Record projected = writeAndRead("id_only", writeSchema, idOnly, record); Assert.assertEquals("Should contain the correct id value", 34L, (long) projected.get("id")); - Assert.assertNull("Should not project properties map", projected.get("properties")); + TestHelpers.assertEmptyAvroField(projected, "properties"); Schema keyOnly = writeSchema.select("properties.key"); projected = writeAndRead("key_only", writeSchema, keyOnly, record); - Assert.assertNull("Should not project id", projected.get("id")); + TestHelpers.assertEmptyAvroField(projected, "id"); Assert.assertEquals("Should project entire map", properties, toStringMap((Map) projected.get("properties"))); Schema valueOnly = writeSchema.select("properties.value"); projected = writeAndRead("value_only", writeSchema, valueOnly, record); - Assert.assertNull("Should not project id", projected.get("id")); + TestHelpers.assertEmptyAvroField(projected, "id"); Assert.assertEquals("Should project entire map", properties, toStringMap((Map) projected.get("properties"))); Schema mapOnly = writeSchema.select("properties"); projected = writeAndRead("map_only", writeSchema, mapOnly, record); - Assert.assertNull("Should not project id", projected.get("id")); + TestHelpers.assertEmptyAvroField(projected, "id"); Assert.assertEquals("Should project entire map", properties, toStringMap((Map) projected.get("properties"))); } @@ -337,16 +337,16 @@ public void testMapOfStructsProjection() throws IOException { Record projected = writeAndRead("id_only", writeSchema, idOnly, record); Assert.assertEquals("Should contain the correct id value", 34L, (long) projected.get("id")); - Assert.assertNull("Should not project locations map", projected.get("locations")); + TestHelpers.assertEmptyAvroField(projected, "locations"); projected = writeAndRead("all_locations", writeSchema, writeSchema.select("locations"), record); - Assert.assertNull("Should not project id", projected.get("id")); + TestHelpers.assertEmptyAvroField(projected, "id"); Assert.assertEquals("Should project locations map", record.get("locations"), toStringMap((Map) projected.get("locations"))); projected = writeAndRead("lat_only", writeSchema, writeSchema.select("locations.lat"), record); - Assert.assertNull("Should not project id", projected.get("id")); + TestHelpers.assertEmptyAvroField(projected, "id"); Map locations = toStringMap((Map) projected.get("locations")); Assert.assertNotNull("Should project locations map", locations); Assert.assertEquals("Should contain L1 and L2", @@ -355,28 +355,28 @@ public void testMapOfStructsProjection() throws IOException { Assert.assertNotNull("L1 should not be null", projectedL1); Assert.assertEquals("L1 should contain lat", 53.992811f, (float) projectedL1.get("lat"), 0.000001); - Assert.assertNull("L1 should not contain long", projectedL1.get("long")); + TestHelpers.assertEmptyAvroField(projectedL1, "long"); Record projectedL2 = (Record) locations.get("L2"); Assert.assertNotNull("L2 should not be null", projectedL2); Assert.assertEquals("L2 should contain lat", 52.995143f, (float) projectedL2.get("lat"), 0.000001); - Assert.assertNull("L2 should not contain long", projectedL2.get("long")); + TestHelpers.assertEmptyAvroField(projectedL2, "long"); projected = writeAndRead("long_only", writeSchema, writeSchema.select("locations.long"), record); - Assert.assertNull("Should not project id", projected.get("id")); + TestHelpers.assertEmptyAvroField(projected, "id"); locations = toStringMap((Map) projected.get("locations")); Assert.assertNotNull("Should project locations map", locations); Assert.assertEquals("Should contain L1 and L2", Sets.newHashSet("L1", "L2"), locations.keySet()); projectedL1 = (Record) locations.get("L1"); Assert.assertNotNull("L1 should not be null", projectedL1); - Assert.assertNull("L1 should not contain lat", projectedL1.get("lat")); + TestHelpers.assertEmptyAvroField(projectedL1, "lat"); Assert.assertEquals("L1 should contain long", -1.542616f, (float) projectedL1.get("long"), 0.000001); projectedL2 = (Record) locations.get("L2"); Assert.assertNotNull("L2 should not be null", projectedL2); - Assert.assertNull("L2 should not contain lat", projectedL2.get("lat")); + TestHelpers.assertEmptyAvroField(projectedL2, "lat"); Assert.assertEquals("L2 should contain long", -1.539054f, (float) projectedL2.get("long"), 0.000001); @@ -390,7 +390,7 @@ public void testMapOfStructsProjection() throws IOException { ); projected = writeAndRead("latitude_renamed", writeSchema, latitiudeRenamed, record); - Assert.assertNull("Should not project id", projected.get("id")); + TestHelpers.assertEmptyAvroField(projected, "id"); locations = toStringMap((Map) projected.get("locations")); Assert.assertNotNull("Should project locations map", locations); Assert.assertEquals("Should contain L1 and L2", @@ -399,14 +399,14 @@ public void testMapOfStructsProjection() throws IOException { Assert.assertNotNull("L1 should not be null", projectedL1); Assert.assertEquals("L1 should contain latitude", 53.992811f, (float) projectedL1.get("latitude"), 0.000001); - Assert.assertNull("L1 should not contain lat", projectedL1.get("lat")); - Assert.assertNull("L1 should not contain long", projectedL1.get("long")); + TestHelpers.assertEmptyAvroField(projectedL1, "lat"); + TestHelpers.assertEmptyAvroField(projectedL1, "long"); projectedL2 = (Record) locations.get("L2"); Assert.assertNotNull("L2 should not be null", projectedL2); Assert.assertEquals("L2 should contain latitude", 52.995143f, (float) projectedL2.get("latitude"), 0.000001); - Assert.assertNull("L2 should not contain lat", projectedL2.get("lat")); - Assert.assertNull("L2 should not contain long", projectedL2.get("long")); + TestHelpers.assertEmptyAvroField(projectedL2, "lat"); + TestHelpers.assertEmptyAvroField(projectedL2, "long"); } @Test @@ -429,16 +429,16 @@ public void testListProjection() throws IOException { Record projected = writeAndRead("id_only", writeSchema, idOnly, record); Assert.assertEquals("Should contain the correct id value", 34L, (long) projected.get("id")); - Assert.assertNull("Should not project values list", projected.get("values")); + TestHelpers.assertEmptyAvroField(projected, "values"); Schema elementOnly = writeSchema.select("values.element"); projected = writeAndRead("element_only", writeSchema, elementOnly, record); - Assert.assertNull("Should not project id", projected.get("id")); + TestHelpers.assertEmptyAvroField(projected, "id"); Assert.assertEquals("Should project entire list", values, projected.get("values")); Schema listOnly = writeSchema.select("values"); projected = writeAndRead("list_only", writeSchema, listOnly, record); - Assert.assertNull("Should not project id", projected.get("id")); + TestHelpers.assertEmptyAvroField(projected, "id"); Assert.assertEquals("Should project entire list", values, projected.get("values")); } @@ -473,36 +473,36 @@ public void testListOfStructsProjection() throws IOException { Record projected = writeAndRead("id_only", writeSchema, idOnly, record); Assert.assertEquals("Should contain the correct id value", 34L, (long) projected.get("id")); - Assert.assertNull("Should not project points list", projected.get("points")); + TestHelpers.assertEmptyAvroField(projected, "points"); projected = writeAndRead("all_points", writeSchema, writeSchema.select("points"), record); - Assert.assertNull("Should not project id", projected.get("id")); + TestHelpers.assertEmptyAvroField(projected, "id"); Assert.assertEquals("Should project points list", record.get("points"), projected.get("points")); projected = writeAndRead("x_only", writeSchema, writeSchema.select("points.x"), record); - Assert.assertNull("Should not project id", projected.get("id")); + TestHelpers.assertEmptyAvroField(projected, "id"); Assert.assertNotNull("Should project points list", projected.get("points")); List points = (List) projected.get("points"); Assert.assertEquals("Should read 2 points", 2, points.size()); Record projectedP1 = points.get(0); Assert.assertEquals("Should project x", 1, (int) projectedP1.get("x")); - Assert.assertNull("Should not project y", projectedP1.get("y")); + TestHelpers.assertEmptyAvroField(projectedP1, "y"); Record projectedP2 = points.get(1); Assert.assertEquals("Should project x", 3, (int) projectedP2.get("x")); - Assert.assertNull("Should not project y", projectedP2.get("y")); + TestHelpers.assertEmptyAvroField(projectedP2, "y"); projected = writeAndRead("y_only", writeSchema, writeSchema.select("points.y"), record); - Assert.assertNull("Should not project id", projected.get("id")); + TestHelpers.assertEmptyAvroField(projected, "id"); Assert.assertNotNull("Should project points list", projected.get("points")); points = (List) projected.get("points"); Assert.assertEquals("Should read 2 points", 2, points.size()); projectedP1 = points.get(0); - Assert.assertNull("Should not project x", projectedP1.get("x")); + TestHelpers.assertEmptyAvroField(projectedP1, "x"); Assert.assertEquals("Should project y", 2, (int) projectedP1.get("y")); projectedP2 = points.get(1); - Assert.assertNull("Should not project x", projectedP2.get("x")); - Assert.assertEquals("Should project null y", null, projectedP2.get("y")); + TestHelpers.assertEmptyAvroField(projectedP2, "x"); + Assert.assertNull("Should project null y", projectedP2.get("y")); Schema yRenamed = new Schema( Types.NestedField.optional(22, "points", @@ -513,17 +513,17 @@ public void testListOfStructsProjection() throws IOException { ); projected = writeAndRead("y_renamed", writeSchema, yRenamed, record); - Assert.assertNull("Should not project id", projected.get("id")); + TestHelpers.assertEmptyAvroField(projected, "id"); Assert.assertNotNull("Should project points list", projected.get("points")); points = (List) projected.get("points"); Assert.assertEquals("Should read 2 points", 2, points.size()); projectedP1 = points.get(0); - Assert.assertNull("Should not project x", projectedP1.get("x")); - Assert.assertNull("Should not project y", projectedP1.get("y")); + TestHelpers.assertEmptyAvroField(projectedP1, "x"); + TestHelpers.assertEmptyAvroField(projectedP1, "y"); Assert.assertEquals("Should project z", 2, (int) projectedP1.get("z")); projectedP2 = points.get(1); - Assert.assertNull("Should not project x", projectedP2.get("x")); - Assert.assertNull("Should not project y", projectedP2.get("y")); - Assert.assertEquals("Should project null z", null, projectedP2.get("z")); + TestHelpers.assertEmptyAvroField(projectedP2, "x"); + TestHelpers.assertEmptyAvroField(projectedP2, "y"); + Assert.assertNull("Should project null z", projectedP2.get("z")); } } diff --git a/python/iceberg/api/partition_spec.py b/python/iceberg/api/partition_spec.py index 395001b60162..e2f106d76b3b 100644 --- a/python/iceberg/api/partition_spec.py +++ b/python/iceberg/api/partition_spec.py @@ -21,7 +21,7 @@ from .partition_field import PartitionField from .schema import Schema -from .transforms import Transforms +from .transforms import Transform, Transforms from .types import (NestedField, StructType) @@ -202,6 +202,7 @@ def __init__(self, schema): self.schema = schema self.fields = list() self.partition_names = set() + self.dedup_fields = dict() self.spec_id = 0 self.last_assigned_field_id = PartitionSpec.PARTITION_DATA_ID_START - 1 @@ -213,15 +214,35 @@ def with_spec_id(self, spec_id): self.spec_id = spec_id return self - def check_and_add_partition_name(self, name): + def check_and_add_partition_name(self, name, source_column_id=None): + schema_field = self.schema.find_field(name) + if source_column_id is not None: + if schema_field is not None and schema_field.field_id != source_column_id: + raise ValueError("Cannot create identity partition sourced from different field in schema: %s" % name) + else: + if schema_field is not None: + raise ValueError("Cannot create partition from name that exists in schema: %s" % name) + if name is None or name == "": - raise RuntimeError("Cannot use empty or null partition name") + raise ValueError("Cannot use empty or null partition name: %s" % name) if name in self.partition_names: - raise RuntimeError("Cannot use partition names more than once: %s" % name) + raise ValueError("Cannot use partition names more than once: %s" % name) self.partition_names.add(name) return self + def check_redundant_and_add_field(self, field_id: int, name: str, transform: Transform) -> None: + field = PartitionField(field_id, + self.__next_field_id(), + name, + transform) + dedup_key = (field.source_id, field.transform.dedup_name()) + partition_field = self.dedup_fields.get(dedup_key) + if partition_field is not None: + raise ValueError("Cannot add redundant partition: %s conflicts with %s" % (partition_field, field)) + self.dedup_fields[dedup_key] = field + self.fields.append(field) + def find_source_column(self, source_name): source_column = self.schema.find_field(source_name) if source_column is None: @@ -229,72 +250,82 @@ def find_source_column(self, source_name): return source_column - def identity(self, source_name): - self.check_and_add_partition_name(source_name) + def identity(self, source_name, target_name=None): + if target_name is None: + target_name = source_name + source_column = self.find_source_column(source_name) - self.fields.append(PartitionField(source_column.field_id, - self.__next_field_id(), - source_name, - Transforms.identity(source_column.type))) + self.check_and_add_partition_name(target_name, source_column.field_id) + self.check_redundant_and_add_field(source_column.field_id, + target_name, + Transforms.identity(source_column.type)) return self - def year(self, source_name): - name = "{}_year".format(source_name) - self.check_and_add_partition_name(name) + def year(self, source_name, target_name=None): + if target_name is None: + target_name = "{}_year".format(source_name) + + self.check_and_add_partition_name(target_name) source_column = self.find_source_column(source_name) - self.fields.append(PartitionField(source_column.field_id, - self.__next_field_id(), - name, - Transforms.year(source_column.type))) + self.check_redundant_and_add_field(source_column.field_id, + target_name, + Transforms.year(source_column.type)) return self - def month(self, source_name): - name = "{}_month".format(source_name) - self.check_and_add_partition_name(name) + def month(self, source_name, target_name=None): + if target_name is None: + target_name = "{}_month".format(source_name) + + self.check_and_add_partition_name(target_name) source_column = self.find_source_column(source_name) - self.fields.append(PartitionField(source_column.field_id, - self.__next_field_id(), - name, - Transforms.month(source_column.type))) + self.check_redundant_and_add_field(source_column.field_id, + target_name, + Transforms.month(source_column.type)) return self - def day(self, source_name): - name = "{}_day".format(source_name) - self.check_and_add_partition_name(name) + def day(self, source_name, target_name=None): + if target_name is None: + target_name = "{}_day".format(source_name) + + self.check_and_add_partition_name(target_name) source_column = self.find_source_column(source_name) - self.fields.append(PartitionField(source_column.field_id, - self.__next_field_id(), - name, - Transforms.day(source_column.type))) + self.check_redundant_and_add_field(source_column.field_id, + target_name, + Transforms.day(source_column.type)) return self - def hour(self, source_name): - name = "{}_hour".format(source_name) - self.check_and_add_partition_name(name) + def hour(self, source_name, target_name=None): + if target_name is None: + target_name = "{}_hour".format(source_name) + + self.check_and_add_partition_name(target_name) source_column = self.find_source_column(source_name) - self.fields.append(PartitionField(source_column.field_id, - self.__next_field_id(), - name, - Transforms.hour(source_column.type))) + self.check_redundant_and_add_field(source_column.field_id, + target_name, + Transforms.hour(source_column.type)) return self - def bucket(self, source_name, num_buckets): - name = "{}_bucket".format(source_name) - self.check_and_add_partition_name(name) + def bucket(self, source_name, num_buckets, target_name=None): + if target_name is None: + target_name = "{}_bucket".format(source_name) + + self.check_and_add_partition_name(target_name) source_column = self.find_source_column(source_name) self.fields.append(PartitionField(source_column.field_id, self.__next_field_id(), - name, + target_name, Transforms.bucket(source_column.type, num_buckets))) return self - def truncate(self, source_name, width): - name = "{}_truncate".format(source_name) - self.check_and_add_partition_name(name) + def truncate(self, source_name, width, target_name=None): + if target_name is None: + target_name = "{}_truncate".format(source_name) + + self.check_and_add_partition_name(target_name) source_column = self.find_source_column(source_name) self.fields.append(PartitionField(source_column.field_id, self.__next_field_id(), - name, + target_name, Transforms.truncate(source_column.type, width))) return self @@ -302,17 +333,16 @@ def add_without_field_id(self, source_id, name, transform): return self.add(source_id, self.__next_field_id(), name, transform) def add(self, source_id: int, field_id: int, name: str, transform: str) -> "PartitionSpecBuilder": - self.check_and_add_partition_name(name) column = self.schema.find_field(source_id) if column is None: - raise RuntimeError("Cannot find source column: %s" % source_id) + raise ValueError("Cannot find source column: %s" % source_id) transform_obj = Transforms.from_string(column.type, transform) - field = PartitionField(source_id, - field_id, - name, - transform_obj) - self.fields.append(field) + self.check_and_add_partition_name(name, source_id) + self.fields.append(PartitionField(source_id, + field_id, + name, + transform_obj)) self.last_assigned_field_id = max(self.last_assigned_field_id, field_id) return self diff --git a/python/iceberg/api/transforms/dates.py b/python/iceberg/api/transforms/dates.py index 474b986f696d..dfc6b9a2cd02 100644 --- a/python/iceberg/api/transforms/dates.py +++ b/python/iceberg/api/transforms/dates.py @@ -74,3 +74,14 @@ def to_human_string(self, value): def __str__(self): return self.name + + def dedup_name(self): + return "time" + + def __eq__(self, other): + if id(self) == id(other): + return True + if other is None or not isinstance(other, Dates): + return False + + return self.granularity == other.granularity and self.name == other.name diff --git a/python/iceberg/api/transforms/timestamps.py b/python/iceberg/api/transforms/timestamps.py index 25c4439bc179..ca38a1c3bedc 100644 --- a/python/iceberg/api/transforms/timestamps.py +++ b/python/iceberg/api/transforms/timestamps.py @@ -70,3 +70,14 @@ def to_human_string(self, value): def __str__(self): return self.name + + def dedup_name(self): + return "time" + + def __eq__(self, other): + if id(self) == id(other): + return True + if other is None or not isinstance(other, Timestamps): + return False + + return self.granularity == other.granularity and self.name == other.name diff --git a/python/iceberg/api/transforms/transform.py b/python/iceberg/api/transforms/transform.py index 776b0a507b3a..0ccbf2e3c25e 100644 --- a/python/iceberg/api/transforms/transform.py +++ b/python/iceberg/api/transforms/transform.py @@ -38,3 +38,6 @@ def project_strict(self, name, predicate): def to_human_string(self, value): return str(value) + + def dedup_name(self): + return self.__str__() diff --git a/python/iceberg/api/transforms/transforms.py b/python/iceberg/api/transforms/transforms.py index 0cf243e0f912..205a9de7376d 100644 --- a/python/iceberg/api/transforms/transforms.py +++ b/python/iceberg/api/transforms/transforms.py @@ -22,6 +22,8 @@ from .identity import Identity from .timestamps import Timestamps from .truncate import Truncate +from .unknown_transform import UnknownTransform +from .void_transform import VoidTransform from ..types import (TypeID) @@ -60,7 +62,10 @@ def from_string(type_var, transform): elif type_var.type_id == TypeID.DATE: return Dates(transform.lower(), transform.lower()) - raise RuntimeError("Unknown transform: %s" % transform) + if transform.lower() == "void": + return VoidTransform.get() + + return UnknownTransform(type_var, transform) @staticmethod def identity(type_var): @@ -109,3 +114,7 @@ def bucket(type_var, num_buckets): @staticmethod def truncate(type_var, width): return Truncate.get(type_var, width) + + @staticmethod + def always_null(): + return VoidTransform.get() diff --git a/python/iceberg/api/transforms/unknown_transform.py b/python/iceberg/api/transforms/unknown_transform.py new file mode 100644 index 000000000000..de326445de99 --- /dev/null +++ b/python/iceberg/api/transforms/unknown_transform.py @@ -0,0 +1,61 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from typing import Union + +from iceberg.api.types import StringType, Type + +from .transform import Transform + + +class UnknownTransform(Transform): + + def __init__(self, source_type: Type, transform: str): + self.source_type = source_type + self.transform = transform + + def apply(self, value): + raise AttributeError(f"Cannot apply unsupported transform: {self.transform}") + + def can_transform(self, type_var) -> bool: + # assume the transform function can be applied for this type because unknown transform is only used when parsing + # a transform in an existing table. a different Iceberg version must have already validated it. + return self.source_type == type_var + + def get_result_type(self, source_type): + # the actual result type is not known + return StringType.get() + + def project(self, name, predicate): + return None + + def project_strict(self, name, predicate): + return None + + def __str__(self): + return self.transform + + def __eq__(self, other: Union['UnknownTransform', Transform, object]): + if id(self) == id(other): + return True + elif not isinstance(other, UnknownTransform): + return False + + return self.source_type == other.source_type and self.transform == other.transform + + def __hash__(self): + return hash((self.source_type, self.transform)) diff --git a/python/iceberg/api/transforms/void_transform.py b/python/iceberg/api/transforms/void_transform.py new file mode 100644 index 000000000000..ea859641ade9 --- /dev/null +++ b/python/iceberg/api/transforms/void_transform.py @@ -0,0 +1,52 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from .transform import Transform + + +class VoidTransform(Transform): + _INSTANCE = None + + @staticmethod + def get(): + if VoidTransform._INSTANCE is None: + VoidTransform._INSTANCE = VoidTransform() + return VoidTransform._INSTANCE + + def __init__(self): + pass + + def apply(self, value): + return None + + def can_transform(self, type_var): + return True + + def get_result_type(self, source_type): + return source_type + + def project(self, name, predicate): + return None + + def project_strict(self, name, predicate): + return None + + def to_human_string(self, value): + return "null" + + def __str__(self): + return "void" diff --git a/python/tests/api/test_partition_spec.py b/python/tests/api/test_partition_spec.py new file mode 100644 index 000000000000..93cee149600c --- /dev/null +++ b/python/tests/api/test_partition_spec.py @@ -0,0 +1,84 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +import unittest + +from iceberg.api import PartitionSpec +from iceberg.api.schema import Schema +from iceberg.api.types import (BinaryType, + DateType, + DecimalType, + FixedType, + IntegerType, + LongType, + NestedField, + StringType, + TimestampType, + TimeType, + UUIDType) +from tests.api.test_helpers import TestHelpers + + +class TestConversions(unittest.TestCase): + + def test_transforms(self): + schema = Schema(NestedField.required(1, "i", IntegerType.get()), + NestedField.required(2, "l", LongType.get()), + NestedField.required(3, "d", DateType.get()), + NestedField.required(4, "t", TimeType.get()), + NestedField.required(5, "ts", TimestampType.without_timezone()), + NestedField.required(6, "dec", DecimalType.of(9, 2)), + NestedField.required(7, "s", StringType.get()), + NestedField.required(8, "u", UUIDType.get()), + NestedField.required(9, "f", FixedType.of_length(3)), + NestedField.required(10, "b", BinaryType.get())) + specs = [PartitionSpec.builder_for(schema).identity("i").build(), + PartitionSpec.builder_for(schema).identity("l").build(), + PartitionSpec.builder_for(schema).identity("d").build(), + PartitionSpec.builder_for(schema).identity("t").build(), + PartitionSpec.builder_for(schema).identity("ts").build(), + PartitionSpec.builder_for(schema).identity("dec").build(), + PartitionSpec.builder_for(schema).identity("s").build(), + PartitionSpec.builder_for(schema).identity("u").build(), + PartitionSpec.builder_for(schema).identity("f").build(), + PartitionSpec.builder_for(schema).identity("b").build(), + PartitionSpec.builder_for(schema).bucket("i", 128).build(), + PartitionSpec.builder_for(schema).bucket("l", 128).build(), + PartitionSpec.builder_for(schema).bucket("d", 128).build(), + PartitionSpec.builder_for(schema).bucket("t", 128).build(), + PartitionSpec.builder_for(schema).bucket("ts", 128).build(), + PartitionSpec.builder_for(schema).bucket("dec", 128).build(), + PartitionSpec.builder_for(schema).bucket("s", 128).build(), + # todo support them + # PartitionSpec.builder_for(schema).bucket("u", 128).build(), + # PartitionSpec.builder_for(schema).bucket("f", 128).build(), + # PartitionSpec.builder_for(schema).bucket("b", 128).build(), + PartitionSpec.builder_for(schema).year("d").build(), + PartitionSpec.builder_for(schema).month("d").build(), + PartitionSpec.builder_for(schema).day("d").build(), + PartitionSpec.builder_for(schema).year("ts").build(), + PartitionSpec.builder_for(schema).month("ts").build(), + PartitionSpec.builder_for(schema).day("ts").build(), + PartitionSpec.builder_for(schema).hour("ts").build(), + PartitionSpec.builder_for(schema).truncate("i", 10).build(), + PartitionSpec.builder_for(schema).truncate("l", 10).build(), + PartitionSpec.builder_for(schema).truncate("dec", 10).build(), + PartitionSpec.builder_for(schema).truncate("s", 10).build(), + # todo support them + # PartitionSpec.builder_for(schema).add_without_field_id(6, "dec_unsupported", "unsupported").build(), + # PartitionSpec.builder_for(schema).add(6, 1111, "dec_unsupported", "unsupported").build(), + ] + + for spec in specs: + self.assertEqual(spec, TestHelpers.round_trip_serialize(spec)) diff --git a/python/tox.ini b/python/tox.ini index dcc30c2d48c8..7de9703cb73f 100644 --- a/python/tox.ini +++ b/python/tox.ini @@ -60,6 +60,8 @@ basepython = python3 skip_install = true deps = mypy + types-pytz + types-python-dateutil commands = mypy --ignore-missing-imports iceberg/ diff --git a/site/docs/aws.md b/site/docs/aws.md index ca91ddda4383..96207a72469d 100644 --- a/site/docs/aws.md +++ b/site/docs/aws.md @@ -163,7 +163,7 @@ an Iceberg table is stored as a [Glue Table](https://docs.aws.amazon.com/glue/la and every Iceberg table version is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`, just like what is shown in the [enabling AWS integration](#enabling-aws-integration) section above. -More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs). +More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark-configuration/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs). ### Glue Catalog ID There is a unique Glue metastore in each AWS account and each AWS region. diff --git a/site/docs/benchmarks.md b/site/docs/benchmarks.md new file mode 100644 index 000000000000..40ee9726337d --- /dev/null +++ b/site/docs/benchmarks.md @@ -0,0 +1,114 @@ + + +## Available Benchmarks and how to run them + +Benchmarks are located under `/jmh`. It is generally favorable to only run the tests of interest rather than running all available benchmarks. +Also note that JMH benchmarks run within the same JVM as the system-under-test, so results might vary between runs. + + +### IcebergSourceNestedListParquetDataWriteBenchmark +A benchmark that evaluates the performance of writing nested Parquet data using Iceberg and the built-in file source in Spark. To run this benchmark for either spark-2 or spark-3: + +`./gradlew :iceberg-spark[2|3]:jmh -PjmhIncludeRegex=IcebergSourceNestedListParquetDataWriteBenchmark -PjmhOutputPath=benchmark/iceberg-source-nested-list-parquet-data-write-benchmark-result.txt` + +### SparkParquetReadersNestedDataBenchmark +A benchmark that evaluates the performance of reading nested Parquet data using Iceberg and Spark Parquet readers. To run this benchmark for either spark-2 or spark-3: + +`./gradlew :iceberg-spark[2|3]:jmh -PjmhIncludeRegex=SparkParquetReadersNestedDataBenchmark -PjmhOutputPath=benchmark/spark-parquet-readers-nested-data-benchmark-result.txt` + +### SparkParquetWritersFlatDataBenchmark +A benchmark that evaluates the performance of writing Parquet data with a flat schema using Iceberg and Spark Parquet writers. To run this benchmark for either spark-2 or spark-3: + +`./gradlew :iceberg-spark[2|3]:jmh -PjmhIncludeRegex=SparkParquetWritersFlatDataBenchmark -PjmhOutputPath=benchmark/spark-parquet-writers-flat-data-benchmark-result.txt` + +### IcebergSourceFlatORCDataReadBenchmark +A benchmark that evaluates the performance of reading ORC data with a flat schema using Iceberg and the built-in file source in Spark. To run this benchmark for either spark-2 or spark-3: + +`./gradlew :iceberg-spark[2|3]:jmh -PjmhIncludeRegex=IcebergSourceFlatORCDataReadBenchmark -PjmhOutputPath=benchmark/iceberg-source-flat-orc-data-read-benchmark-result.txt` + +### SparkParquetReadersFlatDataBenchmark +A benchmark that evaluates the performance of reading Parquet data with a flat schema using Iceberg and Spark Parquet readers. To run this benchmark for either spark-2 or spark-3: + +`./gradlew :iceberg-spark[2|3]:jmh -PjmhIncludeRegex=SparkParquetReadersFlatDataBenchmark -PjmhOutputPath=benchmark/spark-parquet-readers-flat-data-benchmark-result.txt` + +### VectorizedReadDictionaryEncodedFlatParquetDataBenchmark +A benchmark to compare performance of reading Parquet dictionary encoded data with a flat schema using vectorized Iceberg read path and the built-in file source in Spark. To run this benchmark for either spark-2 or spark-3: + +`./gradlew :iceberg-spark[2|3]:jmh -PjmhIncludeRegex=VectorizedReadDictionaryEncodedFlatParquetDataBenchmark -PjmhOutputPath=benchmark/vectorized-read-dict-encoded-flat-parquet-data-result.txt` + +### IcebergSourceNestedListORCDataWriteBenchmark +A benchmark that evaluates the performance of writing nested Parquet data using Iceberg and the built-in file source in Spark. To run this benchmark for either spark-2 or spark-3: + +`./gradlew :iceberg-spark[2|3]:jmh -PjmhIncludeRegex=IcebergSourceNestedListORCDataWriteBenchmark -PjmhOutputPath=benchmark/iceberg-source-nested-list-orc-data-write-benchmark-result.txt` + +### VectorizedReadFlatParquetDataBenchmark +A benchmark to compare performance of reading Parquet data with a flat schema using vectorized Iceberg read path and the built-in file source in Spark. To run this benchmark for either spark-2 or spark-3: + +`./gradlew :iceberg-spark[2|3]:jmh -PjmhIncludeRegex=VectorizedReadFlatParquetDataBenchmark -PjmhOutputPath=benchmark/vectorized-read-flat-parquet-data-result.txt` + +### IcebergSourceFlatParquetDataWriteBenchmark +A benchmark that evaluates the performance of writing Parquet data with a flat schema using Iceberg and the built-in file source in Spark. To run this benchmark for either spark-2 or spark-3: + +`./gradlew :iceberg-spark[2|3]:jmh -PjmhIncludeRegex=IcebergSourceFlatParquetDataWriteBenchmark -PjmhOutputPath=benchmark/iceberg-source-flat-parquet-data-write-benchmark-result.txt` + +### IcebergSourceNestedAvroDataReadBenchmark +A benchmark that evaluates the performance of reading Avro data with a flat schema using Iceberg and the built-in file source in Spark. To run this benchmark for either spark-2 or spark-3: + +`./gradlew :iceberg-spark[2|3]:jmh -PjmhIncludeRegex=IcebergSourceNestedAvroDataReadBenchmark -PjmhOutputPath=benchmark/iceberg-source-nested-avro-data-read-benchmark-result.txt` + +### IcebergSourceFlatAvroDataReadBenchmark +A benchmark that evaluates the performance of reading Avro data with a flat schema using Iceberg and the built-in file source in Spark. To run this benchmark for either spark-2 or spark-3: + +`./gradlew :iceberg-spark[2|3]:jmh -PjmhIncludeRegex=IcebergSourceFlatAvroDataReadBenchmark -PjmhOutputPath=benchmark/iceberg-source-flat-avro-data-read-benchmark-result.txt` + +### IcebergSourceNestedParquetDataWriteBenchmark +A benchmark that evaluates the performance of writing nested Parquet data using Iceberg and the built-in file source in Spark. To run this benchmark for either spark-2 or spark-3: + +`./gradlew :iceberg-spark[2|3]:jmh -PjmhIncludeRegex=IcebergSourceNestedParquetDataWriteBenchmark -PjmhOutputPath=benchmark/iceberg-source-nested-parquet-data-write-benchmark-result.txt` + +### IcebergSourceNestedParquetDataReadBenchmark +* A benchmark that evaluates the performance of reading nested Parquet data using Iceberg and the built-in file source in Spark. To run this benchmark for either spark-2 or spark-3: + +` ./gradlew :iceberg-spark[2|3]:jmh -PjmhIncludeRegex=IcebergSourceNestedParquetDataReadBenchmark -PjmhOutputPath=benchmark/iceberg-source-nested-parquet-data-read-benchmark-result.txt` + +### IcebergSourceNestedORCDataReadBenchmark +A benchmark that evaluates the performance of reading ORC data with a flat schema using Iceberg and the built-in file source in Spark. To run this benchmark for either spark-2 or spark-3: + +`./gradlew :iceberg-spark[2|3]:jmh -PjmhIncludeRegex=IcebergSourceNestedORCDataReadBenchmark -PjmhOutputPath=benchmark/iceberg-source-nested-orc-data-read-benchmark-result.txt` + +### IcebergSourceFlatParquetDataReadBenchmark +A benchmark that evaluates the performance of reading Parquet data with a flat schema using Iceberg and the built-in file source in Spark. To run this benchmark for either spark-2 or spark-3: + +`./gradlew :iceberg-spark[2|3]:jmh -PjmhIncludeRegex=IcebergSourceFlatParquetDataReadBenchmark -PjmhOutputPath=benchmark/iceberg-source-flat-parquet-data-read-benchmark-result.txt` + +### IcebergSourceFlatParquetDataFilterBenchmark +A benchmark that evaluates the file skipping capabilities in the Spark data source for Iceberg. This class uses a dataset with a flat schema, where the records are clustered according to the +column used in the filter predicate. The performance is compared to the built-in file source in Spark. To run this benchmark for either spark-2 or spark-3: + +`./gradlew :iceberg-spark[2|3]:jmh -PjmhIncludeRegex=IcebergSourceFlatParquetDataFilterBenchmark -PjmhOutputPath=benchmark/iceberg-source-flat-parquet-data-filter-benchmark-result.txt` + +### IcebergSourceNestedParquetDataFilterBenchmark +A benchmark that evaluates the file skipping capabilities in the Spark data source for Iceberg. This class uses a dataset with nested data, where the records are clustered according to the +column used in the filter predicate. The performance is compared to the built-in file source in Spark. To run this benchmark for either spark-2 or spark-3: +`./gradlew :iceberg-spark[2|3]:jmh -PjmhIncludeRegex=IcebergSourceNestedParquetDataFilterBenchmark -PjmhOutputPath=benchmark/iceberg-source-nested-parquet-data-filter-benchmark-result.txt` + +### SparkParquetWritersNestedDataBenchmark +* A benchmark that evaluates the performance of writing nested Parquet data using Iceberg and Spark Parquet writers. To run this benchmark for either spark-2 or spark-3: + `./gradlew :iceberg-spark[2|3]:jmh -PjmhIncludeRegex=SparkParquetWritersNestedDataBenchmark -PjmhOutputPath=benchmark/spark-parquet-writers-nested-data-benchmark-result.txt` diff --git a/site/docs/blogs.md b/site/docs/blogs.md index 13a4cfb6466e..760301de4541 100644 --- a/site/docs/blogs.md +++ b/site/docs/blogs.md @@ -19,6 +19,11 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered from most recent to oldest. +### [Migrating to Apache Iceberg at Adobe Experience Platform](https://medium.com/adobetech/migrating-to-apache-iceberg-at-adobe-experience-platform-40fa80f8b8de) +**Date**: Jun 17th, 2021, **Company**: Adobe + +**Author**: [Romin Parekh](https://www.linkedin.com/in/rominparekh/), [Miao Wang](https://www.linkedin.com/in/miao-wang-0406a74/), [Shone Sadler](https://www.linkedin.com/in/shonesadler/) + ### [A Short Introduction to Apache Iceberg](https://medium.com/expedia-group-tech/a-short-introduction-to-apache-iceberg-d34f628b6799) **Date**: Jan 26th, 2021, **Company**: Expedia diff --git a/site/docs/community.md b/site/docs/community.md index c6f0d604ca10..997e568b1e45 100644 --- a/site/docs/community.md +++ b/site/docs/community.md @@ -21,12 +21,12 @@ Apache Iceberg tracks issues in GitHub and prefers to receive contributions as pull requests. -Community discussions happen primarily on the dev mailing list or on specific issues. +Community discussions happen primarily on the dev mailing list, on the #iceberg channel on the ASF's Slack workspace, and on specific Github issues. ## Contributing -Iceberg uses Apache's GitHub integration. The code is available at +The Iceberg Project is hosted on Github at . The Iceberg community prefers to receive contributions as [Github pull requests][github-pr-docs]. @@ -48,7 +48,12 @@ Issues are tracked in GitHub: [new-issue]: https://github.com/apache/iceberg/issues/new ## Slack -[![Slack](https://img.shields.io/badge/chat-on%20Slack-brightgreen.svg)](https://the-asf.slack.com/archives/CF01LKV9S) + +We use the [#iceberg](https://the-asf.slack.com/archives/CF01LKV9S) channel on +the ASF's Slack workspace. You get can an invitation to join this workspace at +. Please note that this link occasionally +breaks when Slack does an upgrade. If you encounter problems using it, please +let us know by sending an email to . ## Mailing Lists @@ -68,3 +73,20 @@ Iceberg has four mailing lists: - [Archive](https://lists.apache.org/list.html?issues@iceberg.apache.org) * **Private**: -- private list for the PMC to discuss sensitive issues related to the health of the project - [Archive](https://lists.apache.org/list.html?private@iceberg.apache.org) + + +## Setting up IDE and Code Style + +### Configuring Code Formatter for IntelliJ IDEA + +In the **Settings/Preferences** dialog go to **Editor > Code Style > Java**. Click on the gear wheel and select **Import Scheme** to import IntelliJ IDEA XML code style settings. +Point to [intellij-java-palantir-style.xml](https://github.com/apache/iceberg/blob/master/.baseline/idea/intellij-java-palantir-style.xml) and hit **OK** (you might need to enable **Show Hidden Files and Directories** in the dialog). The code itself can then be formatted via **Code > Reformat Code**. + +See also the IntelliJ [Code Style docs](https://www.jetbrains.com/help/idea/copying-code-style-settings.html) and [Reformat Code docs](https://www.jetbrains.com/help/idea/reformat-and-rearrange-code.html) for additional details. + +## Running Benchmarks +Some PRs/changesets might require running benchmarks to determine whether they are affecting the baseline performance. Currently there is +no "push a single button to get a performance comparison" solution available, therefore one has to run JMH performance tests on their local machine and +post the results on the PR. + +See [Benchmarks](benchmarks.md) for a summary of available benchmarks and how to run them. diff --git a/site/docs/configuration.md b/site/docs/configuration.md index afa02deb863c..119536cb91a5 100644 --- a/site/docs/configuration.md +++ b/site/docs/configuration.md @@ -61,6 +61,10 @@ Iceberg tables support table properties to configure table behavior, like the de | commit.retry.min-wait-ms | 100 | Minimum time in milliseconds to wait before retrying a commit | | commit.retry.max-wait-ms | 60000 (1 min) | Maximum time in milliseconds to wait before retrying a commit | | commit.retry.total-timeout-ms | 1800000 (30 min) | Maximum time in milliseconds to wait before retrying a commit | +| commit.status-check.num-retries | 3 | Number of times to check whether a commit succeeded after a connection is lost before failing due to an unknown commit state | +| commit.status-check.min-wait-ms | 1000 (1s) | Minimum time in milliseconds to wait before retrying a status-check | +| commit.status-check.max-wait-ms | 60000 (1 min) | Maximum time in milliseconds to wait before retrying a status-check | +| commit.status-check.total-timeout-ms| 1800000 (30 min) | Maximum time in milliseconds to wait before retrying a status-check | | commit.manifest.target-size-bytes | 8388608 (8 MB) | Target size when merging manifest files | | commit.manifest.min-count-to-merge | 100 | Minimum number of manifests to accumulate before merging | | commit.manifest-merge.enabled | true | Controls whether to automatically merge manifests on writes | diff --git a/site/docs/css/extra.css b/site/docs/css/extra.css index 3d79de02b3b6..d5b7a35438fb 100644 --- a/site/docs/css/extra.css +++ b/site/docs/css/extra.css @@ -92,3 +92,10 @@ pre { padding: 0.5em; padding-left: 1em; } + +.spec-img { + float: right; + max-width: 50%; + min-width: 265px; + margin: 1em 0 1em 2em; +} \ No newline at end of file diff --git a/site/docs/evolution.md b/site/docs/evolution.md index 0b4bec2fb2e4..624a9866a31a 100644 --- a/site/docs/evolution.md +++ b/site/docs/evolution.md @@ -74,7 +74,7 @@ sampleTable.updateSpec() .commit(); ``` -Spark supports updating partition spec through its `ALTER TABLE` SQL statement, see more details in [Spark SQL](../spark/#alter-table-add-partition-field). +Spark supports updating partition spec through its `ALTER TABLE` SQL statement, see more details in [Spark SQL](../spark-ddl/#alter-table-add-partition-field). ## Sort order evolution @@ -95,4 +95,4 @@ sampleTable.replaceSortOrder() .commit(); ``` -Spark supports updating sort order through its `ALTER TABLE` SQL statement, see more details in [Spark SQL](../spark/#alter-table-write-ordered-by). +Spark supports updating sort order through its `ALTER TABLE` SQL statement, see more details in [Spark SQL](../spark-ddl/#alter-table-write-ordered-by). diff --git a/site/docs/flink.md b/site/docs/flink.md index ccc559c88605..b4209d09bc65 100644 --- a/site/docs/flink.md +++ b/site/docs/flink.md @@ -196,6 +196,7 @@ CREATE CATALOG hive_catalog WITH ( * `property-version`: Version number to describe the property version. This property can be used for backwards compatibility in case the property format changes. The current property version is `1`. (Optional) * `warehouse`: The Hive warehouse location, users should specify this path if neither set the `hive-conf-dir` to specify a location containing a `hive-site.xml` configuration file nor add a correct `hive-site.xml` to classpath. * `hive-conf-dir`: Path to a directory containing a `hive-site.xml` configuration file which will be used to provide custom Hive configuration values. The value of `hive.metastore.warehouse.dir` from `/hive-site.xml` (or hive configure file from classpath) will be overwrote with the `warehouse` value if setting both `hive-conf-dir` and `warehouse` when creating iceberg catalog. +* `cache-enabled`: Whether to enable catalog cache, default value is `true` ### Hadoop catalog @@ -405,10 +406,10 @@ This example will read all records from iceberg table and then print to the stdo ```java StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(); -TableLoader tableLoader = TableLoader.fromHadooptable("hdfs://nn:8020/warehouse/path"); +TableLoader tableLoader = TableLoader.fromHadoopTable("hdfs://nn:8020/warehouse/path"); DataStream batch = FlinkSource.forRowData() .env(env) - .tableLoader(loader) + .tableLoader(tableLoader) .streaming(false) .build(); @@ -428,7 +429,7 @@ StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironme TableLoader tableLoader = TableLoader.fromHadoopTable("hdfs://nn:8020/warehouse/path"); DataStream stream = FlinkSource.forRowData() .env(env) - .tableLoader(loader) + .tableLoader(tableLoader) .streaming(true) .startSnapshotId(3821550127947089987L) .build(); @@ -440,7 +441,7 @@ stream.print(); env.execute("Test Iceberg Batch Read"); ``` -There are other options that we could set by Java API, please see the [FlinkSource#Builder](./javadoc/master/org/apache/iceberg/flink/source/FlinkSource.html). +There are other options that we could set by Java API, please see the [FlinkSource#Builder](./javadoc/0.11.1/org/apache/iceberg/flink/source/FlinkSource.html). ## Writing with DataStream @@ -504,7 +505,7 @@ RewriteDataFilesActionResult result = Actions.forTable(table) .execute(); ``` -For more doc about options of the rewrite files action, please see [RewriteDataFilesAction](./javadoc/master/org/apache/iceberg/flink/actions/RewriteDataFilesAction.html) +For more doc about options of the rewrite files action, please see [RewriteDataFilesAction](./javadoc/0.11.1/org/apache/iceberg/flink/actions/RewriteDataFilesAction.html) ## Future improvement. diff --git a/site/docs/img/iceberg-metadata.png b/site/docs/img/iceberg-metadata.png index b3146d354cb0..48a1b0cee80e 100644 Binary files a/site/docs/img/iceberg-metadata.png and b/site/docs/img/iceberg-metadata.png differ diff --git a/site/docs/java-api-quickstart.md b/site/docs/java-api-quickstart.md index 660b8dc15b71..de8bd3145d1e 100644 --- a/site/docs/java-api-quickstart.md +++ b/site/docs/java-api-quickstart.md @@ -108,9 +108,9 @@ Spark uses both `HiveCatalog` and `HadoopTables` to load tables. Hive is used wh To read and write to tables from Spark see: -* [Reading a table in Spark](./spark.md#reading-an-iceberg-table) -* [Appending to a table in Spark](./spark.md#appending-data) -* [Overwriting data in a table in Spark](./spark.md#overwriting-data) +* [SQL queries in Spark](spark-queries.md#querying-with-sql) +* [`INSERT INTO` in Spark](spark-writes.md#insert-into) +* [`MERGE INTO` in Spark](spark-writes.md#merge-into) ## Schemas diff --git a/site/docs/maintenance.md b/site/docs/maintenance.md index 203c10313bd9..3624fe70271f 100644 --- a/site/docs/maintenance.md +++ b/site/docs/maintenance.md @@ -26,7 +26,7 @@ Each write to an Iceberg table creates a new _snapshot_, or version, of a table. Snapshots can be used for time-travel queries, or the table can be rolled back to any valid snapshot. -Snapshots accumulate until they are expired by the [`expireSnapshots`](./javadoc/master/org/apache/iceberg/Table.html#expireSnapshots--) operation. Regularly expiring snapshots is recommended to delete data files that are no longer needed, and to keep the size of table metadata small. +Snapshots accumulate until they are expired by the [`expireSnapshots`](./javadoc/0.11.1/org/apache/iceberg/Table.html#expireSnapshots--) operation. Regularly expiring snapshots is recommended to delete data files that are no longer needed, and to keep the size of table metadata small. This example expires snapshots that are older than 1 day: @@ -38,7 +38,7 @@ table.expireSnapshots() .commit(); ``` -See the [`ExpireSnapshots` Javadoc](./javadoc/master/org/apache/iceberg/ExpireSnapshots.html) to see more configuration options. +See the [`ExpireSnapshots` Javadoc](./javadoc/0.11.1/org/apache/iceberg/ExpireSnapshots.html) to see more configuration options. There is also a Spark action that can run table expiration in parallel for large tables: @@ -83,7 +83,7 @@ Actions.forTable(table) .execute(); ``` -See the [RemoveOrphanFilesAction Javadoc](./javadoc/master/org/apache/iceberg/RemoveOrphanFilesAction.html) to see more configuration options. +See the [RemoveOrphanFilesAction Javadoc](./javadoc/0.11.1/org/apache/iceberg/actions/RemoveOrphanFilesAction.html) to see more configuration options. This action may take a long time to finish if you have lots of files in data and metadata directories. It is recommended to execute this periodically, but you may not need to execute this often. @@ -119,7 +119,7 @@ Actions.forTable(table).rewriteDataFiles() The `files` metadata table is useful for inspecting data file sizes and determining when to compact partitons. -See the [`RewriteDataFilesAction` Javadoc](./javadoc/master/org/apache/iceberg/RewriteDataFilesAction.html) to see more configuration options. +See the [`RewriteDataFilesAction` Javadoc](./javadoc/0.11.1/org/apache/iceberg/actions/RewriteDataFilesAction.html) to see more configuration options. ### Rewrite manifests @@ -139,4 +139,4 @@ table.rewriteManifests() .commit(); ``` -See the [`RewriteManifestsAction` Javadoc](./javadoc/master/org/apache/iceberg/RewriteManifestsAction.html) to see more configuration options. +See the [`RewriteManifestsAction` Javadoc](./javadoc/0.11.1/org/apache/iceberg/actions/RewriteManifestsAction.html) to see more configuration options. diff --git a/site/docs/spark-configuration.md b/site/docs/spark-configuration.md index 0ae27cacfacb..dd633f189516 100644 --- a/site/docs/spark-configuration.md +++ b/site/docs/spark-configuration.md @@ -59,6 +59,7 @@ Both catalogs are configured using properties nested under the catalog name. Com | spark.sql.catalog._catalog-name_.default-namespace | default | The default current namespace for the catalog | | spark.sql.catalog._catalog-name_.uri | thrift://host:port | Metastore connect URI; default from `hive-site.xml` | | spark.sql.catalog._catalog-name_.warehouse | hdfs://nn:8020/warehouse/path | Base path for the warehouse directory | +| spark.sql.catalog._catalog-name_.cache-enabled | `true` or `false` | Whether to enable catalog cache, default value is `true` | Additional properties can be found in common [catalog configuration](./configuration.md#catalog-properties). diff --git a/site/docs/spark-procedures.md b/site/docs/spark-procedures.md index 39247f5c720e..6e190dce2d2b 100644 --- a/site/docs/spark-procedures.md +++ b/site/docs/spark-procedures.md @@ -246,7 +246,7 @@ Rewrite manifests for a table to optimize scan planning. Data files in manifests are sorted by fields in the partition spec. This procedure runs in parallel using a Spark job. -See the [`RewriteManifestsAction` Javadoc](./javadoc/master/org/apache/iceberg/actions/RewriteManifestsAction.html) +See the [`RewriteManifestsAction` Javadoc](./javadoc/0.11.1/org/apache/iceberg/actions/RewriteManifestsAction.html) to see more configuration options. **Note** this procedure invalidates all cached Spark plans that reference the affected table. diff --git a/site/docs/spark-structured-streaming.md b/site/docs/spark-structured-streaming.md index f4e54bde8053..b969dcbcf686 100644 --- a/site/docs/spark-structured-streaming.md +++ b/site/docs/spark-structured-streaming.md @@ -53,14 +53,14 @@ Iceberg supports `append` and `complete` output modes: * `append`: appends the rows of every micro-batch to the table * `complete`: replaces the table contents every micro-batch -The table should be created in prior to start the streaming query. Refer [SQL create table](/spark/#create-table) +The table should be created in prior to start the streaming query. Refer [SQL create table](/spark-ddl/#create-table) on Spark page to see how to create the Iceberg table. ### Writing against partitioned table Iceberg requires the data to be sorted according to the partition spec per task (Spark partition) in prior to write against partitioned table. For batch queries you're encouraged to do explicit sort to fulfill the requirement -(see [here](/spark/#writing-against-partitioned-table)), but the approach would bring additional latency as +(see [here](/spark-writes/#writing-to-partitioned-tables)), but the approach would bring additional latency as repartition and sort are considered as heavy operations for streaming workload. To avoid additional latency, you can enable fanout writer to eliminate the requirement. diff --git a/site/docs/spark.md b/site/docs/spark.md deleted file mode 100644 index 3615a764a4b9..000000000000 --- a/site/docs/spark.md +++ /dev/null @@ -1,804 +0,0 @@ - - -# Spark - -To use Iceberg in Spark, first configure [Spark catalogs](./spark-configuration.md). - -Iceberg uses Apache Spark's DataSourceV2 API for data source and catalog implementations. Spark DSv2 is an evolving API with different levels of support in Spark versions: - -| SQL feature support | Spark 3.0| Spark 2.4 | Notes | -|--------------------------------------------------|----------|------------|------------------------------------------------| -| [`CREATE TABLE`](#create-table) | ✔️ | | | -| [`CREATE TABLE AS`](#create-table-as-select) | ✔️ | | | -| [`REPLACE TABLE AS`](#replace-table-as-select) | ✔️ | | | -| [`ALTER TABLE`](#alter-table) | ✔️ | | ⚠ Requires [SQL extensions](./spark-configuration.md#sql-extensions) enabled to update partition field and sort order | -| [`DROP TABLE`](#drop-table) | ✔️ | | | -| [`SELECT`](#querying-with-sql) | ✔️ | | | -| [`INSERT INTO`](#insert-into) | ✔️ | | | -| [`INSERT OVERWRITE`](#insert-overwrite) | ✔️ | | | - -| DataFrame feature support | Spark 3.0| Spark 2.4 | Notes | -|--------------------------------------------------|----------|------------|------------------------------------------------| -| [DataFrame reads](#querying-with-dataframes) | ✔️ | ✔️ | | -| [DataFrame append](#appending-data) | ✔️ | ✔️ | | -| [DataFrame overwrite](#overwriting-data) | ✔️ | ✔️ | ⚠ Behavior changed in Spark 3.0 | -| [DataFrame CTAS and RTAS](#creating-tables) | ✔️ | | | -| [Metadata tables](#inspecting-tables) | ✔️ | ✔️ | | - -## Configuring catalogs - -Spark 3.0 adds an API to plug in table catalogs that are used to load, create, and manage Iceberg tables. Spark catalogs are configured by setting [Spark properties](./configuration.md#catalogs) under `spark.sql.catalog`. - -This creates an Iceberg catalog named `hive_prod` that loads tables from a Hive metastore: - -```plain -spark.sql.catalog.hive_prod = org.apache.iceberg.spark.SparkCatalog -spark.sql.catalog.hive_prod.type = hive -spark.sql.catalog.hive_prod.uri = thrift://metastore-host:port -# omit uri to use the same URI as Spark: hive.metastore.uris in hive-site.xml -``` - -Iceberg also supports a directory-based catalog in HDFS that can be configured using `type=hadoop`: - -```plain -spark.sql.catalog.hadoop_prod = org.apache.iceberg.spark.SparkCatalog -spark.sql.catalog.hadoop_prod.type = hadoop -spark.sql.catalog.hadoop_prod.warehouse = hdfs://nn:8020/warehouse/path -``` - -!!! Note - The Hive-based catalog only loads Iceberg tables. To load non-Iceberg tables in the same Hive metastore, use a [session catalog](#replacing-the-session-catalog). - -### Using catalogs - -Catalog names are used in SQL queries to identify a table. In the examples above, `hive_prod` and `hadoop_prod` can be used to prefix database and table names that will be loaded from those catalogs. - -```sql -SELECT * FROM hive_prod.db.table -- load db.table from catalog hive_prod -``` - -Spark 3 keeps track of the current catalog and namespace, which can be omitted from table names. - -```sql -USE hive_prod.db; -SELECT * FROM table -- load db.table from catalog hive_prod -``` - -To see the current catalog and namespace, run `SHOW CURRENT NAMESPACE`. - -### Replacing the session catalog - -To add Iceberg table support to Spark's built-in catalog, configure `spark_catalog` to use Iceberg's `SparkSessionCatalog`. - -```plain -spark.sql.catalog.spark_catalog = org.apache.iceberg.spark.SparkSessionCatalog -spark.sql.catalog.spark_catalog.type = hive -``` - -Spark's built-in catalog supports existing v1 and v2 tables tracked in a Hive Metastore. This configures Spark to use Iceberg's `SparkSessionCatalog` as a wrapper around that session catalog. When a table is not an Iceberg table, the built-in catalog will be used to load it instead. - -This configuration can use same Hive Metastore for both Iceberg and non-Iceberg tables. - -### Loading a custom catalog - -Spark supports loading a custom Iceberg `Catalog` implementation by specifying the `catalog-impl` property. -When `catalog-impl` is set, the value of `type` is ignored. Here is an example: - -```plain -spark.sql.catalog.custom_prod = org.apache.iceberg.spark.SparkCatalog -spark.sql.catalog.custom_prod.catalog-impl = com.my.custom.CatalogImpl -spark.sql.catalog.custom_prod.my-additional-catalog-config = my-value -``` - -## DDL commands - -!!! Note - Spark 2.4 can't create Iceberg tables with DDL, instead use the [Iceberg API](./java-api-quickstart.md). - -### `CREATE TABLE` - -Spark 3.0 can create tables in any Iceberg catalog with the clause `USING iceberg`: - -```sql -CREATE TABLE prod.db.sample ( - id bigint COMMENT 'unique id', - data string) -USING iceberg -``` - -Iceberg will convert the column type in Spark to corresponding Iceberg type. Please check the section of [type compatibility on creating table](#spark-type-to-iceberg-type) for details. - -Table create commands, including CTAS and RTAS, support the full range of Spark create clauses, including: - -* `PARTITION BY (partition-expressions)` to configure partitioning -* `LOCATION '(fully-qualified-uri)'` to set the table location -* `COMMENT 'table documentation'` to set a table description -* `TBLPROPERTIES ('key'='value', ...)` to set [table configuration](./configuration.md) - -Create commands may also set the default format with the `USING` clause. This is only supported for `SparkCatalog` because Spark handles the `USING` clause differently for the built-in catalog. - -#### `PARTITIONED BY` - -To create a partitioned table, use `PARTITIONED BY`: - -```sql -CREATE TABLE prod.db.sample ( - id bigint, - data string, - category string) -USING iceberg -PARTITIONED BY (category) -``` - -The `PARTITIONED BY` clause supports transform expressions to create [hidden partitions](./partitioning.md). - -```sql -CREATE TABLE prod.db.sample ( - id bigint, - data string, - category string, - ts timestamp) -USING iceberg -PARTITIONED BY (bucket(16, id), days(ts), category) -``` - -Supported partition transforms are: - -* `years` for yearly partitions -* `months` for monthly partitions -* `days` for daily partitions -* `hours` for hourly partitions -* `bucket` for bucketing (with width) -* `truncate` to truncate integers or strings (with length) - -### `CREATE TABLE ... AS SELECT` - -Iceberg supports CTAS as an atomic operation when using a [`SparkCatalog`](#configuring-catalogs). CTAS is supported, but is not atomic when using [`SparkSessionCatalog`](#replacing-the-session-catalog). - -```sql -CREATE TABLE prod.db.sample -USING iceberg -AS SELECT ... -``` - -### `REPLACE TABLE ... AS SELECT` - -Iceberg supports RTAS as an atomic operation when using a [`SparkCatalog`](#configuring-catalogs). RTAS is supported, but is not atomic when using [`SparkSessionCatalog`](#replacing-the-session-catalog). - -Atomic table replacement creates a new snapshot with the results of the `SELECT` query, but keeps table history. - -```sql -REPLACE TABLE prod.db.sample -USING iceberg -AS SELECT ... -``` -```sql -CREATE OR REPLACE TABLE prod.db.sample -USING iceberg -AS SELECT ... -``` - -The schema and partition spec will be replaced if changed. To avoid modifying the table's schema and partitioning, use `INSERT OVERWRITE` instead of `REPLACE TABLE`. -The new table properties in the `REPLACE TABLE` command will be merged with any existing table properties. The existing table properties will be updated if changed else they are preserved. -### `ALTER TABLE` - -Iceberg has full `ALTER TABLE` support in Spark 3, including: - -* Renaming a table -* Setting or removing table properties -* Adding, deleting, and renaming columns -* Adding, deleting, and renaming nested fields -* Reordering top-level columns and nested struct fields -* Widening the type of `int`, `float`, and `decimal` fields -* Making required columns optional - -### `ALTER TABLE ... RENAME TO` - -```sql -ALTER TABLE prod.db.sample RENAME TO prod.db.new_name -``` - -### `ALTER TABLE ... SET TBLPROPERTIES` - -```sql -ALTER TABLE prod.db.sample SET TBLPROPERTIES ( - 'read.split.target-size'='268435456' -) -``` - -Iceberg uses table properties to control table behavior. For a list of available properties, see [Table configuration](./configuration.md). - -`UNSET` is used to remove properties: - -```sql -ALTER TABLE prod.db.sample UNSET TBLPROPERTIES ('read.split.target-size') -``` - -### `ALTER TABLE ... ADD COLUMN` - -```sql -ALTER TABLE prod.db.sample ADD COLUMN point struct AFTER data -ALTER TABLE prod.db.sample ADD COLUMN point.z double FIRST -``` - -### `ALTER TABLE ... RENAME COLUMN` - -```sql -ALTER TABLE prod.db.sample RENAME COLUMN data TO payload -ALTER TABLE prod.db.sample RENAME COLUMN location.lat TO latitude -``` - -Note that nested rename commands only rename the leaf field. The above command renames `location.lat` to `location.latitude` - -### `ALTER TABLE ... ALTER COLUMN` - -Alter column is used to widen types, make a field optional, set comments, and reorder fields. - -```sql -ALTER TABLE prod.db.sample ALTER COLUMN id DROP NOT NULL -ALTER TABLE prod.db.sample ALTER COLUMN location.lat TYPE double -ALTER TABLE prod.db.sample ALTER COLUMN point.z AFTER y -ALTER TABLE prod.db.sample ALTER COLUMN id COMMENT 'unique id' -``` - -### `ALTER TABLE ... DROP COLUMN` - -```sql -ALTER TABLE prod.db.sample DROP COLUMN id -ALTER TABLE prod.db.sample DROP COLUMN point.z -``` - -### `ALTER TABLE ... ADD PARTITION FIELD` - -```sql -ALTER TABLE prod.db.sample ADD PARTITION FIELD catalog -- identity transform -ALTER TABLE prod.db.sample ADD PARTITION FIELD bucket(16, id) -ALTER TABLE prod.db.sample ADD PARTITION FIELD truncate(data, 4) -ALTER TABLE prod.db.sample ADD PARTITION FIELD years(ts) --- use optional AS keyword to specify a custom name for the partition field -ALTER TABLE prod.db.sample ADD PARTITION FIELD bucket(16, id) AS shard -``` - -!!! Warning - Changing partitioning will change the behavior of dynamic writes, which overwrite any partition that is written to. - For example, if you partition by days and move to partitioning by hours, overwrites will overwrite hourly partitions but not days anymore. - - -### `ALTER TABLE ... DROP PARTITION FIELD` - -```sql -ALTER TABLE prod.db.sample DROP PARTITION FIELD catalog -ALTER TABLE prod.db.sample DROP PARTITION FIELD bucket(16, id) -ALTER TABLE prod.db.sample DROP PARTITION FIELD truncate(data, 4) -ALTER TABLE prod.db.sample DROP PARTITION FIELD years(ts) -ALTER TABLE prod.db.sample DROP PARTITION FIELD shard -``` - -!!! Warning - Changing partitioning will change the behavior of dynamic writes, which overwrite any partition that is written to. - For example, if you partition by days and move to partitioning by hours, overwrites will overwrite hourly partitions but not days anymore. - - -### `ALTER TABLE ... WRITE ORDERED BY` - -```sql -ALTER TABLE prod.db.sample WRITE ORDERED BY category, id --- use optional ASC/DEC keyword to specify sort order of each field (default ASC) -ALTER TABLE prod.db.sample WRITE ORDERED BY category ASC, id DESC --- use optional NULLS FIRST/NULLS LAST keyword to specify null order of each field (default FIRST) -ALTER TABLE prod.db.sample WRITE ORDERED BY category ASC NULLS LAST, id DESC NULLS FIRST -``` - -### `DROP TABLE` - -To delete a table, run: - -```sql -DROP TABLE prod.db.sample -``` - -## Querying with SQL - -In Spark 3, tables use identifiers that include a [catalog name](#using-catalogs). - -```sql -SELECT * FROM prod.db.table -- catalog: prod, namespace: db, table: table -``` - -Metadata tables, like `history` and `snapshots`, can use the Iceberg table name as a namespace. - -For example, to read from the `files` metadata table for `prod.db.table`, run: - -``` -SELECT * FROM prod.db.table.files -``` - -## Querying with DataFrames - -To load a table as a DataFrame, use `table`: - -```scala -val df = spark.table("prod.db.table") -``` - -!!! Warning - When reading with DataFrames in Spark 3, use `table` to load a table by name from a catalog unless `option` is also required. - Using `format("iceberg")` loads an isolated table reference that is not refreshed when other queries update the table. - - -### Time travel - -To select a specific table snapshot or the snapshot at some time, Iceberg supports two Spark read options: - -* `snapshot-id` selects a specific table snapshot -* `as-of-timestamp` selects the current snapshot at a timestamp, in milliseconds - -```scala -// time travel to October 26, 1986 at 01:21:00 -spark.read - .option("as-of-timestamp", "499162860000") - .format("iceberg") - .load("path/to/table") -``` - -```scala -// time travel to snapshot with ID 10963874102873L -spark.read - .option("snapshot-id", 10963874102873L) - .format("iceberg") - .load("path/to/table") -``` - -!!! Note - Spark does not currently support using `option` with `table` in DataFrameReader commands. All options will be silently - ignored. Do not use `table` when attempting to time-travel or use other options. Options will be supported with `table` - in [Spark 3.1 - SPARK-32592](https://issues.apache.org/jira/browse/SPARK-32592). - -Time travel is not yet supported by Spark's SQL syntax. - -### Table names and paths - -Paths and table names can be loaded from the Spark3 dataframe interface. How paths/tables are loaded depends on how -the identifier is specified. When using `spark.read().format("iceberg").path(table)` or `spark.table(table)` the `table` -variable can take a number of forms as listed below: - -* `file:/path/to/table` -> loads a HadoopTable at given path -* `tablename` -> loads `currentCatalog.currentNamespace.tablename` -* `catalog.tablename` -> load `tablename` from the specified catalog. -* `namespace.tablename` -> load `namespace.tablename` from current catalog -* `catalog.namespace.tablename` -> load `namespace.tablename` from the specified catalog. -* `namespace1.namespace2.tablename` -> load `namespace1.namespace2.tablename` from current catalog - -The above list is in order of priority. For example: a matching catalog will take priority over any namespace resolution. - -### Spark 2.4 - -Spark 2.4 requires using the DataFrame reader with `iceberg` as a format, because 2.4 does not support catalogs: - -```scala -// named metastore table -spark.read.format("iceberg").load("db.table") -// Hadoop path table -spark.read.format("iceberg").load("hdfs://nn:8020/path/to/table") -``` - -#### Spark 2.4 with SQL - -To run SQL `SELECT` statements on Iceberg tables in 2.4, register the DataFrame as a temporary table: - -```scala -val df = spark.read.format("iceberg").load("db.table") -df.createOrReplaceTempView("table") - -spark.sql("""select count(1) from table""").show() -``` - - -## Writing with SQL - -Spark 3 supports SQL `INSERT INTO` and `INSERT OVERWRITE`, as well as the new `DataFrameWriterV2` API. - -### `INSERT INTO` - -To append new data to a table, use `INSERT INTO`. - -```sql -INSERT INTO prod.db.table VALUES (1, 'a'), (2, 'b') -``` -```sql -INSERT INTO prod.db.table SELECT ... -``` - -### `INSERT OVERWRITE` - -To replace data in the table with the result of a query, use `INSERT OVERWRITE`. Overwrites are atomic operations for Iceberg tables. - -The partitions that will be replaced by `INSERT OVERWRITE` depends on Spark's partition overwrite mode and the partitioning of a table. - -!!! Warning - Spark 3.0.0 has a correctness bug that affects dynamic `INSERT OVERWRITE` with hidden partitioning, [SPARK-32168][spark-32168]. - For tables with [hidden partitions](./partitioning.md), wait for Spark 3.0.1. - -[spark-32168]: https://issues.apache.org/jira/browse/SPARK-32168 - - -#### Overwrite behavior - -Spark's default overwrite mode is **static**, but **dynamic overwrite mode is recommended when writing to Iceberg tables.** Static overwrite mode determines which partitions to overwrite in a table by converting the `PARTITION` clause to a filter, but the `PARTITION` clause can only reference table columns. - -Dynamic overwrite mode is configured by setting `spark.sql.sources.partitionOverwriteMode=dynamic`. - -To demonstrate the behavior of dynamic and static overwrites, consider a `logs` table defined by the following DDL: - -```sql -CREATE TABLE prod.my_app.logs ( - uuid string NOT NULL, - level string NOT NULL, - ts timestamp NOT NULL, - message string) -USING iceberg -PARTITIONED BY (level, hours(ts)) -``` - -#### Dynamic overwrite - -When Spark's overwrite mode is dynamic, partitions that have rows produced by the `SELECT` query will be replaced. - -For example, this query removes duplicate log events from the example `logs` table. - -```sql -INSERT OVERWRITE prod.my_app.logs -SELECT uuid, first(level), first(ts), first(message) -FROM prod.my_app.logs -WHERE cast(ts as date) = '2020-07-01' -GROUP BY uuid -``` - -In dynamic mode, this will replace any partition with rows in the `SELECT` result. Because the date of all rows is restricted to 1 July, only hours of that day will be replaced. - -#### Static overwrite - -When Spark's overwrite mode is static, the `PARTITION` clause is converted to a filter that is used to delete from the table. If the `PARTITION` clause is omitted, all partitions will be replaced. - -Because there is no `PARTITION` clause in the query above, it will drop all existing rows in the table when run in static mode, but will only write the logs from 1 July. - -To overwrite just the partitions that were loaded, add a `PARTITION` clause that aligns with the `SELECT` query filter: - -```sql -INSERT OVERWRITE prod.my_app.logs -PARTITION (level = 'INFO') -SELECT uuid, first(level), first(ts), first(message) -FROM prod.my_app.logs -WHERE level = 'INFO' -GROUP BY uuid -``` - -Note that this mode cannot replace hourly partitions like the dynamic example query because the `PARTITION` clause can only reference table columns, not hidden partitions. - -### `DELETE FROM` - -Spark 3 added support for `DELETE FROM` queries to remove data from tables. - -Delete queries accept a filter to match rows to delete. Iceberg can delete data as long as the filter matches entire partitions of the table, or it can determine that all rows of a file match. If a file contains some rows that should be deleted and some that should not, Iceberg will throw an exception. - -```sql -DELETE FROM prod.db.table -WHERE ts >= '2020-05-01 00:00:00' and ts < '2020-06-01 00:00:00' -``` - - -## Writing with DataFrames - -Spark 3 introduced the new `DataFrameWriterV2` API for writing to tables using data frames. The v2 API is recommended for several reasons: - -* CTAS, RTAS, and overwrite by filter are supported -* All operations consistently write columns to a table by name -* Hidden partition expressions are supported in `partitionedBy` -* Overwrite behavior is explicit, either dynamic or by a user-supplied filter -* The behavior of each operation corresponds to SQL statements - - `df.writeTo(t).create()` is equivalent to `CREATE TABLE AS SELECT` - - `df.writeTo(t).replace()` is equivalent to `REPLACE TABLE AS SELECT` - - `df.writeTo(t).append()` is equivalent to `INSERT INTO` - - `df.writeTo(t).overwritePartitions()` is equivalent to dynamic `INSERT OVERWRITE` - -The v1 DataFrame `write` API is still supported, but is not recommended. - -!!! Warning - When writing with the v1 DataFrame API in Spark 3, use `saveAsTable` or `insertInto` to load tables with a catalog. - Using `format("iceberg")` loads an isolated table reference that will not automatically refresh tables used by queries. - - -### Appending data - -To append a dataframe to an Iceberg table, use `append`: - -```scala -val data: DataFrame = ... -data.writeTo("prod.db.table").append() -``` - -#### Spark 2.4 - -In Spark 2.4, use the v1 API with `append` mode and `iceberg` format: - -```scala -data.write - .format("iceberg") - .mode("append") - .save("db.table") -``` - -### Overwriting data - -To overwrite partitions dynamically, use `overwritePartitions()`: - -```scala -val data: DataFrame = ... -data.writeTo("prod.db.table").overwritePartitions() -``` - -To explicitly overwrite partitions, use `overwrite` to supply a filter: - -```scala -data.writeTo("prod.db.table").overwrite($"level" === "INFO") -``` - -#### Spark 2.4 - -In Spark 2.4, overwrite values in an Iceberg table with `overwrite` mode and `iceberg` format: - -```scala -data.write - .format("iceberg") - .mode("overwrite") - .save("db.table") -``` - -!!! Warning - **The behavior of overwrite mode changed between Spark 2.4 and Spark 3**. - -The behavior of DataFrameWriter overwrite mode was undefined in Spark 2.4, but is required to overwrite the entire table in Spark 3. Because of this new requirement, the Iceberg source's behavior changed in Spark 3. In Spark 2.4, the behavior was to dynamically overwrite partitions. To use the Spark 2.4 behavior, add option `overwrite-mode=dynamic`. - -### Creating tables - -To run a CTAS or RTAS, use `create`, `replace`, or `createOrReplace` operations: - -```scala -val data: DataFrame = ... -data.writeTo("prod.db.table").create() -``` - -Create and replace operations support table configuration methods, like `partitionedBy` and `tableProperty`: - -```scala -data.writeTo("prod.db.table") - .tableProperty("write.format.default", "orc") - .partitionBy($"level", days($"ts")) - .createOrReplace() -``` - -## Writing against partitioned table - -Iceberg requires the data to be sorted according to the partition spec per task (Spark partition) in prior to write -against partitioned table. This applies both Writing with SQL and Writing with DataFrames. - -!!! Note - Explicit sort is necessary because Spark doesn't allow Iceberg to request a sort before writing as of Spark 3.0. - [SPARK-23889](https://issues.apache.org/jira/browse/SPARK-23889) is filed to enable Iceberg to require specific - distribution & sort order to Spark. - -!!! Note - Both global sort (`orderBy`/`sort`) and local sort (`sortWithinPartitions`) work for the requirement. - -Let's go through writing the data against below sample table: - -```sql -CREATE TABLE prod.db.sample ( - id bigint, - data string, - category string, - ts timestamp) -USING iceberg -PARTITIONED BY (days(ts), category) -``` - -To write data to the sample table, your data needs to be sorted by `days(ts), category`. - -If you're inserting data with SQL statement, you can use `ORDER BY` to achieve it, like below: - -```sql -INSERT INTO prod.db.sample -SELECT id, data, category, ts FROM another_table -ORDER BY ts, category -``` - -If you're inserting data with DataFrame, you can use either `orderBy`/`sort` to trigger global sort, or `sortWithinPartitions` -to trigger local sort. Local sort for example: - -```scala -data.sortWithinPartitions("ts", "category") - .writeTo("prod.db.sample") - .append() -``` - -You can simply add the original column to the sort condition for the most partition transformations, except `bucket`. - -For `bucket` partition transformation, you need to register the Iceberg transform function in Spark to specify it during sort. - -Let's go through another sample table having bucket partition: - -```sql -CREATE TABLE prod.db.sample ( - id bigint, - data string, - category string, - ts timestamp) -USING iceberg -PARTITIONED BY (bucket(16, id)) -``` - -You need to register the function to deal with bucket, like below: - -```scala -import org.apache.iceberg.spark.IcebergSpark -import org.apache.spark.sql.types.DataTypes - -IcebergSpark.registerBucketUDF(spark, "iceberg_bucket16", DataTypes.LongType, 16) -``` - -!!! Note - Explicit registration of the function is necessary because Spark doesn't allow Iceberg to provide functions. - [SPARK-27658](https://issues.apache.org/jira/browse/SPARK-27658) is filed to enable Iceberg to provide functions - which can be used in query. - -Here we just registered the bucket function as `iceberg_bucket16`, which can be used in sort clause. - -If you're inserting data with SQL statement, you can use the function like below: - -```sql -INSERT INTO prod.db.sample -SELECT id, data, category, ts FROM another_table -ORDER BY iceberg_bucket16(id) -``` - -If you're inserting data with DataFrame, you can use the function like below: - -```scala -data.sortWithinPartitions(expr("iceberg_bucket16(id)")) - .writeTo("prod.db.sample") - .append() -``` - -## Inspecting tables - -To inspect a table's history, snapshots, and other metadata, Iceberg supports metadata tables. - -Metadata tables are identified by adding the metadata table name after the original table name. For example, history for `db.table` is read using `db.table.history`. - -!!! Note - As of Spark 3.0, the format of the table name for inspection (`catalog.database.table.metadata`) doesn't work with Spark's default catalog (`spark_catalog`). If you've replaced the default catalog, you may want to use DataFrameReader API to inspect the table. - -### History - -To show table history, run: - -```sql -SELECT * FROM prod.db.table.history -``` -```text -+-------------------------+---------------------+---------------------+---------------------+ -| made_current_at | snapshot_id | parent_id | is_current_ancestor | -+-------------------------+---------------------+---------------------+---------------------+ -| 2019-02-08 03:29:51.215 | 5781947118336215154 | NULL | true | -| 2019-02-08 03:47:55.948 | 5179299526185056830 | 5781947118336215154 | true | -| 2019-02-09 16:24:30.13 | 296410040247533544 | 5179299526185056830 | false | -| 2019-02-09 16:32:47.336 | 2999875608062437330 | 5179299526185056830 | true | -| 2019-02-09 19:42:03.919 | 8924558786060583479 | 2999875608062437330 | true | -| 2019-02-09 19:49:16.343 | 6536733823181975045 | 8924558786060583479 | true | -+-------------------------+---------------------+---------------------+---------------------+ -``` - -!!! Note - **This shows a commit that was rolled back.** The example has two snapshots with the same parent, and one is *not* an ancestor of the current table state. - -### Snapshots - -To show the valid snapshots for a table, run: - -```sql -SELECT * FROM prod.db.table.snapshots -``` -```text -+-------------------------+----------------+-----------+-----------+----------------------------------------------------+-------------------------------------------------------+ -| committed_at | snapshot_id | parent_id | operation | manifest_list | summary | -+-------------------------+----------------+-----------+-----------+----------------------------------------------------+-------------------------------------------------------+ -| 2019-02-08 03:29:51.215 | 57897183625154 | null | append | s3://.../table/metadata/snap-57897183625154-1.avro | { added-records -> 2478404, total-records -> 2478404, | -| | | | | | added-data-files -> 438, total-data-files -> 438, | -| | | | | | spark.app.id -> application_1520379288616_155055 } | -| ... | ... | ... | ... | ... | ... | -+-------------------------+----------------+-----------+-----------+----------------------------------------------------+-------------------------------------------------------+ -``` - -You can also join snapshots to table history. For example, this query will show table history, with the application ID that wrote each snapshot: - -```sql -select - h.made_current_at, - s.operation, - h.snapshot_id, - h.is_current_ancestor, - s.summary['spark.app.id'] -from prod.db.table.history h -join prod.db.table.snapshots s - on h.snapshot_id = s.snapshot_id -order by made_current_at -``` -```text -+-------------------------+-----------+----------------+---------------------+----------------------------------+ -| made_current_at | operation | snapshot_id | is_current_ancestor | summary[spark.app.id] | -+-------------------------+-----------+----------------+---------------------+----------------------------------+ -| 2019-02-08 03:29:51.215 | append | 57897183625154 | true | application_1520379288616_155055 | -| 2019-02-09 16:24:30.13 | delete | 29641004024753 | false | application_1520379288616_151109 | -| 2019-02-09 16:32:47.336 | append | 57897183625154 | true | application_1520379288616_155055 | -| 2019-02-08 03:47:55.948 | overwrite | 51792995261850 | true | application_1520379288616_152431 | -+-------------------------+-----------+----------------+---------------------+----------------------------------+ -``` - -### Manifests - -To show a table's file manifests and each file's metadata, run: - -```sql -SELECT * FROM prod.db.table.manifests -``` -```text -+----------------------------------------------------------------------+--------+-------------------+---------------------+------------------------+---------------------------+--------------------------+---------------------------------+ -| path | length | partition_spec_id | added_snapshot_id | added_data_files_count | existing_data_files_count | deleted_data_files_count | partitions | -+----------------------------------------------------------------------+--------+-------------------+---------------------+------------------------+---------------------------+--------------------------+---------------------------------+ -| s3://.../table/metadata/45b5290b-ee61-4788-b324-b1e2735c0e10-m0.avro | 4479 | 0 | 6668963634911763636 | 8 | 0 | 0 | [[false,2019-05-13,2019-05-15]] | -+----------------------------------------------------------------------+--------+-------------------+---------------------+------------------------+---------------------------+--------------------------+---------------------------------+ -``` - -### Files - -To show a table's data files and each file's metadata, run: - -```sql -SELECT * FROM prod.db.table.files -``` -```text -+-------------------------------------------------------------------------+-------------+--------------+--------------------+--------------------+------------------+-------------------+-----------------+-----------------+--------------+---------------+ -| file_path | file_format | record_count | file_size_in_bytes | column_sizes | value_counts | null_value_counts | lower_bounds | upper_bounds | key_metadata | split_offsets | -+-------------------------------------------------------------------------+-------------+--------------+--------------------+--------------------+------------------+-------------------+-----------------+-----------------+--------------+---------------+ -| s3:/.../table/data/00000-3-8d6d60e8-d427-4809-bcf0-f5d45a4aad96.parquet | PARQUET | 1 | 597 | [1 -> 90, 2 -> 62] | [1 -> 1, 2 -> 1] | [1 -> 0, 2 -> 0] | [1 -> , 2 -> c] | [1 -> , 2 -> c] | null | [4] | -| s3:/.../table/data/00001-4-8d6d60e8-d427-4809-bcf0-f5d45a4aad96.parquet | PARQUET | 1 | 597 | [1 -> 90, 2 -> 62] | [1 -> 1, 2 -> 1] | [1 -> 0, 2 -> 0] | [1 -> , 2 -> b] | [1 -> , 2 -> b] | null | [4] | -| s3:/.../table/data/00002-5-8d6d60e8-d427-4809-bcf0-f5d45a4aad96.parquet | PARQUET | 1 | 597 | [1 -> 90, 2 -> 62] | [1 -> 1, 2 -> 1] | [1 -> 0, 2 -> 0] | [1 -> , 2 -> a] | [1 -> , 2 -> a] | null | [4] | -+-------------------------------------------------------------------------+-------------+--------------+--------------------+--------------------+------------------+-------------------+-----------------+-----------------+--------------+---------------+ -``` - -### Inspecting with DataFrames - -Metadata tables can be loaded in Spark 2.4 or Spark 3 using the DataFrameReader API: - -```scala -// named metastore table -spark.read.format("iceberg").load("db.table.files").show(truncate = false) -// Hadoop path table -spark.read.format("iceberg").load("hdfs://nn:8020/path/to/table#files").show(truncate = false) -``` - diff --git a/site/docs/spec.md b/site/docs/spec.md index fa162616680e..d9ccb4647e25 100644 --- a/site/docs/spec.md +++ b/site/docs/spec.md @@ -27,7 +27,7 @@ This is a specification for the Iceberg table format that is designed to manage The Iceberg community is currently working on version 2 of the Iceberg format that supports encoding row-level deletes. **The v2 specification is incomplete and may change until it is finished and adopted.** This document includes tentative v2 format requirements, but there are currently no compatibility guarantees with the unfinished v2 spec. -The goal of version 2 is to provide a way to encode row-level deletes. This update can be used to delete or replace individual rows in an immutable data file without rewriting the file. +The primary goal of version 2 is to provide a way to encode row-level deletes. This update can be used to delete or replace individual rows in an immutable data file without rewriting the file. ## Goals @@ -42,7 +42,7 @@ The goal of version 2 is to provide a way to encode row-level deletes. This upda ## Overview -![Iceberg snapshot structure](img/iceberg-metadata.png){.floating} +![Iceberg snapshot structure](img/iceberg-metadata.png){.spec-img} This table format tracks individual data files in a table instead of directories. This allows writers to create data files in-place and only adds files to the table in an explicit commit. @@ -110,6 +110,31 @@ Tables do not require rename, except for tables that use atomic rename to implem * **Data file** -- A file that contains rows of a table. * **Delete file** -- A file that encodes rows of a table that are deleted by position or data values. +#### Writer requirements + +Some tables in this spec have columns that specify requirements for v1 and v2 tables. These requirements are intended for writers when adding metadata files to a table with the given version. + +| Requirement | Write behavior | +|-------------|----------------| +| (blank) | The field should be omitted | +| _optional_ | The field can be written | +| _required_ | The field must be written | + +Readers should be more permissive because v1 metadata files are allowed in v2 tables so that tables can be upgraded to v2 without rewriting the metadata tree. For manifest list and manifest files, this table shows the expected v2 read behavior: + +| v1 | v2 | v2 read behavior | +|------------|------------|------------------| +| | _optional_ | Read the field as _optional_ | +| | _required_ | Read the field as _optional_; it may be missing in v1 files | +| _optional_ | | Ignore the field | +| _optional_ | _optional_ | Read the field as _optional_ | +| _optional_ | _required_ | Read the field as _optional_; it may be missing in v1 files | +| _required_ | | Ignore the field | +| _required_ | _optional_ | Read the field as _optional_ | +| _required_ | _required_ | Fill in a default or throw an exception if the field is missing | + +Readers may be more strict for metadata JSON files because the JSON files are not reused and will always match the table version. Required v2 fields that were not present in v1 or optional in v1 may be handled as required fields. For example, a v2 table that is missing `last-sequence-number` can throw an exception. + ### Schemas and Data Types A table's **schema** is a list of named columns. All data types are either primitives or nested types, which are maps, lists, or structs. A table schema is also a struct type. @@ -156,7 +181,9 @@ For details on how to serialize a schema to JSON, see Appendix C. #### Schema Evolution -Schema evolution is limited to type promotion and adding, deleting, and renaming fields in structs (both nested structs and the top-level schema’s struct). +Schemas may be evolved by type promotion or adding, deleting, renaming, or reordering fields in structs (both nested structs and the top-level schema’s struct). + +Evolution applies changes to the table's current schema to produce a new schema that is identified by a unique schema ID, is added to the table's list of schemas, and is set as the table's current schema. Valid type promotions are: @@ -176,6 +203,15 @@ Columns in Iceberg data files are selected by field id. The table schema's colum For example, a file may be written with schema `1: a int, 2: b string, 3: c double` and read using projection schema `3: measurement, 2: name, 4: a`. This must select file columns `c` (renamed to `measurement`), `b` (now called `name`), and a column of `null` values called `a`; in that order. +#### Identifier Field IDs + +A schema can optionally track the set of primitive fields that identify rows in a table, using the property `identifier-field-ids` (see JSON encoding in Appendix C). + +Two rows are the "same"---that is, the rows represent the same entity---if the identifier fields are equal. However, uniqueness of rows by this identifier is not guaranteed or required by Iceberg and it is the responsibility of processing engines or data providers to enforce. + +Identifier fields may be nested in structs but cannot be nested within maps or lists. Float, double, and optional fields cannot be used as identifier fields and a nested field cannot be used as an identifier field if it is nested in an optional struct, to avoid null values in identifiers. + + #### Reserved Field IDs Iceberg tables must not use field ids greater than 2147483447 (`Integer.MAX_VALUE - 200`). This id range is reserved for metadata columns that can be used in user data schemas, like the `_file` column that holds the file path in which a row was stored. @@ -209,18 +245,21 @@ Partition specs capture the transform from table data to partition values. This #### Partition Transforms -| Transform | Description | Source types | Result type | +| Transform name | Description | Source types | Result type | |-------------------|--------------------------------------------------------------|-----------------------------------------------------------------------------------------------------------|-------------| | **`identity`** | Source value, unmodified | Any | Source type | | **`bucket[N]`** | Hash of value, mod `N` (see below) | `int`, `long`, `decimal`, `date`, `time`, `timestamp`, `timestamptz`, `string`, `uuid`, `fixed`, `binary` | `int` | | **`truncate[W]`** | Value truncated to width `W` (see below) | `int`, `long`, `decimal`, `string` | Source type | -| **`year`** | Extract a date or timestamp year, as years from 1970 | `date`, `timestamp(tz)` | `int` | -| **`month`** | Extract a date or timestamp month, as months from 1970-01-01 | `date`, `timestamp(tz)` | `int` | -| **`day`** | Extract a date or timestamp day, as days from 1970-01-01 | `date`, `timestamp(tz)` | `date` | +| **`year`** | Extract a date or timestamp year, as years from 1970 | `date`, `timestamp`, `timestamptz` | `int` | +| **`month`** | Extract a date or timestamp month, as months from 1970-01-01 | `date`, `timestamp`, `timestamptz` | `int` | +| **`day`** | Extract a date or timestamp day, as days from 1970-01-01 | `date`, `timestamp`, `timestamptz` | `date` | | **`hour`** | Extract a timestamp hour, as hours from 1970-01-01 00:00:00 | `timestamp(tz)` | `int` | +| **`void`** | Always produces `null` | Any | Source type or `int` | All transforms must return `null` for a `null` input value. +The `void` transform may be used to replace the transform in an existing partition field so that the field is effectively dropped in v1 tables. See partition evolution below. + #### Bucket Transform Details @@ -254,6 +293,23 @@ Notes: 2. The width, `W`, used to truncate decimal values is applied using the scale of the decimal column to avoid additional (and potentially conflicting) parameters. +#### Partition Evolution + +Table partitioning can be evolved by adding, removing, renaming, or reordering partition spec fields. + +Changing a partition spec produces a new spec identified by a unique spec ID that is added to the table's list of partition specs and may be set as the table's default spec. + +When evolving a spec, changes should not cause partition field IDs to change because the partition field IDs are used as the partition tuple field IDs in manifest files. + +In v2, partition field IDs must be explicitly tracked for each partition field. New IDs are assigned based on the last assigned partition ID in table metadata. + +In v1, partition field IDs were not tracked, but were assigned sequentially starting at 1000 in the reference implementation. This assignment caused problems when reading metadata tables based on manifest files from multiple specs because partition fields with the same ID may contain different data types. For compatibility with old versions, the following rules are recommended for partition evolution in v1 tables: + +1. Do not reorder partition fields +2. Do not drop partition fields; instead replace the field's transform with the `void` transform +3. Only add partition fields at the end of the previous partition spec + + ### Sorting Users can sort their data within partitions by columns to gain performance. The information on how the data is sorted can be declared per data or delete file, by a **sort order**. @@ -287,8 +343,9 @@ A manifest file must store the partition spec and other metadata as properties i | v1 | v2 | Key | Value | |------------|------------|---------------------|------------------------------------------------------------------------------| | _required_ | _required_ | `schema` | JSON representation of the table schema at the time the manifest was written | +| _optional_ | _required_ | `schema-id` | ID of the schema used to write the manifest as a string | | _required_ | _required_ | `partition-spec` | JSON fields representation of the partition spec used to write the manifest | -| _optional_ | _required_ | `partition-spec-id` | Id of the partition spec used to write the manifest as a string | +| _optional_ | _required_ | `partition-spec-id` | ID of the partition spec used to write the manifest as a string | | _optional_ | _required_ | `format-version` | Table format version number of the manifest as a string | | | _required_ | `content` | Type of content files tracked by the manifest: "data" or "deletes" | @@ -308,7 +365,7 @@ The schema of a manifest file is a struct called `manifest_entry` with the follo | | _required_ | **`134 content`** | `int` with meaning: `0: DATA`, `1: POSITION DELETES`, `2: EQUALITY DELETES` | Type of content stored by the data file: data, equality deletes, or position deletes (all v1 files are data files) | | _required_ | _required_ | **`100 file_path`** | `string` | Full URI for the file with FS scheme | | _required_ | _required_ | **`101 file_format`** | `string` | String file format name, avro, orc or parquet | -| _required_ | _required_ | **`102 partition`** | `struct<...>` | Partition data tuple, schema based on the partition spec | +| _required_ | _required_ | **`102 partition`** | `struct<...>` | Partition data tuple, schema based on the partition spec output using partition field ids for the struct field ids | | _required_ | _required_ | **`103 record_count`** | `long` | Number of records in this file | | _required_ | _required_ | **`104 file_size_in_bytes`** | `long` | Total file size in bytes | | _required_ | | ~~**`105 block_size_in_bytes`**~~ | `long` | **Deprecated. Always write a default in v1. Do not write in v2.** | @@ -378,6 +435,7 @@ A snapshot consists of the following fields: | _optional_ | _required_ | **`manifest-list`** | The location of a manifest list for this snapshot that tracks manifest files with additional meadata | | _optional_ | | **`manifests`** | A list of manifest file locations. Must be omitted if `manifest-list` is present | | _optional_ | _required_ | **`summary`** | A string map that summarizes the snapshot changes, including `operation` (see below) | +| _optional_ | _optional_ | **`schema-id`** | ID of the table's current schema when the snapshot was created | The snapshot summary's `operation` field is used by some operations, like snapshot expiration, to skip processing certain snapshots. Possible `operation` values are: @@ -425,6 +483,7 @@ Manifest list files store `manifest_file`, a struct with the following fields: | _optional_ | _required_ | **`513 existing_rows_count`** | `long` | Number of rows in all of files in the manifest that have status `EXISTING`, when `null` this is assumed to be non-zero | | _optional_ | _required_ | **`514 deleted_rows_count`** | `long` | Number of rows in all of files in the manifest that have status `DELETED`, when `null` this is assumed to be non-zero | | _optional_ | _optional_ | **`507 partitions`** | `list<508: field_summary>` (see below) | A list of field summaries for each partition field in the spec. Each field in the list corresponds to a field in the manifest file’s partition spec. | +| _optional_ | _optional_ | **`519 key_metadata`** | `binary` | Implementation-specific key metadata for encryption | `field_summary` is a struct with the following fields: @@ -495,10 +554,12 @@ Table metadata consists of the following fields: | | _required_ | **`last-sequence-number`**| The table's highest assigned sequence number, a monotonically increasing long that tracks the order of snapshots in a table. | | _required_ | _required_ | **`last-updated-ms`**| Timestamp in milliseconds from the unix epoch when the table was last updated. Each table metadata file should update this field just before writing. | | _required_ | _required_ | **`last-column-id`**| An integer; the highest assigned column ID for the table. This is used to ensure columns are always assigned an unused ID when evolving schemas. | -| _required_ | _required_ | **`schema`**| The table’s current schema. | -| _required_ | | **`partition-spec`**| The table’s current partition spec, stored as only fields. Note that this is used by writers to partition data, but is not used when reading because reads use the specs stored in manifest files. (**Deprecated**: use `partition-specs` and `default-spec-id`instead ) | +| _required_ | | **`schema`**| The table’s current schema. (**Deprecated**: use `schemas` and `current-schema-id` instead) | +| _optional_ | _required_ | **`schemas`**| A list of schemas, stored as objects with `schema-id`. | +| _optional_ | _required_ | **`current-schema-id`**| ID of the table's current schema. | +| _required_ | | **`partition-spec`**| The table’s current partition spec, stored as only fields. Note that this is used by writers to partition data, but is not used when reading because reads use the specs stored in manifest files. (**Deprecated**: use `partition-specs` and `default-spec-id` instead) | | _optional_ | _required_ | **`partition-specs`**| A list of partition specs, stored as full partition spec objects. | -| _optional_ | _required_ | **`default-spec-id`**| ID of the “current” spec that writers should use by default. | +| _optional_ | _required_ | **`default-spec-id`**| ID of the "current" spec that writers should use by default. | | _optional_ | _required_ | **`last-partition-id`**| An integer; the highest assigned partition field ID across all partition specs for the table. This is used to ensure partition fields are always assigned an unused ID when evolving specs. | | _optional_ | _optional_ | **`properties`**| A string to string map of table properties. This is used to control settings that affect reading and writing and is not intended to be used for arbitrary metadata. For example, `commit.retry.num-retries` is used to control the number of commit retries. | | _optional_ | _optional_ | **`current-snapshot-id`**| `long` ID of the current table snapshot. | @@ -595,7 +656,7 @@ The rows in the delete file must be sorted by `file_path` then `position` to opt Equality delete files identify deleted rows in a collection of data files by one or more column values, and may optionally contain additional columns of the deleted row. -Equality delete files store any subset of a table's columns and use the table's field ids. The _delete columns_ are the columns of the delete file used to match data rows. Delete columns are identified by id in the delete file [metadata column `equality_ids`](#manifests). +Equality delete files store any subset of a table's columns and use the table's field ids. The _delete columns_ are the columns of the delete file used to match data rows. Delete columns are identified by id in the delete file [metadata column `equality_ids`](#manifests). Float and double columns cannot be used as delete columns in equality delete files. A data row is deleted if its values are equal to all delete columns for any row in an equality delete file that applies to the row's data file (see [`Scan Planning`](#scan-planning)). @@ -810,7 +871,14 @@ Hash results are not dependent on decimal scale, which is part of the type, not ### Schemas -Schemas are serialized to JSON as a struct. Types are serialized according to this table: +Schemas are serialized as a JSON object with the same fields as a struct in the table below, and the following additional fields: + +| v1 | v2 |Field|JSON representation|Example| +| ---------- | ---------- |--- |--- |--- | +| _optional_ | _required_ |**`schema-id`**|`JSON int`|`0`| +| _optional_ | _optional_ |**`identifier-field-ids`**|`JSON list of ints`|`[1, 2]`| + +Types are serialized according to this table: |Type|JSON representation|Example| |--- |--- |--- | @@ -857,6 +925,7 @@ Each partition field in the fields list is stored as an object. See the table fo In some cases partition specs are stored using only the field list instead of the object format that includes the spec ID, like the deprecated `partition-spec` field in table metadata. The object format should be used unless otherwise noted in this spec. +The `field-id` property was added for each partition field in v2. In v1, the reference implementation assigned field ids sequentially in each spec starting at 1,000. See Partition Evolution for more details. ### Sort Orders @@ -892,14 +961,16 @@ Table metadata is serialized as a JSON object according to the following table. |**`location`**|`JSON string`|`"s3://b/wh/data.db/table"`| |**`last-updated-ms`**|`JSON long`|`1515100955770`| |**`last-column-id`**|`JSON int`|`22`| -|**`schema`**|`JSON schema (object)`|`See above`| +|**`schema`**|`JSON schema (object)`|`See above, read schemas instead`| +|**`schemas`**|`JSON schemas (list of objects)`|`See above`| +|**`current-schema-id`**|`JSON int`|`0`| |**`partition-spec`**|`JSON partition fields (list)`|`See above, read partition-specs instead`| |**`partition-specs`**|`JSON partition specs (list of objects)`|`See above`| |**`default-spec-id`**|`JSON int`|`0`| |**`last-partition-id`**|`JSON int`|`1000`| |**`properties`**|`JSON object: {`
        `"": "",`
        `...`
      `}`|`{`
        `"write.format.default": "avro",`
        `"commit.retry.num-retries": "4"`
      `}`| |**`current-snapshot-id`**|`JSON long`|`3051729675574597004`| -|**`snapshots`**|`JSON list of objects: [ {`
        `"snapshot-id": ,`
        `"timestamp-ms": ,`
        `"summary": {`
          `"operation": ,`
          `... },`
        `"manifest-list": ""`
        `},`
        `...`
      `]`|`[ {`
        `"snapshot-id": 3051729675574597004,`
        `"timestamp-ms": 1515100955770,`
        `"summary": {`
          `"operation": "append"`
        `},`
        `"manifest-list": "s3://b/wh/.../s1.avro"`
      `} ]`| +|**`snapshots`**|`JSON list of objects: [ {`
        `"snapshot-id": ,`
        `"timestamp-ms": ,`
        `"summary": {`
          `"operation": ,`
          `... },`
        `"manifest-list": "",`
        `"schema-id": ""`
        `},`
        `...`
      `]`|`[ {`
        `"snapshot-id": 3051729675574597004,`
        `"timestamp-ms": 1515100955770,`
        `"summary": {`
          `"operation": "append"`
        `},`
        `"manifest-list": "s3://b/wh/.../s1.avro"`
        `"schema-id": 0`
      `} ]`| |**`snapshot-log`**|`JSON list of objects: [`
        `{`
        `"snapshot-id": ,`
        `"timestamp-ms": `
        `},`
        `...`
      `]`|`[ {`
        `"snapshot-id": 30517296...,`
        `"timestamp-ms": 1515100...`
      `} ]`| |**`metadata-log`**|`JSON list of objects: [`
        `{`
        `"metadata-file": ,`
        `"timestamp-ms": `
        `},`
        `...`
      `]`|`[ {`
        `"metadata-file": "s3://bucket/.../v1.json",`
        `"timestamp-ms": 1515100...`
      `} ]` | |**`sort-orders`**|`JSON sort orders (list of sort field object)`|`See above`| @@ -937,26 +1008,68 @@ This serialization scheme is for storing single values as individual binary valu Writing v1 metadata: -* Table metadata field `last-sequence-number` should not be written. -* Snapshot field `sequence-number` should not be written. +* Table metadata field `last-sequence-number` should not be written +* Snapshot field `sequence-number` should not be written +* Manifest list field `sequence-number` should not be written +* Manifest list field `min-sequence-number` should not be written +* Manifest list field `content` must be 0 (data) or omitted +* Manifest entry field `sequence_number` should not be written +* Data file field `content` must be 0 (data) or omitted -Reading v1 metadata: +Reading v1 metadata for v2: -* Table metadata field `last-sequence-number` must default to 0. -* Snapshot field `sequence-number` must default to 0. +* Table metadata field `last-sequence-number` must default to 0 +* Snapshot field `sequence-number` must default to 0 +* Manifest list field `sequence-number` must default to 0 +* Manifest list field `min-sequence-number` must default to 0 +* Manifest list field `content` must default to 0 (data) +* Manifest entry field `sequence_number` must default to 0 +* Data file field `content` must default to 0 (data) Writing v2 metadata: -* Table metadata added required field `last-sequence-number`. -* Table metadata now requires field `table-uuid`. -* Table metadata now requires field `partition-specs`. -* Table metadata now requires field `default-spec-id`. -* Table metadata now requires field `last-partition-id`. -* Table metadata field `partition-spec` is no longer required and may be omitted. -* Snapshot added required field `sequence-number`. -* Snapshot now requires field `manifest-list`. -* Snapshot field `manifests` is no longer allowed. -* Table metadata now requires field `sort-orders`. -* Table metadata now requires field `default-sort-order-id`. +* Table metadata JSON: + * `last-sequence-number` was added and is required; default to 0 when reading v1 metadata + * `table-uuid` is now required + * `current-schema-id` is now required + * `schemas` is now required + * `partition-specs` is now required + * `default-spec-id` is now required + * `last-partition-id` is now required + * `sort-orders` is now required + * `default-sort-order-id` is now required + * `schema` is no longer required and should be omitted; use `schemas` and `current-schema-id` instead + * `partition-spec` is no longer required and should be omitted; use `partition-specs` and `default-spec-id` instead +* Snapshot JSON: + * `sequence-number` was added and is required; default to 0 when reading v1 metadata + * `manifest-list` is now required + * `manifests` is no longer required and should be omitted; always use `manifest-list` instead +* Manifest list `manifest_file`: + * `content` was added and is required; 0=data, 1=deletes; default to 0 when reading v1 manifest lists + * `sequence_number` was added and is required + * `min_sequence_number` was added and is required + * `added_files_count` is now required + * `existing_files_count` is now required + * `deleted_files_count` is now required + * `added_rows_count` is now required + * `existing_rows_count` is now required + * `deleted_rows_count` is now required +* Manifest list `field_summary`: + * `contains_nan` is now required +* Manifest key-value metadata: + * `schema-id` is now required + * `partition-spec-id` is now required + * `format-version` is now required + * `content` was added and is required (must be "data" or "deletes") +* Manifest `manifest_entry`: + * `snapshot_id` is now optional to support inheritance + * `sequence_number` was added and is optional, to support inheritance +* Manifest `data_file`: + * `content` was added and is required; 0=data, 1=position deletes, 2=equality deletes; default to 0 when reading v1 manifests + * `equality_ids` was added, to be used for equality deletes only + * `block_size_in_bytes` was removed (breaks v1 reader compatibility) + * `file_ordinal` was removed + * `sort_columns` was removed + * `distinct_counts` was removed Note that these requirements apply when writing data to a v2 table. Tables that are upgraded from v1 may contain metadata that does not follow these requirements. Implementations should remain backward-compatible with v1 metadata requirements. diff --git a/site/mkdocs.yml b/site/mkdocs.yml index 73f9167786b4..dbda3f3764d1 100644 --- a/site/mkdocs.yml +++ b/site/mkdocs.yml @@ -18,6 +18,7 @@ # site_name: Apache Iceberg +site_url: https://iceberg.apache.org/ site_description: A table format for large, slow-moving tabular data remote_name: apache @@ -32,7 +33,10 @@ extra: versions: iceberg: 0.11.1 plugins: - - redirects + - redirects: + redirect_maps: + 'time-travel.md': 'spark-queries/#time-travel' + 'presto.md': 'trino.md' - markdownextradata markdown_extensions: - toc: @@ -67,7 +71,7 @@ nav: - Writes: spark-writes.md - Maintenance Procedures: spark-procedures.md - Structured Streaming: spark-structured-streaming.md - - Time Travel: spark#time-travel + - Time Travel: spark-queries/#time-travel - Trino: https://trino.io/docs/current/connector/iceberg.html - Flink: flink.md - Hive: hive.md @@ -92,6 +96,3 @@ nav: - Sponsors: https://www.apache.org/foundation/thanks.html - Donate: https://www.apache.org/foundation/sponsorship.html - Events: https://www.apache.org/events/current-event.html -redirects: - time-travel/index: snapshots/index - presto/index: trino/index diff --git a/spark2/src/jmh/java/org/apache/iceberg/spark/SparkBenchmarkUtil.java b/spark/src/jmh/java/org/apache/iceberg/spark/SparkBenchmarkUtil.java similarity index 100% rename from spark2/src/jmh/java/org/apache/iceberg/spark/SparkBenchmarkUtil.java rename to spark/src/jmh/java/org/apache/iceberg/spark/SparkBenchmarkUtil.java diff --git a/spark2/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersFlatDataBenchmark.java b/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersFlatDataBenchmark.java similarity index 98% rename from spark2/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersFlatDataBenchmark.java rename to spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersFlatDataBenchmark.java index 35fc88fc1b38..24d07ccf55f1 100644 --- a/spark2/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersFlatDataBenchmark.java +++ b/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersFlatDataBenchmark.java @@ -59,9 +59,9 @@ * A benchmark that evaluates the performance of reading Parquet data with a flat schema using * Iceberg and Spark Parquet readers. * - * To run this benchmark: + * To run this benchmark for either spark-2 or spark-3: * - * ./gradlew :iceberg-spark2:jmh + * ./gradlew :iceberg-spark[2|3]:jmh * -PjmhIncludeRegex=SparkParquetReadersFlatDataBenchmark * -PjmhOutputPath=benchmark/spark-parquet-readers-flat-data-benchmark-result.txt * diff --git a/spark2/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersNestedDataBenchmark.java b/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersNestedDataBenchmark.java similarity index 98% rename from spark2/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersNestedDataBenchmark.java rename to spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersNestedDataBenchmark.java index 76ab16757c1c..d98ee6f566e7 100644 --- a/spark2/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersNestedDataBenchmark.java +++ b/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersNestedDataBenchmark.java @@ -59,9 +59,9 @@ * A benchmark that evaluates the performance of reading nested Parquet data using * Iceberg and Spark Parquet readers. * - * To run this benchmark: + * To run this benchmark for either spark-2 or spark-3: * - * ./gradlew :iceberg-spark2:jmh + * ./gradlew :iceberg-spark[2|3]:jmh * -PjmhIncludeRegex=SparkParquetReadersNestedDataBenchmark * -PjmhOutputPath=benchmark/spark-parquet-readers-nested-data-benchmark-result.txt * diff --git a/spark2/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersFlatDataBenchmark.java b/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersFlatDataBenchmark.java similarity index 97% rename from spark2/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersFlatDataBenchmark.java rename to spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersFlatDataBenchmark.java index eadb3d72b339..95c9d6f90119 100644 --- a/spark2/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersFlatDataBenchmark.java +++ b/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersFlatDataBenchmark.java @@ -51,9 +51,9 @@ * A benchmark that evaluates the performance of writing Parquet data with a flat schema using * Iceberg and Spark Parquet writers. * - * To run this benchmark: + * To run this benchmark for either spark-2 or spark-3: * - * ./gradlew :iceberg-spark2:jmh + * ./gradlew :iceberg-spark[2|3]:jmh * -PjmhIncludeRegex=SparkParquetWritersFlatDataBenchmark * -PjmhOutputPath=benchmark/spark-parquet-writers-flat-data-benchmark-result.txt * diff --git a/spark2/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersNestedDataBenchmark.java b/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersNestedDataBenchmark.java similarity index 97% rename from spark2/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersNestedDataBenchmark.java rename to spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersNestedDataBenchmark.java index a8748370d063..989d63547117 100644 --- a/spark2/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersNestedDataBenchmark.java +++ b/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersNestedDataBenchmark.java @@ -51,9 +51,9 @@ * A benchmark that evaluates the performance of writing nested Parquet data using * Iceberg and Spark Parquet writers. * - * To run this benchmark: + * To run this benchmark for either spark-2 or spark-3: * - * ./gradlew :iceberg-spark2:jmh + * ./gradlew :iceberg-spark[2|3]:jmh * -PjmhIncludeRegex=SparkParquetWritersNestedDataBenchmark * -PjmhOutputPath=benchmark/spark-parquet-writers-nested-data-benchmark-result.txt * diff --git a/spark2/src/jmh/java/org/apache/iceberg/spark/source/Action.java b/spark/src/jmh/java/org/apache/iceberg/spark/source/Action.java similarity index 100% rename from spark2/src/jmh/java/org/apache/iceberg/spark/source/Action.java rename to spark/src/jmh/java/org/apache/iceberg/spark/source/Action.java diff --git a/spark2/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceBenchmark.java b/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceBenchmark.java similarity index 100% rename from spark2/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceBenchmark.java rename to spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceBenchmark.java diff --git a/spark2/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceFlatDataBenchmark.java b/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceFlatDataBenchmark.java similarity index 100% rename from spark2/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceFlatDataBenchmark.java rename to spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceFlatDataBenchmark.java diff --git a/spark2/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceNestedDataBenchmark.java b/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceNestedDataBenchmark.java similarity index 100% rename from spark2/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceNestedDataBenchmark.java rename to spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceNestedDataBenchmark.java diff --git a/spark2/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceNestedListDataBenchmark.java b/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceNestedListDataBenchmark.java similarity index 100% rename from spark2/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceNestedListDataBenchmark.java rename to spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceNestedListDataBenchmark.java diff --git a/spark2/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceFlatAvroDataReadBenchmark.java b/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceFlatAvroDataReadBenchmark.java similarity index 98% rename from spark2/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceFlatAvroDataReadBenchmark.java rename to spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceFlatAvroDataReadBenchmark.java index c0957bc9af9f..b8fcc13a3080 100644 --- a/spark2/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceFlatAvroDataReadBenchmark.java +++ b/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceFlatAvroDataReadBenchmark.java @@ -41,9 +41,9 @@ * A benchmark that evaluates the performance of reading Avro data with a flat schema * using Iceberg and the built-in file source in Spark. * - * To run this benchmark: + * To run this benchmark for either spark-2 or spark-3: * - * ./gradlew :iceberg-spark2:jmh + * ./gradlew :iceberg-spark[2|3]:jmh * -PjmhIncludeRegex=IcebergSourceFlatAvroDataReadBenchmark * -PjmhOutputPath=benchmark/iceberg-source-flat-avro-data-read-benchmark-result.txt * diff --git a/spark2/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceNestedAvroDataReadBenchmark.java b/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceNestedAvroDataReadBenchmark.java similarity index 97% rename from spark2/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceNestedAvroDataReadBenchmark.java rename to spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceNestedAvroDataReadBenchmark.java index 1fffded981f2..8a9ed0df1c91 100644 --- a/spark2/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceNestedAvroDataReadBenchmark.java +++ b/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceNestedAvroDataReadBenchmark.java @@ -42,9 +42,9 @@ * A benchmark that evaluates the performance of reading Avro data with a flat schema * using Iceberg and the built-in file source in Spark. * - * To run this benchmark: + * To run this benchmark for either spark-2 or spark-3: * - * ./gradlew :iceberg-spark2:jmh + * ./gradlew :iceberg-spark[2|3]:jmh * -PjmhIncludeRegex=IcebergSourceNestedAvroDataReadBenchmark * -PjmhOutputPath=benchmark/iceberg-source-nested-avro-data-read-benchmark-result.txt * diff --git a/spark2/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataBenchmark.java b/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataBenchmark.java similarity index 100% rename from spark2/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataBenchmark.java rename to spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataBenchmark.java diff --git a/spark2/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataReadBenchmark.java b/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataReadBenchmark.java similarity index 98% rename from spark2/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataReadBenchmark.java rename to spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataReadBenchmark.java index 0ed8b6b07339..8cbdc4b89a36 100644 --- a/spark2/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataReadBenchmark.java +++ b/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataReadBenchmark.java @@ -41,9 +41,9 @@ * A benchmark that evaluates the performance of reading ORC data with a flat schema * using Iceberg and the built-in file source in Spark. * - * To run this benchmark: + * To run this benchmark for either spark-2 or spark-3: * - * ./gradlew :iceberg-spark2:jmh + * ./gradlew :iceberg-spark[2|3]:jmh * -PjmhIncludeRegex=IcebergSourceFlatORCDataReadBenchmark * -PjmhOutputPath=benchmark/iceberg-source-flat-orc-data-read-benchmark-result.txt * diff --git a/spark2/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedListORCDataWriteBenchmark.java b/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedListORCDataWriteBenchmark.java similarity index 97% rename from spark2/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedListORCDataWriteBenchmark.java rename to spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedListORCDataWriteBenchmark.java index a1106cb88111..53c747962dad 100644 --- a/spark2/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedListORCDataWriteBenchmark.java +++ b/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedListORCDataWriteBenchmark.java @@ -40,9 +40,9 @@ * A benchmark that evaluates the performance of writing nested Parquet data using Iceberg * and the built-in file source in Spark. * - * To run this benchmark: + * To run this benchmark for either spark-2 or spark-3: * - * ./gradlew :iceberg-spark2:jmh + * ./gradlew :iceberg-spark[2|3]:jmh * -PjmhIncludeRegex=IcebergSourceNestedListORCDataWriteBenchmark * -PjmhOutputPath=benchmark/iceberg-source-nested-list-orc-data-write-benchmark-result.txt * diff --git a/spark2/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedORCDataReadBenchmark.java b/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedORCDataReadBenchmark.java similarity index 98% rename from spark2/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedORCDataReadBenchmark.java rename to spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedORCDataReadBenchmark.java index bfbf4cb64815..f88fe3c95141 100644 --- a/spark2/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedORCDataReadBenchmark.java +++ b/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedORCDataReadBenchmark.java @@ -43,9 +43,9 @@ * A benchmark that evaluates the performance of reading ORC data with a flat schema * using Iceberg and the built-in file source in Spark. * - * To run this benchmark: + * To run this benchmark for either spark-2 or spark-3: * - * ./gradlew :iceberg-spark2:jmh + * ./gradlew :iceberg-spark[2|3]:jmh * -PjmhIncludeRegex=IcebergSourceNestedORCDataReadBenchmark * -PjmhOutputPath=benchmark/iceberg-source-nested-orc-data-read-benchmark-result.txt * diff --git a/spark2/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataFilterBenchmark.java b/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataFilterBenchmark.java similarity index 97% rename from spark2/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataFilterBenchmark.java rename to spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataFilterBenchmark.java index 8860d2f2e0b3..ed42069e25ee 100644 --- a/spark2/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataFilterBenchmark.java +++ b/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataFilterBenchmark.java @@ -44,9 +44,9 @@ * * The performance is compared to the built-in file source in Spark. * - * To run this benchmark: + * To run this benchmark for either spark-2 or spark-3: * - * ./gradlew :iceberg-spark2:jmh + * ./gradlew :iceberg-spark[2|3]:jmh * -PjmhIncludeRegex=IcebergSourceFlatParquetDataFilterBenchmark * -PjmhOutputPath=benchmark/iceberg-source-flat-parquet-data-filter-benchmark-result.txt * diff --git a/spark2/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataReadBenchmark.java b/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataReadBenchmark.java similarity index 98% rename from spark2/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataReadBenchmark.java rename to spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataReadBenchmark.java index 51fb314c52ae..675a4de73487 100644 --- a/spark2/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataReadBenchmark.java +++ b/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataReadBenchmark.java @@ -40,9 +40,9 @@ * A benchmark that evaluates the performance of reading Parquet data with a flat schema * using Iceberg and the built-in file source in Spark. * - * To run this benchmark: + * To run this benchmark for either spark-2 or spark-3: * - * ./gradlew :iceberg-spark2:jmh + * ./gradlew :iceberg-spark[2|3]:jmh * -PjmhIncludeRegex=IcebergSourceFlatParquetDataReadBenchmark * -PjmhOutputPath=benchmark/iceberg-source-flat-parquet-data-read-benchmark-result.txt * diff --git a/spark2/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataWriteBenchmark.java b/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataWriteBenchmark.java similarity index 97% rename from spark2/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataWriteBenchmark.java rename to spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataWriteBenchmark.java index 066f1bce8210..b0b2cf27fb4a 100644 --- a/spark2/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataWriteBenchmark.java +++ b/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataWriteBenchmark.java @@ -38,9 +38,9 @@ * A benchmark that evaluates the performance of writing Parquet data with a flat schema * using Iceberg and the built-in file source in Spark. * - * To run this benchmark: + * To run this benchmark for either spark-2 or spark-3: * - * ./gradlew :iceberg-spark2:jmh + * ./gradlew :iceberg-spark[2|3]:jmh * -PjmhIncludeRegex=IcebergSourceFlatParquetDataWriteBenchmark * -PjmhOutputPath=benchmark/iceberg-source-flat-parquet-data-write-benchmark-result.txt * diff --git a/spark2/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedListParquetDataWriteBenchmark.java b/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedListParquetDataWriteBenchmark.java similarity index 96% rename from spark2/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedListParquetDataWriteBenchmark.java rename to spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedListParquetDataWriteBenchmark.java index 160b3707c537..b45be33357bc 100644 --- a/spark2/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedListParquetDataWriteBenchmark.java +++ b/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedListParquetDataWriteBenchmark.java @@ -41,9 +41,9 @@ * A benchmark that evaluates the performance of writing nested Parquet data using Iceberg * and the built-in file source in Spark. * - * To run this benchmark: + * To run this benchmark for either spark-2 or spark-3: * - * ./gradlew :iceberg-spark2:jmh + * ./gradlew :iceberg-spark[2|3]:jmh * -PjmhIncludeRegex=IcebergSourceNestedListParquetDataWriteBenchmark * -PjmhOutputPath=benchmark/iceberg-source-nested-list-parquet-data-write-benchmark-result.txt * diff --git a/spark2/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataFilterBenchmark.java b/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataFilterBenchmark.java similarity index 97% rename from spark2/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataFilterBenchmark.java rename to spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataFilterBenchmark.java index 398dc162c7ba..3cee9d54cd5d 100644 --- a/spark2/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataFilterBenchmark.java +++ b/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataFilterBenchmark.java @@ -44,9 +44,9 @@ * * The performance is compared to the built-in file source in Spark. * - * To run this benchmark: + * To run this benchmark for either spark-2 or spark-3: * - * ./gradlew :iceberg-spark2:jmh + * ./gradlew :iceberg-spark[2|3]:jmh * -PjmhIncludeRegex=IcebergSourceNestedParquetDataFilterBenchmark * -PjmhOutputPath=benchmark/iceberg-source-nested-parquet-data-filter-benchmark-result.txt * diff --git a/spark2/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataReadBenchmark.java b/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataReadBenchmark.java similarity index 98% rename from spark2/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataReadBenchmark.java rename to spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataReadBenchmark.java index 62ed4b61ae65..9ef0f1eae6ca 100644 --- a/spark2/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataReadBenchmark.java +++ b/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataReadBenchmark.java @@ -40,9 +40,9 @@ * A benchmark that evaluates the performance of reading nested Parquet data using Iceberg * and the built-in file source in Spark. * - * To run this benchmark: + * To run this benchmark for either spark-2 or spark-3: * - * ./gradlew :iceberg-spark2:jmh + * ./gradlew :iceberg-spark[2|3]:jmh * -PjmhIncludeRegex=IcebergSourceNestedParquetDataReadBenchmark * -PjmhOutputPath=benchmark/iceberg-source-nested-parquet-data-read-benchmark-result.txt * diff --git a/spark2/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataWriteBenchmark.java b/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataWriteBenchmark.java similarity index 96% rename from spark2/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataWriteBenchmark.java rename to spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataWriteBenchmark.java index bbaebed2c0f9..193a062f3899 100644 --- a/spark2/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataWriteBenchmark.java +++ b/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataWriteBenchmark.java @@ -39,9 +39,9 @@ * A benchmark that evaluates the performance of writing nested Parquet data using Iceberg * and the built-in file source in Spark. * - * To run this benchmark: + * To run this benchmark for either spark-2 or spark-3: * - * ./gradlew :iceberg-spark2:jmh + * ./gradlew :iceberg-spark[2|3]:jmh * -PjmhIncludeRegex=IcebergSourceNestedParquetDataWriteBenchmark * -PjmhOutputPath=benchmark/iceberg-source-nested-parquet-data-write-benchmark-result.txt * diff --git a/spark2/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.java b/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.java similarity index 97% rename from spark2/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.java rename to spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.java index a54b66da8007..94e522171e86 100644 --- a/spark2/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.java +++ b/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.java @@ -41,10 +41,11 @@ * Benchmark to compare performance of reading Parquet dictionary encoded data with a flat schema using vectorized * Iceberg read path and the built-in file source in Spark. *

      - * To run the benchmark: + * To run this benchmark for either spark-2 or spark-3: * - * ./gradlew :iceberg-spark2:jmh -PjmhIncludeRegex=VectorizedReadDictionaryEncodedFlatParquetDataBenchmark - * -PjmhOutputPath=benchmark/results.txt + * ./gradlew :iceberg-spark[2|3]:jmh + * -PjmhIncludeRegex=VectorizedReadDictionaryEncodedFlatParquetDataBenchmark + * -PjmhOutputPath=benchmark/results.txt * */ public class VectorizedReadDictionaryEncodedFlatParquetDataBenchmark extends VectorizedReadFlatParquetDataBenchmark { diff --git a/spark2/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadFlatParquetDataBenchmark.java b/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadFlatParquetDataBenchmark.java similarity index 98% rename from spark2/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadFlatParquetDataBenchmark.java rename to spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadFlatParquetDataBenchmark.java index 17230271451a..1b4121a47c54 100644 --- a/spark2/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadFlatParquetDataBenchmark.java +++ b/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadFlatParquetDataBenchmark.java @@ -51,10 +51,11 @@ * Benchmark to compare performance of reading Parquet data with a flat schema using vectorized Iceberg read path and * the built-in file source in Spark. *

      - * To run the benchmark: + * To run this benchmark for either spark-2 or spark-3: * - * ./gradlew :iceberg-spark2:jmh -PjmhIncludeRegex=VectorizedReadFlatParquetDataBenchmark - * -PjmhOutputPath=benchmark/results.txt + * ./gradlew :iceberg-spark[2|3]:jmh + * -PjmhIncludeRegex=VectorizedReadFlatParquetDataBenchmark + * -PjmhOutputPath=benchmark/results.txt * */ public class VectorizedReadFlatParquetDataBenchmark extends IcebergSourceBenchmark { diff --git a/spark/src/main/java/org/apache/iceberg/actions/Actions.java b/spark/src/main/java/org/apache/iceberg/actions/Actions.java index d1b7d778cc7c..1fc019294a89 100644 --- a/spark/src/main/java/org/apache/iceberg/actions/Actions.java +++ b/spark/src/main/java/org/apache/iceberg/actions/Actions.java @@ -22,11 +22,17 @@ import org.apache.iceberg.Table; import org.apache.iceberg.common.DynConstructors; import org.apache.iceberg.common.DynMethods; +import org.apache.iceberg.spark.actions.BaseDeleteOrphanFilesSparkAction; import org.apache.iceberg.spark.actions.BaseExpireSnapshotsSparkAction; -import org.apache.iceberg.spark.actions.BaseRemoveOrphanFilesSparkAction; import org.apache.iceberg.spark.actions.BaseRewriteManifestsSparkAction; import org.apache.spark.sql.SparkSession; +/** + * An API for interacting with actions in Spark. + * + * @deprecated since 0.12.0, will be removed in 0.13.0; use an implementation of {@link ActionsProvider} instead. + */ +@Deprecated public class Actions { /* @@ -61,28 +67,52 @@ protected Actions(SparkSession spark, Table table) { this.table = table; } + /** + * @deprecated since 0.12.0, will be removed in 0.13.0; use an implementation of {@link ActionsProvider} instead. + */ + @Deprecated public static Actions forTable(SparkSession spark, Table table) { return actionConstructor().newInstance(spark, table); } + /** + * @deprecated since 0.12.0, will be removed in 0.13.0; use an implementation of {@link ActionsProvider} instead. + */ + @Deprecated public static Actions forTable(Table table) { return forTable(SparkSession.active(), table); } + /** + * @deprecated since 0.12.0, will be removed in 0.13.0; use {@link DeleteOrphanFiles} instead. + */ + @Deprecated public RemoveOrphanFilesAction removeOrphanFiles() { - BaseRemoveOrphanFilesSparkAction delegate = new BaseRemoveOrphanFilesSparkAction(spark, table); + BaseDeleteOrphanFilesSparkAction delegate = new BaseDeleteOrphanFilesSparkAction(spark, table); return new RemoveOrphanFilesAction(delegate); } + /** + * @deprecated since 0.12.0, will be removed in 0.13.0; use {@link RewriteManifests} instead. + */ + @Deprecated public RewriteManifestsAction rewriteManifests() { BaseRewriteManifestsSparkAction delegate = new BaseRewriteManifestsSparkAction(spark, table); return new RewriteManifestsAction(delegate); } + /** + * @deprecated since 0.12.0, will be removed in 0.13.0; use {@link RewriteDataFiles} instead. + */ + @Deprecated public RewriteDataFilesAction rewriteDataFiles() { return new RewriteDataFilesAction(spark, table); } + /** + * @deprecated since 0.12.0, will be removed in 0.13.0; use {@link ExpireSnapshots} instead. + */ + @Deprecated public ExpireSnapshotsAction expireSnapshots() { BaseExpireSnapshotsSparkAction delegate = new BaseExpireSnapshotsSparkAction(spark, table); return new ExpireSnapshotsAction(delegate); @@ -94,7 +124,9 @@ public ExpireSnapshotsAction expireSnapshots() { * * @param tableName Table to be converted * @return {@link CreateAction} to perform migration + * @deprecated since 0.12.0, will be removed in 0.13.0; use {@link MigrateTable} instead. */ + @Deprecated public static CreateAction migrate(String tableName) { try { return DynMethods.builder("migrate") @@ -112,7 +144,9 @@ public static CreateAction migrate(String tableName) { * @param tableName Table to be converted * @param spark Spark session to use for looking up table * @return {@link CreateAction} to perform migration + * @deprecated since 0.12.0, will be removed in 0.13.0; use {@link MigrateTable} instead. */ + @Deprecated public static CreateAction migrate(SparkSession spark, String tableName) { try { return DynMethods.builder("migrate") @@ -131,7 +165,9 @@ public static CreateAction migrate(SparkSession spark, String tableName) { * @param sourceTable Original table which is the basis for the new Iceberg table * @param destTable New Iceberg table being created * @return {@link SnapshotAction} to perform snapshot + * @deprecated since 0.12.0, will be removed in 0.13.0; use {@link SnapshotTable} instead. */ + @Deprecated public static SnapshotAction snapshot(SparkSession spark, String sourceTable, String destTable) { try { return DynMethods.builder("snapshot") @@ -150,7 +186,9 @@ public static SnapshotAction snapshot(SparkSession spark, String sourceTable, St * @param sourceTable Original table which is the basis for the new Iceberg table * @param destTable New Iceberg table being created * @return {@link SnapshotAction} to perform snapshot + * @deprecated since 0.12.0, will be removed in 0.13.0; use {@link SnapshotTable} instead. */ + @Deprecated public static SnapshotAction snapshot(String sourceTable, String destTable) { try { return DynMethods.builder("snapshot") diff --git a/spark/src/main/java/org/apache/iceberg/actions/CreateAction.java b/spark/src/main/java/org/apache/iceberg/actions/CreateAction.java index a567bda9139a..9f78167ff9c8 100644 --- a/spark/src/main/java/org/apache/iceberg/actions/CreateAction.java +++ b/spark/src/main/java/org/apache/iceberg/actions/CreateAction.java @@ -21,6 +21,9 @@ import java.util.Map; +/** + * @deprecated since 0.12.0, will be removed in 0.13.0; use {@link SnapshotTable} or {@link MigrateTable} instead. + */ @Deprecated public interface CreateAction extends Action { diff --git a/spark/src/main/java/org/apache/iceberg/actions/ManifestFileBean.java b/spark/src/main/java/org/apache/iceberg/actions/ManifestFileBean.java index f51499dcf1aa..4beb12c5e945 100644 --- a/spark/src/main/java/org/apache/iceberg/actions/ManifestFileBean.java +++ b/spark/src/main/java/org/apache/iceberg/actions/ManifestFileBean.java @@ -19,6 +19,7 @@ package org.apache.iceberg.actions; +import java.nio.ByteBuffer; import java.util.List; import org.apache.iceberg.ManifestContent; import org.apache.iceberg.ManifestFile; @@ -131,6 +132,11 @@ public List partitions() { return null; } + @Override + public ByteBuffer keyMetadata() { + return null; + } + @Override public ManifestFile copy() { throw new UnsupportedOperationException("Cannot copy"); diff --git a/spark/src/main/java/org/apache/iceberg/actions/RemoveOrphanFilesAction.java b/spark/src/main/java/org/apache/iceberg/actions/RemoveOrphanFilesAction.java index a3298ac1b41f..ac03b5f06933 100644 --- a/spark/src/main/java/org/apache/iceberg/actions/RemoveOrphanFilesAction.java +++ b/spark/src/main/java/org/apache/iceberg/actions/RemoveOrphanFilesAction.java @@ -39,13 +39,13 @@ * Note: It is dangerous to call this action with a short retention interval as it might corrupt * the state of the table if another operation is writing at the same time. * - * @deprecated since 0.12.0, will be removed in 0.13.0; use {@link RemoveOrphanFiles} instead. + * @deprecated since 0.12.0, will be removed in 0.13.0; use {@link DeleteOrphanFiles} instead. */ @Deprecated public class RemoveOrphanFilesAction implements Action> { - private final RemoveOrphanFiles delegate; + private final DeleteOrphanFiles delegate; - RemoveOrphanFilesAction(RemoveOrphanFiles delegate) { + RemoveOrphanFilesAction(DeleteOrphanFiles delegate) { this.delegate = delegate; } @@ -84,7 +84,7 @@ public RemoveOrphanFilesAction deleteWith(Consumer newDeleteFunc) { @Override public List execute() { - RemoveOrphanFiles.Result result = delegate.execute(); + DeleteOrphanFiles.Result result = delegate.execute(); return ImmutableList.copyOf(result.orphanFileLocations()); } } diff --git a/spark/src/main/java/org/apache/iceberg/actions/RewriteDataFilesAction.java b/spark/src/main/java/org/apache/iceberg/actions/RewriteDataFilesAction.java index 735e7190e3ee..c01c59220086 100644 --- a/spark/src/main/java/org/apache/iceberg/actions/RewriteDataFilesAction.java +++ b/spark/src/main/java/org/apache/iceberg/actions/RewriteDataFilesAction.java @@ -32,6 +32,10 @@ import org.apache.spark.broadcast.Broadcast; import org.apache.spark.sql.SparkSession; +/** + * @deprecated since 0.12.0, will be removed in 0.13.0; use {@link RewriteDataFilesAction} instead. + */ +@Deprecated public class RewriteDataFilesAction extends BaseRewriteDataFilesAction { @@ -40,7 +44,7 @@ public class RewriteDataFilesAction RewriteDataFilesAction(SparkSession spark, Table table) { super(table); - this.sparkContext = new JavaSparkContext(spark.sparkContext()); + this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); } @Override diff --git a/spark/src/main/java/org/apache/iceberg/actions/SnapshotAction.java b/spark/src/main/java/org/apache/iceberg/actions/SnapshotAction.java index 6c3e8e6d31a5..34450d0c933a 100644 --- a/spark/src/main/java/org/apache/iceberg/actions/SnapshotAction.java +++ b/spark/src/main/java/org/apache/iceberg/actions/SnapshotAction.java @@ -19,6 +19,9 @@ package org.apache.iceberg.actions; +/** + * @deprecated since 0.12.0, will be removed in 0.13.0; use {@link SnapshotTable} instead. + */ @Deprecated public interface SnapshotAction extends CreateAction { SnapshotAction withLocation(String location); diff --git a/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithoutReordering.java b/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithoutReordering.java index 653c27313ec3..c6984e2fe8cd 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithoutReordering.java +++ b/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithoutReordering.java @@ -202,11 +202,6 @@ public Type primitive(Type.PrimitiveType primitive) { "Cannot project decimal with incompatible precision: %s < %s", requestedDecimal.precision(), decimal.precision()); break; - case TIMESTAMP: - Types.TimestampType timestamp = (Types.TimestampType) primitive; - Preconditions.checkArgument(timestamp.shouldAdjustToUTC(), - "Cannot project timestamp (without time zone) as timestamptz (with time zone)"); - break; default: } diff --git a/spark/src/main/java/org/apache/iceberg/spark/SparkFixupTimestampType.java b/spark/src/main/java/org/apache/iceberg/spark/SparkFixupTimestampType.java new file mode 100644 index 000000000000..06282c8e1fe0 --- /dev/null +++ b/spark/src/main/java/org/apache/iceberg/spark/SparkFixupTimestampType.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.spark; + +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.FixupTypes; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; + +/** + * By default Spark type {@link org.apache.iceberg.types.Types.TimestampType} should be converted to + * {@link Types.TimestampType#withZone()} iceberg type. But we also can convert + * {@link org.apache.iceberg.types.Types.TimestampType} to {@link Types.TimestampType#withoutZone()} iceberg type + * by setting {@link SparkUtil#USE_TIMESTAMP_WITHOUT_TIME_ZONE_IN_NEW_TABLES} to 'true' + */ +class SparkFixupTimestampType extends FixupTypes { + + private SparkFixupTimestampType(Schema referenceSchema) { + super(referenceSchema); + } + + static Schema fixup(Schema schema) { + return new Schema(TypeUtil.visit(schema, + new SparkFixupTimestampType(schema)).asStructType().fields()); + } + + @Override + public Type primitive(Type.PrimitiveType primitive) { + if (primitive.typeId() == Type.TypeID.TIMESTAMP) { + return Types.TimestampType.withoutZone(); + } + return primitive; + } + + @Override + protected boolean fixupPrimitive(Type.PrimitiveType type, Type source) { + return Type.TypeID.TIMESTAMP.equals(type.typeId()); + } +} diff --git a/spark/src/main/java/org/apache/iceberg/spark/SparkFixupTypes.java b/spark/src/main/java/org/apache/iceberg/spark/SparkFixupTypes.java index 2d3ea4c81f00..5508965af249 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/SparkFixupTypes.java +++ b/spark/src/main/java/org/apache/iceberg/spark/SparkFixupTypes.java @@ -52,6 +52,11 @@ protected boolean fixupPrimitive(Type.PrimitiveType type, Type source) { return true; } break; + case TIMESTAMP: + if (source.typeId() == Type.TypeID.TIMESTAMP) { + return true; + } + break; default: } return false; diff --git a/spark/src/main/java/org/apache/iceberg/spark/SparkReadOptions.java b/spark/src/main/java/org/apache/iceberg/spark/SparkReadOptions.java index bce0bf4e8bb5..2fd522acd1c3 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/SparkReadOptions.java +++ b/spark/src/main/java/org/apache/iceberg/spark/SparkReadOptions.java @@ -50,4 +50,7 @@ private SparkReadOptions() { // Set ID that is used to fetch file scan tasks public static final String FILE_SCAN_TASK_SET_ID = "file-scan-task-set-id"; + + // skip snapshots of type delete while reading stream out of iceberg table + public static final String STREAMING_SKIP_DELETE_SNAPSHOTS = "streaming-skip-delete-snapshots"; } diff --git a/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java b/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java index 484c407e0247..b503ba634d85 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java +++ b/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java @@ -122,8 +122,31 @@ public static DataType convert(Type type) { * @throws IllegalArgumentException if the type cannot be converted */ public static Schema convert(StructType sparkType) { + return convert(sparkType, false); + } + + /** + * Convert a Spark {@link StructType struct} to a {@link Schema} with new field ids. + *

      + * This conversion assigns fresh ids. + *

      + * Some data types are represented as the same Spark type. These are converted to a default type. + *

      + * To convert using a reference schema for field ids and ambiguous types, use + * {@link #convert(Schema, StructType)}. + * + * @param sparkType a Spark StructType + * @param useTimestampWithoutZone boolean flag indicates that timestamp should be stored without timezone + * @return the equivalent Schema + * @throws IllegalArgumentException if the type cannot be converted + */ + public static Schema convert(StructType sparkType, boolean useTimestampWithoutZone) { Type converted = SparkTypeVisitor.visit(sparkType, new SparkTypeToType(sparkType)); - return new Schema(converted.asNestedType().asStructType().fields()); + Schema schema = new Schema(converted.asNestedType().asStructType().fields()); + if (useTimestampWithoutZone) { + schema = SparkFixupTimestampType.fixup(schema); + } + return schema; } /** diff --git a/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java b/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java index c9bfd49697a1..e822d45147b9 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java +++ b/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java @@ -80,6 +80,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; import scala.Function2; import scala.Option; +import scala.Predef; import scala.Some; import scala.Tuple2; import scala.collection.JavaConverters; @@ -140,7 +141,7 @@ public static Dataset partitionDFByFilter(SparkSession spark, String table, public static List getPartitions(SparkSession spark, String table) { try { TableIdentifier tableIdent = spark.sessionState().sqlParser().parseTableIdentifier(table); - return getPartitions(spark, tableIdent); + return getPartitions(spark, tableIdent, null); } catch (ParseException e) { throw SparkExceptionUtil.toUncheckedException(e, "Unable to parse table identifier: %s", table); } @@ -151,15 +152,23 @@ public static List getPartitions(SparkSession spark, String tabl * * @param spark a Spark session * @param tableIdent a table identifier + * @param partitionFilter partition filter, or null if no filter * @return all table's partitions */ - public static List getPartitions(SparkSession spark, TableIdentifier tableIdent) { + public static List getPartitions(SparkSession spark, TableIdentifier tableIdent, + Map partitionFilter) { try { SessionCatalog catalog = spark.sessionState().catalog(); CatalogTable catalogTable = catalog.getTableMetadata(tableIdent); - Seq partitions = catalog.listPartitions(tableIdent, Option.empty()); - + Option> scalaPartitionFilter; + if (partitionFilter != null && !partitionFilter.isEmpty()) { + scalaPartitionFilter = Option.apply(JavaConverters.mapAsScalaMapConverter(partitionFilter).asScala() + .toMap(Predef.conforms())); + } else { + scalaPartitionFilter = Option.empty(); + } + Seq partitions = catalog.listPartitions(tableIdent, scalaPartitionFilter); return JavaConverters .seqAsJavaListConverter(partitions) .asJava() @@ -372,17 +381,14 @@ public static void importSparkTable(SparkSession spark, TableIdentifier sourceTa try { PartitionSpec spec = SparkSchemaUtil.specForTable(spark, sourceTableIdentWithDB.unquotedString()); - if (spec == PartitionSpec.unpartitioned()) { + if (Objects.equal(spec, PartitionSpec.unpartitioned())) { importUnpartitionedSparkTable(spark, sourceTableIdentWithDB, targetTable); } else { - List sourceTablePartitions = getPartitions(spark, sourceTableIdent); + List sourceTablePartitions = getPartitions(spark, sourceTableIdent, + partitionFilter); Preconditions.checkArgument(!sourceTablePartitions.isEmpty(), "Cannot find any partitions in table %s", sourceTableIdent); - List filteredPartitions = filterPartitions(sourceTablePartitions, partitionFilter); - Preconditions.checkArgument(!filteredPartitions.isEmpty(), - "Cannot find any partitions which match the given filter. Partition filter is %s", - MAP_JOINER.join(partitionFilter)); - importSparkPartitions(spark, filteredPartitions, targetTable, spec, stagingDir); + importSparkPartitions(spark, sourceTablePartitions, targetTable, spec, stagingDir); } } catch (AnalysisException e) { throw SparkExceptionUtil.toUncheckedException( diff --git a/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java b/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java index 90c612957d4d..f0b8b2a9762b 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java +++ b/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java @@ -64,6 +64,7 @@ private int getNextId() { } @Override + @SuppressWarnings("ReferenceEquality") public Type struct(StructType struct, List types) { StructField[] fields = struct.fields(); List newFields = Lists.newArrayListWithExpectedSize(fields.length); diff --git a/spark/src/main/java/org/apache/iceberg/spark/SparkUtil.java b/spark/src/main/java/org/apache/iceberg/spark/SparkUtil.java index 3537d94ed3cf..4d5c3ec9e48e 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/SparkUtil.java +++ b/spark/src/main/java/org/apache/iceberg/spark/SparkUtil.java @@ -20,32 +20,48 @@ package org.apache.iceberg.spark; import java.util.List; +import java.util.Map; import java.util.function.BiFunction; import java.util.function.Function; import java.util.stream.Collectors; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; import org.apache.iceberg.Table; -import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.hadoop.HadoopConfigurable; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.transforms.Transform; import org.apache.iceberg.transforms.UnknownTransform; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; import org.apache.iceberg.util.Pair; +import org.apache.spark.sql.RuntimeConfig; import org.apache.spark.util.SerializableConfiguration; public class SparkUtil { + + public static final String HANDLE_TIMESTAMP_WITHOUT_TIMEZONE = + "spark.sql.iceberg.handle-timestamp-without-timezone"; + public static final String TIMESTAMP_WITHOUT_TIMEZONE_ERROR = String.format("Cannot handle timestamp without" + + " timezone fields in Spark. Spark does not natively support this type but if you would like to handle all" + + " timestamps as timestamp with timezone set '%s' to true. This will not change the underlying values stored" + + " but will change their displayed values in Spark. For more information please see" + + " https://docs.databricks.com/spark/latest/dataframes-datasets/dates-timestamps.html#ansi-sql-and" + + "-spark-sql-timestamps", HANDLE_TIMESTAMP_WITHOUT_TIMEZONE); + public static final String USE_TIMESTAMP_WITHOUT_TIME_ZONE_IN_NEW_TABLES = + "spark.sql.iceberg.use-timestamp-without-timezone-in-new-tables"; + private SparkUtil() { } public static FileIO serializableFileIO(Table table) { - if (table.io() instanceof HadoopFileIO) { + if (table.io() instanceof HadoopConfigurable) { // we need to use Spark's SerializableConfiguration to avoid issues with Kryo serialization - SerializableConfiguration conf = new SerializableConfiguration(((HadoopFileIO) table.io()).conf()); - return new HadoopFileIO(conf::value); - } else { - return table.io(); + ((HadoopConfigurable) table.io()).serializeConfWith(conf -> new SerializableConfiguration(conf)::value); } + + return table.io(); } /** @@ -100,4 +116,58 @@ public static Pair catalogAndIdentifier(List nameParts, } } } + + /** + * Responsible for checking if the table schema has a timestamp without timezone column + * @param schema table schema to check if it contains a timestamp without timezone column + * @return boolean indicating if the schema passed in has a timestamp field without a timezone + */ + public static boolean hasTimestampWithoutZone(Schema schema) { + return TypeUtil.find(schema, t -> Types.TimestampType.withoutZone().equals(t)) != null; + } + + /** + * Allow reading/writing timestamp without time zone as timestamp with time zone. Generally, + * this is not safe as timestamp without time zone is supposed to represent wall clock time semantics, + * i.e. no matter the reader/writer timezone 3PM should always be read as 3PM, + * but timestamp with time zone represents instant semantics, i.e the timestamp + * is adjusted so that the corresponding time in the reader timezone is displayed. + * When set to false (default), we throw an exception at runtime + * "Spark does not support timestamp without time zone fields" if reading timestamp without time zone fields + * + * @param readerConfig table read options + * @param sessionConf spark session configurations + * @return boolean indicating if reading timestamps without timezone is allowed + */ + public static boolean canHandleTimestampWithoutZone(Map readerConfig, RuntimeConfig sessionConf) { + String readerOption = readerConfig.get(HANDLE_TIMESTAMP_WITHOUT_TIMEZONE); + if (readerOption != null) { + return Boolean.parseBoolean(readerOption); + } + String sessionConfValue = sessionConf.get(HANDLE_TIMESTAMP_WITHOUT_TIMEZONE, null); + if (sessionConfValue != null) { + return Boolean.parseBoolean(sessionConfValue); + } + return false; + } + + /** + * Check whether the spark session config contains a {@link SparkUtil#USE_TIMESTAMP_WITHOUT_TIME_ZONE_IN_NEW_TABLES} + * property. + * Default value - false + * If true in new table all timestamp fields will be stored as {@link Types.TimestampType#withoutZone()}, + * otherwise {@link Types.TimestampType#withZone()} will be used + * + * @param sessionConf a spark runtime config + * @return true if the session config has {@link SparkUtil#USE_TIMESTAMP_WITHOUT_TIME_ZONE_IN_NEW_TABLES} property + * and this property is set to true + */ + public static boolean useTimestampWithoutZoneInNewTables(RuntimeConfig sessionConf) { + String sessionConfValue = sessionConf.get(USE_TIMESTAMP_WITHOUT_TIME_ZONE_IN_NEW_TABLES, null); + if (sessionConfValue != null) { + return Boolean.parseBoolean(sessionConfValue); + } + return false; + } + } diff --git a/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java b/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java index b9e9dfade792..6a8be60eb078 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java +++ b/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java @@ -104,12 +104,7 @@ public DataType primitive(Type.PrimitiveType primitive) { throw new UnsupportedOperationException( "Spark does not support time fields"); case TIMESTAMP: - Types.TimestampType timestamp = (Types.TimestampType) primitive; - if (timestamp.shouldAdjustToUTC()) { - return TimestampType$.MODULE$; - } - throw new UnsupportedOperationException( - "Spark does not support timestamp without time zone fields"); + return TimestampType$.MODULE$; case STRING: return StringType$.MODULE$; case UUID: diff --git a/spark/src/main/java/org/apache/iceberg/spark/actions/BaseRemoveOrphanFilesSparkAction.java b/spark/src/main/java/org/apache/iceberg/spark/actions/BaseDeleteOrphanFilesSparkAction.java similarity index 92% rename from spark/src/main/java/org/apache/iceberg/spark/actions/BaseRemoveOrphanFilesSparkAction.java rename to spark/src/main/java/org/apache/iceberg/spark/actions/BaseDeleteOrphanFilesSparkAction.java index 844dfd03701b..daa96b797f41 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/actions/BaseRemoveOrphanFilesSparkAction.java +++ b/spark/src/main/java/org/apache/iceberg/spark/actions/BaseDeleteOrphanFilesSparkAction.java @@ -31,8 +31,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.iceberg.Table; -import org.apache.iceberg.actions.BaseRemoveOrphanFilesActionResult; -import org.apache.iceberg.actions.RemoveOrphanFiles; +import org.apache.iceberg.actions.BaseDeleteOrphanFilesActionResult; +import org.apache.iceberg.actions.DeleteOrphanFiles; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.hadoop.HiddenPathFilter; @@ -73,10 +73,10 @@ * Note: It is dangerous to call this action with a short retention interval as it might corrupt * the state of the table if another operation is writing at the same time. */ -public class BaseRemoveOrphanFilesSparkAction - extends BaseSparkAction implements RemoveOrphanFiles { +public class BaseDeleteOrphanFilesSparkAction + extends BaseSparkAction implements DeleteOrphanFiles { - private static final Logger LOG = LoggerFactory.getLogger(BaseRemoveOrphanFilesSparkAction.class); + private static final Logger LOG = LoggerFactory.getLogger(BaseDeleteOrphanFilesSparkAction.class); private static final UserDefinedFunction filenameUDF = functions.udf((String path) -> { int lastIndex = path.lastIndexOf(File.separator); if (lastIndex == -1) { @@ -99,7 +99,7 @@ public void accept(String file) { } }; - public BaseRemoveOrphanFilesSparkAction(SparkSession spark, Table table) { + public BaseDeleteOrphanFilesSparkAction(SparkSession spark, Table table) { super(spark); this.hadoopConf = new SerializableConfiguration(spark.sessionState().newHadoopConf()); @@ -113,30 +113,30 @@ public BaseRemoveOrphanFilesSparkAction(SparkSession spark, Table table) { } @Override - protected RemoveOrphanFiles self() { + protected DeleteOrphanFiles self() { return this; } @Override - public BaseRemoveOrphanFilesSparkAction location(String newLocation) { + public BaseDeleteOrphanFilesSparkAction location(String newLocation) { this.location = newLocation; return this; } @Override - public BaseRemoveOrphanFilesSparkAction olderThan(long newOlderThanTimestamp) { + public BaseDeleteOrphanFilesSparkAction olderThan(long newOlderThanTimestamp) { this.olderThanTimestamp = newOlderThanTimestamp; return this; } @Override - public BaseRemoveOrphanFilesSparkAction deleteWith(Consumer newDeleteFunc) { + public BaseDeleteOrphanFilesSparkAction deleteWith(Consumer newDeleteFunc) { this.deleteFunc = newDeleteFunc; return this; } @Override - public RemoveOrphanFiles.Result execute() { + public DeleteOrphanFiles.Result execute() { JobGroupInfo info = newJobGroupInfo("REMOVE-ORPHAN-FILES", jobDesc()); return withJobGroupInfo(info, this::doExecute); } @@ -150,7 +150,7 @@ private String jobDesc() { return String.format("Removing orphan files (%s) from %s", Joiner.on(',').join(options), table.name()); } - private RemoveOrphanFiles.Result doExecute() { + private DeleteOrphanFiles.Result doExecute() { Dataset validDataFileDF = buildValidDataFileDF(table); Dataset validMetadataFileDF = buildValidMetadataFileDF(table); Dataset validFileDF = validDataFileDF.union(validMetadataFileDF); @@ -171,7 +171,7 @@ private RemoveOrphanFiles.Result doExecute() { .onFailure((file, exc) -> LOG.warn("Failed to delete file: {}", file, exc)) .run(deleteFunc::accept); - return new BaseRemoveOrphanFilesActionResult(orphanFiles); + return new BaseDeleteOrphanFilesActionResult(orphanFiles); } private Dataset buildActualFileDF() { diff --git a/spark/src/main/java/org/apache/iceberg/spark/actions/BaseRemoveReachableFilesSparkAction.java b/spark/src/main/java/org/apache/iceberg/spark/actions/BaseDeleteReachableFilesSparkAction.java similarity index 80% rename from spark/src/main/java/org/apache/iceberg/spark/actions/BaseRemoveReachableFilesSparkAction.java rename to spark/src/main/java/org/apache/iceberg/spark/actions/BaseDeleteReachableFilesSparkAction.java index 70eb8b2e4b5d..6534617d2dec 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/actions/BaseRemoveReachableFilesSparkAction.java +++ b/spark/src/main/java/org/apache/iceberg/spark/actions/BaseDeleteReachableFilesSparkAction.java @@ -28,9 +28,10 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableMetadataParser; -import org.apache.iceberg.actions.BaseRemoveFilesActionResult; -import org.apache.iceberg.actions.RemoveReachableFiles; +import org.apache.iceberg.actions.BaseDeleteReachableFilesActionResult; +import org.apache.iceberg.actions.DeleteReachableFiles; import org.apache.iceberg.exceptions.NotFoundException; +import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.hadoop.HadoopFileIO; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -46,14 +47,17 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.iceberg.TableProperties.GC_ENABLED; +import static org.apache.iceberg.TableProperties.GC_ENABLED_DEFAULT; + /** - * An implementation of {@link RemoveReachableFiles} that uses metadata tables in Spark + * An implementation of {@link DeleteReachableFiles} that uses metadata tables in Spark * to determine which files should be deleted. */ @SuppressWarnings("UnnecessaryAnonymousClass") -public class BaseRemoveReachableFilesSparkAction - extends BaseSparkAction implements RemoveReachableFiles { - private static final Logger LOG = LoggerFactory.getLogger(BaseRemoveReachableFilesSparkAction.class); +public class BaseDeleteReachableFilesSparkAction + extends BaseSparkAction implements DeleteReachableFiles { + private static final Logger LOG = LoggerFactory.getLogger(BaseDeleteReachableFilesSparkAction.class); private static final String DATA_FILE = "Data File"; private static final String MANIFEST = "Manifest"; @@ -65,7 +69,8 @@ public class BaseRemoveReachableFilesSparkAction // Creates an executor service that runs each task in the thread that invokes execute/submit. private static final ExecutorService DEFAULT_DELETE_EXECUTOR_SERVICE = null; - private final String metadataLocation; + private final TableMetadata tableMetadata; + private final Consumer defaultDelete = new Consumer() { @Override public void accept(String file) { @@ -77,31 +82,34 @@ public void accept(String file) { private ExecutorService removeExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE; private FileIO io = new HadoopFileIO(spark().sessionState().newHadoopConf()); - public BaseRemoveReachableFilesSparkAction(SparkSession spark, String metadataLocation) { + public BaseDeleteReachableFilesSparkAction(SparkSession spark, String metadataLocation) { super(spark); - this.metadataLocation = metadataLocation; + this.tableMetadata = TableMetadataParser.read(io, metadataLocation); + ValidationException.check( + PropertyUtil.propertyAsBoolean(tableMetadata.properties(), GC_ENABLED, GC_ENABLED_DEFAULT), + "Cannot remove files: GC is disabled (deleting files may corrupt other tables)"); } @Override - protected RemoveReachableFiles self() { + protected DeleteReachableFiles self() { return this; } @Override - public RemoveReachableFiles io(FileIO fileIO) { + public DeleteReachableFiles io(FileIO fileIO) { this.io = fileIO; return this; } @Override - public RemoveReachableFiles deleteWith(Consumer removeFn) { - this.removeFunc = removeFn; + public DeleteReachableFiles deleteWith(Consumer deleteFunc) { + this.removeFunc = deleteFunc; return this; } @Override - public RemoveReachableFiles executeDeleteWith(ExecutorService executorService) { + public DeleteReachableFiles executeDeleteWith(ExecutorService executorService) { this.removeExecutorService = executorService; return this; } @@ -109,15 +117,14 @@ public RemoveReachableFiles executeDeleteWith(ExecutorService executorService) { @Override public Result execute() { Preconditions.checkArgument(io != null, "File IO cannot be null"); - String msg = String.format("Removing files reachable from %s", metadataLocation); + String msg = String.format("Removing files reachable from %s", tableMetadata.metadataFileLocation()); JobGroupInfo info = newJobGroupInfo("REMOVE-FILES", msg); return withJobGroupInfo(info, this::doExecute); } private Result doExecute() { boolean streamResults = PropertyUtil.propertyAsBoolean(options(), STREAM_RESULTS, false); - TableMetadata metadata = TableMetadataParser.read(io, metadataLocation); - Dataset validFileDF = buildValidFileDF(metadata).distinct(); + Dataset validFileDF = buildValidFileDF(tableMetadata).distinct(); if (streamResults) { return deleteFiles(validFileDF.toLocalIterator()); } else { @@ -151,7 +158,7 @@ protected Dataset buildOtherMetadataFileDF(Table table) { * @param deleted an Iterator of Spark Rows of the structure (path: String, type: String) * @return Statistics on which files were deleted */ - private BaseRemoveFilesActionResult deleteFiles(Iterator deleted) { + private BaseDeleteReachableFilesActionResult deleteFiles(Iterator deleted) { AtomicLong dataFileCount = new AtomicLong(0L); AtomicLong manifestCount = new AtomicLong(0L); AtomicLong manifestListCount = new AtomicLong(0L); @@ -191,7 +198,7 @@ private BaseRemoveFilesActionResult deleteFiles(Iterator deleted) { long filesCount = dataFileCount.get() + manifestCount.get() + manifestListCount.get() + otherFilesCount.get(); LOG.info("Total files removed: {}", filesCount); - return new BaseRemoveFilesActionResult(dataFileCount.get(), manifestCount.get(), manifestListCount.get(), + return new BaseDeleteReachableFilesActionResult(dataFileCount.get(), manifestCount.get(), manifestListCount.get(), otherFilesCount.get()); } } diff --git a/spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java b/spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java new file mode 100644 index 000000000000..a86c8de1e730 --- /dev/null +++ b/spark/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSparkAction.java @@ -0,0 +1,377 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.spark.actions; + +import java.io.IOException; +import java.math.RoundingMode; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.BaseRewriteDataFilesFileGroupInfo; +import org.apache.iceberg.actions.BaseRewriteDataFilesResult; +import org.apache.iceberg.actions.BinPackStrategy; +import org.apache.iceberg.actions.RewriteDataFiles; +import org.apache.iceberg.actions.RewriteDataFilesCommitManager; +import org.apache.iceberg.actions.RewriteFileGroup; +import org.apache.iceberg.actions.RewriteStrategy; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Queues; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.relocated.com.google.common.collect.Streams; +import org.apache.iceberg.relocated.com.google.common.math.IntMath; +import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; +import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.Tasks; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +abstract class BaseRewriteDataFilesSparkAction + extends BaseSnapshotUpdateSparkAction implements RewriteDataFiles { + + private static final Logger LOG = LoggerFactory.getLogger(BaseRewriteDataFilesSparkAction.class); + private static final Set VALID_OPTIONS = ImmutableSet.of( + MAX_CONCURRENT_FILE_GROUP_REWRITES, + MAX_FILE_GROUP_SIZE_BYTES, + PARTIAL_PROGRESS_ENABLED, + PARTIAL_PROGRESS_MAX_COMMITS, + TARGET_FILE_SIZE_BYTES + ); + + private final Table table; + + private Expression filter = Expressions.alwaysTrue(); + private int maxConcurrentFileGroupRewrites; + private int maxCommits; + private boolean partialProgressEnabled; + private RewriteStrategy strategy; + + protected BaseRewriteDataFilesSparkAction(SparkSession spark, Table table) { + super(spark); + this.table = table; + this.strategy = binPackStrategy(); + } + + protected Table table() { + return table; + } + + /** + * The framework specific {@link BinPackStrategy} + */ + protected abstract BinPackStrategy binPackStrategy(); + + @Override + public RewriteDataFiles binPack() { + this.strategy = binPackStrategy(); + return this; + } + + @Override + public RewriteDataFiles filter(Expression expression) { + filter = Expressions.and(filter, expression); + return this; + } + + @Override + public RewriteDataFiles.Result execute() { + validateAndInitOptions(); + strategy = strategy.options(options()); + + Map>> fileGroupsByPartition = planFileGroups(); + RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition); + Stream groupStream = toGroupStream(ctx, fileGroupsByPartition); + + if (ctx.totalGroupCount() == 0) { + LOG.info("Nothing found to rewrite in {}", table.name()); + return new BaseRewriteDataFilesResult(Collections.emptyList()); + } + + if (partialProgressEnabled) { + return doExecuteWithPartialProgress(ctx, groupStream); + } else { + return doExecute(ctx, groupStream); + } + } + + private Map>> planFileGroups() { + CloseableIterable fileScanTasks = table.newScan() + .filter(filter) + .ignoreResiduals() + .planFiles(); + + try { + Map> filesByPartition = Streams.stream(fileScanTasks) + .collect(Collectors.groupingBy(task -> task.file().partition())); + + Map>> fileGroupsByPartition = Maps.newHashMap(); + + filesByPartition.forEach((partition, tasks) -> { + Iterable filtered = strategy.selectFilesToRewrite(tasks); + Iterable> groupedTasks = strategy.planFileGroups(filtered); + List> fileGroups = ImmutableList.copyOf(groupedTasks); + if (fileGroups.size() > 0) { + fileGroupsByPartition.put(partition, fileGroups); + } + }); + + return fileGroupsByPartition; + } finally { + try { + fileScanTasks.close(); + } catch (IOException io) { + LOG.error("Cannot properly close file iterable while planning for rewrite", io); + } + } + } + + @VisibleForTesting + RewriteFileGroup rewriteFiles(RewriteExecutionContext ctx, RewriteFileGroup fileGroup) { + String desc = jobDesc(fileGroup, ctx); + Set addedFiles = withJobGroupInfo( + newJobGroupInfo("REWRITE-DATA-FILES", desc), + () -> strategy.rewriteFiles(fileGroup.fileScans())); + + fileGroup.setOutputFiles(addedFiles); + LOG.info("Rewrite Files Ready to be Committed - {}", desc); + return fileGroup; + } + + private ExecutorService rewriteService() { + return MoreExecutors.getExitingExecutorService( + (ThreadPoolExecutor) Executors.newFixedThreadPool( + maxConcurrentFileGroupRewrites, + new ThreadFactoryBuilder() + .setNameFormat("Rewrite-Service-%d") + .build())); + } + + @VisibleForTesting + RewriteDataFilesCommitManager commitManager() { + return new RewriteDataFilesCommitManager(table); + } + + private Result doExecute(RewriteExecutionContext ctx, Stream groupStream) { + ExecutorService rewriteService = rewriteService(); + RewriteDataFilesCommitManager commitManager = commitManager(); + + ConcurrentLinkedQueue rewrittenGroups = Queues.newConcurrentLinkedQueue(); + + Tasks.Builder rewriteTaskBuilder = Tasks.foreach(groupStream) + .executeWith(rewriteService) + .stopOnFailure() + .noRetry() + .onFailure((fileGroup, exception) -> { + LOG.warn("Failure during rewrite process for group {}", fileGroup.info(), exception); + }); + + try { + rewriteTaskBuilder.run(fileGroup -> { + rewrittenGroups.add(rewriteFiles(ctx, fileGroup)); + }); + } catch (Exception e) { + // At least one rewrite group failed, clean up all completed rewrites + LOG.error("Cannot complete rewrite, {} is not enabled and one of the file set groups failed to " + + "be rewritten. This error occurred during the writing of new files, not during the commit process. This " + + "indicates something is wrong that doesn't involve conflicts with other Iceberg operations. Enabling " + + "{} may help in this case but the root cause should be investigated. Cleaning up {} groups which finished " + + "being written.", PARTIAL_PROGRESS_ENABLED, PARTIAL_PROGRESS_ENABLED, rewrittenGroups.size(), e); + + Tasks.foreach(rewrittenGroups) + .suppressFailureWhenFinished() + .run(group -> commitManager.abortFileGroup(group)); + throw e; + } finally { + rewriteService.shutdown(); + } + + try { + commitManager.commitOrClean(Sets.newHashSet(rewrittenGroups)); + } catch (ValidationException | CommitFailedException e) { + String errorMessage = String.format( + "Cannot commit rewrite because of a ValidationException or CommitFailedException. This usually means that " + + "this rewrite has conflicted with another concurrent Iceberg operation. To reduce the likelihood of " + + "conflicts, set %s which will break up the rewrite into multiple smaller commits controlled by %s. " + + "Separate smaller rewrite commits can succeed independently while any commits that conflict with " + + "another Iceberg operation will be ignored. This mode will create additional snapshots in the table " + + "history, one for each commit.", + PARTIAL_PROGRESS_ENABLED, PARTIAL_PROGRESS_MAX_COMMITS); + throw new RuntimeException(errorMessage, e); + } + + List rewriteResults = rewrittenGroups.stream() + .map(RewriteFileGroup::asResult) + .collect(Collectors.toList()); + return new BaseRewriteDataFilesResult(rewriteResults); + } + + private Result doExecuteWithPartialProgress(RewriteExecutionContext ctx, Stream groupStream) { + ExecutorService rewriteService = rewriteService(); + + // Start Commit Service + int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING); + RewriteDataFilesCommitManager.CommitService commitService = commitManager().service(groupsPerCommit); + commitService.start(); + + // Start rewrite tasks + Tasks.foreach(groupStream) + .suppressFailureWhenFinished() + .executeWith(rewriteService) + .noRetry() + .onFailure((fileGroup, exception) -> LOG.error("Failure during rewrite group {}", fileGroup.info(), exception)) + .run(fileGroup -> commitService.offer(rewriteFiles(ctx, fileGroup))); + rewriteService.shutdown(); + + // Stop Commit service + commitService.close(); + List commitResults = commitService.results(); + if (commitResults.size() == 0) { + LOG.error("{} is true but no rewrite commits succeeded. Check the logs to determine why the individual " + + "commits failed. If this is persistent it may help to increase {} which will break the rewrite operation " + + "into smaller commits.", PARTIAL_PROGRESS_ENABLED, PARTIAL_PROGRESS_MAX_COMMITS); + } + + List rewriteResults = commitResults.stream() + .map(RewriteFileGroup::asResult) + .collect(Collectors.toList()); + return new BaseRewriteDataFilesResult(rewriteResults); + } + + private Stream toGroupStream(RewriteExecutionContext ctx, + Map>> fileGroupsByPartition) { + + // Todo Add intelligence to the order in which we do rewrites instead of just using partition order + return fileGroupsByPartition.entrySet().stream() + .flatMap(e -> { + StructLike partition = e.getKey(); + List> fileGroups = e.getValue(); + return fileGroups.stream().map(tasks -> { + int globalIndex = ctx.currentGlobalIndex(); + int partitionIndex = ctx.currentPartitionIndex(partition); + FileGroupInfo info = new BaseRewriteDataFilesFileGroupInfo(globalIndex, partitionIndex, partition); + return new RewriteFileGroup(info, tasks); + }); + }); + } + + private void validateAndInitOptions() { + Set validOptions = Sets.newHashSet(strategy.validOptions()); + validOptions.addAll(VALID_OPTIONS); + + Set invalidKeys = Sets.newHashSet(options().keySet()); + invalidKeys.removeAll(validOptions); + + Preconditions.checkArgument(invalidKeys.isEmpty(), + "Cannot use options %s, they are not supported by the action or the strategy %s", + invalidKeys, strategy.name()); + + maxConcurrentFileGroupRewrites = PropertyUtil.propertyAsInt(options(), + MAX_CONCURRENT_FILE_GROUP_REWRITES, + MAX_CONCURRENT_FILE_GROUP_REWRITES_DEFAULT); + + maxCommits = PropertyUtil.propertyAsInt(options(), + PARTIAL_PROGRESS_MAX_COMMITS, + PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT); + + partialProgressEnabled = PropertyUtil.propertyAsBoolean(options(), + PARTIAL_PROGRESS_ENABLED, + PARTIAL_PROGRESS_ENABLED_DEFAULT); + + Preconditions.checkArgument(maxConcurrentFileGroupRewrites >= 1, + "Cannot set %s to %s, the value must be positive.", + MAX_CONCURRENT_FILE_GROUP_REWRITES, maxConcurrentFileGroupRewrites); + + Preconditions.checkArgument(!partialProgressEnabled || partialProgressEnabled && maxCommits > 0, + "Cannot set %s to %s, the value must be positive when %s is true", + PARTIAL_PROGRESS_MAX_COMMITS, maxCommits, PARTIAL_PROGRESS_ENABLED); + } + + private String jobDesc(RewriteFileGroup group, RewriteExecutionContext ctx) { + StructLike partition = group.info().partition(); + if (partition.size() > 0) { + return String.format("Rewriting %d files (%s, file group %d/%d, %s (%d/%d)) in %s", + group.rewrittenFiles().size(), + strategy.name(), group.info().globalIndex(), + ctx.totalGroupCount(), partition, group.info().partitionIndex(), ctx.groupsInPartition(partition), + table.name()); + } else { + return String.format("Rewriting %d files (%s, file group %d/%d) in %s", + group.rewrittenFiles().size(), + strategy.name(), group.info().globalIndex(), ctx.totalGroupCount(), + table.name()); + } + } + + @VisibleForTesting + static class RewriteExecutionContext { + private final Map numGroupsByPartition; + private final int totalGroupCount; + private final Map partitionIndexMap; + private final AtomicInteger groupIndex; + + RewriteExecutionContext(Map>> fileGroupsByPartition) { + this.numGroupsByPartition = fileGroupsByPartition.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().size())); + this.totalGroupCount = numGroupsByPartition.values().stream() + .reduce(Integer::sum) + .orElse(0); + this.partitionIndexMap = Maps.newConcurrentMap(); + this.groupIndex = new AtomicInteger(1); + } + + public int currentGlobalIndex() { + return groupIndex.getAndIncrement(); + } + + public int currentPartitionIndex(StructLike partition) { + return partitionIndexMap.merge(partition, 1, Integer::sum); + } + + public int groupsInPartition(StructLike partition) { + return numGroupsByPartition.get(partition); + } + + public int totalGroupCount() { + return totalGroupCount; + } + } +} diff --git a/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java b/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java index fe7d9805fa26..785d9a3a14ae 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java +++ b/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java @@ -64,7 +64,7 @@ abstract class BaseSparkAction implements Action { protected BaseSparkAction(SparkSession spark) { this.spark = spark; - this.sparkContext = new JavaSparkContext(spark.sparkContext()); + this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); } protected SparkSession spark() { @@ -115,7 +115,7 @@ protected Table newStaticTable(TableMetadata metadata, FileIO io) { } protected Dataset buildValidDataFileDF(Table table) { - JavaSparkContext context = new JavaSparkContext(spark.sparkContext()); + JavaSparkContext context = JavaSparkContext.fromSparkContext(spark.sparkContext()); Broadcast ioBroadcast = context.broadcast(SparkUtil.serializableFileIO(table)); Dataset allManifests = loadMetadataTable(table, ALL_MANIFESTS) diff --git a/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkActions.java b/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkActions.java index 65b69b701000..58b57177cf73 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkActions.java +++ b/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkActions.java @@ -21,9 +21,9 @@ import org.apache.iceberg.Table; import org.apache.iceberg.actions.ActionsProvider; +import org.apache.iceberg.actions.DeleteOrphanFiles; +import org.apache.iceberg.actions.DeleteReachableFiles; import org.apache.iceberg.actions.ExpireSnapshots; -import org.apache.iceberg.actions.RemoveOrphanFiles; -import org.apache.iceberg.actions.RemoveReachableFiles; import org.apache.iceberg.actions.RewriteManifests; import org.apache.spark.sql.SparkSession; @@ -40,8 +40,8 @@ protected SparkSession spark() { } @Override - public RemoveOrphanFiles removeOrphanFiles(Table table) { - return new BaseRemoveOrphanFilesSparkAction(spark, table); + public DeleteOrphanFiles deleteOrphanFiles(Table table) { + return new BaseDeleteOrphanFilesSparkAction(spark, table); } @Override @@ -55,7 +55,7 @@ public ExpireSnapshots expireSnapshots(Table table) { } @Override - public RemoveReachableFiles removeReachableFiles(String metadataLocation) { - return new BaseRemoveReachableFilesSparkAction(spark, metadataLocation); + public DeleteReachableFiles deleteReachableFiles(String metadataLocation) { + return new BaseDeleteReachableFilesSparkAction(spark, metadataLocation); } } diff --git a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java index 8a45aabf5fc2..4ed6420a9aa4 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java @@ -105,6 +105,7 @@ public OrcValueReader primitive(Type.PrimitiveType iPrimitive, TypeDescriptio case DOUBLE: return OrcValueReaders.doubles(); case TIMESTAMP_INSTANT: + case TIMESTAMP: return SparkOrcValueReaders.timestampTzs(); case DECIMAL: return SparkOrcValueReaders.decimals(primitive.getPrecision(), primitive.getScale()); diff --git a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueWriter.java b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueWriter.java index 131a93d6f9d9..b4124468687f 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueWriter.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueWriter.java @@ -53,7 +53,7 @@ default void write(int rowId, int column, SpecializedGetters data, ColumnVector * counters, and only return non-empty stream if the writer writes double or float values either by itself or * transitively. */ - default Stream metrics() { + default Stream> metrics() { return Stream.empty(); } } diff --git a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueWriters.java b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueWriters.java index 4b4075070f6f..df1b079bc7fa 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueWriters.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueWriters.java @@ -20,6 +20,7 @@ package org.apache.iceberg.spark.data; import java.util.stream.Stream; +import org.apache.iceberg.DoubleFieldMetrics; import org.apache.iceberg.FieldMetrics; import org.apache.iceberg.FloatFieldMetrics; import org.apache.orc.storage.common.type.HiveDecimal; @@ -141,52 +142,42 @@ public void nonNullWrite(int rowId, int column, SpecializedGetters data, ColumnV } private static class FloatWriter implements SparkOrcValueWriter { - private final int id; - private long nanCount; + private final FloatFieldMetrics.Builder floatFieldMetricsBuilder; private FloatWriter(int id) { - this.id = id; - this.nanCount = 0; + this.floatFieldMetricsBuilder = new FloatFieldMetrics.Builder(id); } @Override public void nonNullWrite(int rowId, int column, SpecializedGetters data, ColumnVector output) { float floatValue = data.getFloat(column); ((DoubleColumnVector) output).vector[rowId] = floatValue; - - if (Float.isNaN(floatValue)) { - nanCount++; - } + floatFieldMetricsBuilder.addValue(floatValue); } @Override - public Stream metrics() { - return Stream.of(new FloatFieldMetrics(id, nanCount)); + public Stream> metrics() { + return Stream.of(floatFieldMetricsBuilder.build()); } } private static class DoubleWriter implements SparkOrcValueWriter { - private final int id; - private long nanCount; + private final DoubleFieldMetrics.Builder doubleFieldMetricsBuilder; private DoubleWriter(int id) { - this.id = id; - this.nanCount = 0; + this.doubleFieldMetricsBuilder = new DoubleFieldMetrics.Builder(id); } @Override public void nonNullWrite(int rowId, int column, SpecializedGetters data, ColumnVector output) { double doubleValue = data.getDouble(column); ((DoubleColumnVector) output).vector[rowId] = doubleValue; - - if (Double.isNaN(doubleValue)) { - nanCount++; - } + doubleFieldMetricsBuilder.addValue(doubleValue); } @Override - public Stream metrics() { - return Stream.of(new FloatFieldMetrics(id, nanCount)); + public Stream> metrics() { + return Stream.of(doubleFieldMetricsBuilder.build()); } } @@ -281,7 +272,7 @@ public void nonNullWrite(int rowId, int column, SpecializedGetters data, ColumnV } @Override - public Stream metrics() { + public Stream> metrics() { return writer.metrics(); } } @@ -317,7 +308,7 @@ public void nonNullWrite(int rowId, int column, SpecializedGetters data, ColumnV } @Override - public Stream metrics() { + public Stream> metrics() { return Stream.concat(keyWriter.metrics(), valueWriter.metrics()); } } diff --git a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java index ce1b2bec0ec1..2c1edea1ffef 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java @@ -65,7 +65,7 @@ public void write(InternalRow value, VectorizedRowBatch output) { } @Override - public Stream metrics() { + public Stream> metrics() { return writer.metrics(); } @@ -118,6 +118,7 @@ public SparkOrcValueWriter primitive(Type.PrimitiveType iPrimitive, TypeDescript case DECIMAL: return SparkOrcValueWriters.decimal(primitive.getPrecision(), primitive.getScale()); case TIMESTAMP_INSTANT: + case TIMESTAMP: return SparkOrcValueWriters.timestampTz(); default: throw new IllegalArgumentException("Unhandled type " + primitive); @@ -146,7 +147,7 @@ public void nonNullWrite(int rowId, int column, SpecializedGetters data, ColumnV } @Override - public Stream metrics() { + public Stream> metrics() { return writers.stream().flatMap(SparkOrcValueWriter::metrics); } diff --git a/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java b/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java index ec91e32e9906..8abee4a575e1 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java @@ -166,6 +166,9 @@ public ParquetValueReader struct(Types.StructType expected, GroupType struct, } else if (id == MetadataColumns.ROW_POSITION.fieldId()) { reorderedFields.add(ParquetValueReaders.position()); types.add(null); + } else if (id == MetadataColumns.IS_DELETED.fieldId()) { + reorderedFields.add(ParquetValueReaders.constant(false)); + types.add(null); } else { ParquetValueReader reader = readersById.get(id); if (reader != null) { diff --git a/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java b/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java index c3c54996136e..418c25993a7e 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java @@ -128,6 +128,7 @@ public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primit primitiveValueReader = OrcValueReaders.doubles(); break; case TIMESTAMP_INSTANT: + case TIMESTAMP: primitiveValueReader = SparkOrcValueReaders.timestampTzs(); break; case DECIMAL: @@ -406,6 +407,8 @@ public ColumnVector convert(org.apache.orc.storage.ql.exec.vector.ColumnVector v fieldVectors.add(new ConstantColumnVector(field.type(), batchSize, idToConstant.get(field.fieldId()))); } else if (field.equals(MetadataColumns.ROW_POSITION)) { fieldVectors.add(new RowPositionColumnVector(batchOffsetInFile)); + } else if (field.equals(MetadataColumns.IS_DELETED)) { + fieldVectors.add(new ConstantColumnVector(field.type(), batchSize, false)); } else { fieldVectors.add(fieldConverters.get(vectorIndex) .convert(structVector.fields[vectorIndex], batchSize, batchOffsetInFile)); diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/BaseDataReader.java b/spark/src/main/java/org/apache/iceberg/spark/source/BaseDataReader.java index 566dcabd829c..c8b33dd2f706 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/BaseDataReader.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/BaseDataReader.java @@ -23,7 +23,6 @@ import java.io.IOException; import java.math.BigDecimal; import java.nio.ByteBuffer; -import java.util.Collections; import java.util.Iterator; import java.util.Map; import java.util.stream.Stream; @@ -38,6 +37,7 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Type; import org.apache.iceberg.util.ByteBuffers; @@ -76,7 +76,7 @@ abstract class BaseDataReader implements Closeable { Map files = Maps.newHashMapWithExpectedSize(task.files().size()); decryptedFiles.forEach(decrypted -> files.putIfAbsent(decrypted.location(), decrypted)); - this.inputFiles = Collections.unmodifiableMap(files); + this.inputFiles = ImmutableMap.copyOf(files); this.currentIterator = CloseableIterator.empty(); } diff --git a/spark/src/test/java/org/apache/iceberg/actions/TestRemoveReachableFilesAction.java b/spark/src/test/java/org/apache/iceberg/actions/TestDeleteReachableFilesAction.java similarity index 84% rename from spark/src/test/java/org/apache/iceberg/actions/TestRemoveReachableFilesAction.java rename to spark/src/test/java/org/apache/iceberg/actions/TestDeleteReachableFilesAction.java index d9130289d109..51db4ad58928 100644 --- a/spark/src/test/java/org/apache/iceberg/actions/TestRemoveReachableFilesAction.java +++ b/spark/src/test/java/org/apache/iceberg/actions/TestDeleteReachableFilesAction.java @@ -35,6 +35,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -50,7 +51,7 @@ import static org.apache.iceberg.types.Types.NestedField.optional; -public abstract class TestRemoveReachableFilesAction extends SparkTestBase { +public abstract class TestDeleteReachableFilesAction extends SparkTestBase { private static final HadoopTables TABLES = new HadoopTables(new Configuration()); private static final Schema SCHEMA = new Schema( optional(1, "c1", Types.IntegerType.get()), @@ -101,15 +102,15 @@ public void setupTableLocation() throws Exception { private void checkRemoveFilesResults(long expectedDatafiles, long expectedManifestsDeleted, long expectedManifestListsDeleted, long expectedOtherFilesDeleted, - RemoveReachableFiles.Result results) { + DeleteReachableFiles.Result results) { Assert.assertEquals("Incorrect number of manifest files deleted", - expectedManifestsDeleted, results.removedManifestsCount()); + expectedManifestsDeleted, results.deletedManifestsCount()); Assert.assertEquals("Incorrect number of datafiles deleted", - expectedDatafiles, results.removedDataFilesCount()); + expectedDatafiles, results.deletedDataFilesCount()); Assert.assertEquals("Incorrect number of manifest lists deleted", - expectedManifestListsDeleted, results.removedManifestListsCount()); + expectedManifestListsDeleted, results.deletedManifestListsCount()); Assert.assertEquals("Incorrect number of other lists deleted", - expectedOtherFilesDeleted, results.otherRemovedFilesCount()); + expectedOtherFilesDeleted, results.deletedOtherFilesCount()); } @Test @@ -134,7 +135,7 @@ public void dataFilesCleanupWithParallelTasks() { Set deleteThreads = ConcurrentHashMap.newKeySet(); AtomicInteger deleteThreadsIndex = new AtomicInteger(0); - RemoveReachableFiles.Result result = sparkActions().removeReachableFiles(metadataLocation(table)) + DeleteReachableFiles.Result result = sparkActions().deleteReachableFiles(metadataLocation(table)) .io(table.io()) .executeDeleteWith(Executors.newFixedThreadPool(4, runnable -> { Thread thread = new Thread(runnable); @@ -177,7 +178,7 @@ public void testWithExpiringDanglingStageCommit() { .appendFile(FILE_C) .commit(); - RemoveReachableFiles.Result result = sparkActions().removeReachableFiles(metadataLocation(table)) + DeleteReachableFiles.Result result = sparkActions().deleteReachableFiles(metadataLocation(table)) .io(table.io()) .execute(); @@ -186,7 +187,7 @@ public void testWithExpiringDanglingStageCommit() { @Test public void testRemoveFileActionOnEmptyTable() { - RemoveReachableFiles.Result result = sparkActions().removeReachableFiles(metadataLocation(table)) + DeleteReachableFiles.Result result = sparkActions().deleteReachableFiles(metadataLocation(table)) .io(table.io()) .execute(); @@ -217,10 +218,10 @@ public void testRemoveFilesActionWithReducedVersionsTable() { .appendFile(FILE_D) .commit(); - RemoveReachableFiles baseRemoveFilesSparkAction = sparkActions() - .removeReachableFiles(metadataLocation(table)) + DeleteReachableFiles baseRemoveFilesSparkAction = sparkActions() + .deleteReachableFiles(metadataLocation(table)) .io(table.io()); - RemoveReachableFiles.Result result = baseRemoveFilesSparkAction.execute(); + DeleteReachableFiles.Result result = baseRemoveFilesSparkAction.execute(); checkRemoveFilesResults(4, 5, 5, 8, result); } @@ -235,8 +236,8 @@ public void testRemoveFilesAction() { .appendFile(FILE_B) .commit(); - RemoveReachableFiles baseRemoveFilesSparkAction = sparkActions() - .removeReachableFiles(metadataLocation(table)) + DeleteReachableFiles baseRemoveFilesSparkAction = sparkActions() + .deleteReachableFiles(metadataLocation(table)) .io(table.io()); checkRemoveFilesResults(2, 2, 2, 4, baseRemoveFilesSparkAction.execute()); } @@ -253,8 +254,8 @@ public void testRemoveFilesActionWithDefaultIO() { // IO not set explicitly on removeReachableFiles action // IO defaults to HadoopFileIO - RemoveReachableFiles baseRemoveFilesSparkAction = sparkActions() - .removeReachableFiles(metadataLocation(table)); + DeleteReachableFiles baseRemoveFilesSparkAction = sparkActions() + .deleteReachableFiles(metadataLocation(table)); checkRemoveFilesResults(2, 2, 2, 4, baseRemoveFilesSparkAction.execute()); } @@ -275,7 +276,7 @@ public void testUseLocalIterator() { int jobsBefore = spark.sparkContext().dagScheduler().nextJobId().get(); - RemoveReachableFiles.Result results = sparkActions().removeReachableFiles(metadataLocation(table)) + DeleteReachableFiles.Result results = sparkActions().deleteReachableFiles(metadataLocation(table)) .io(table.io()) .option("stream-results", "true").execute(); @@ -304,24 +305,35 @@ public void testIgnoreMetadataFilesNotFound() { Assert.assertEquals("Should delete 1 file", 1, result.size()); Assert.assertTrue("Should remove v1 file", result.get(0).contains("v1.metadata.json")); - RemoveReachableFiles baseRemoveFilesSparkAction = sparkActions() - .removeReachableFiles(metadataLocation(table)) + DeleteReachableFiles baseRemoveFilesSparkAction = sparkActions() + .deleteReachableFiles(metadataLocation(table)) .io(table.io()); - RemoveReachableFiles.Result res = baseRemoveFilesSparkAction.execute(); + DeleteReachableFiles.Result res = baseRemoveFilesSparkAction.execute(); checkRemoveFilesResults(1, 1, 1, 4, res); } @Test public void testEmptyIOThrowsException() { - RemoveReachableFiles baseRemoveFilesSparkAction = sparkActions() - .removeReachableFiles(metadataLocation(table)) + DeleteReachableFiles baseRemoveFilesSparkAction = sparkActions() + .deleteReachableFiles(metadataLocation(table)) .io(null); AssertHelpers.assertThrows("FileIO needs to be set to use RemoveFiles action", IllegalArgumentException.class, "File IO cannot be null", baseRemoveFilesSparkAction::execute); } + @Test + public void testRemoveFilesActionWhenGarabageCollectionDisabled() { + table.updateProperties() + .set(TableProperties.GC_ENABLED, "false") + .commit(); + + AssertHelpers.assertThrows("Should complain about removing files when GC is disabled", + ValidationException.class, "Cannot remove files: GC is disabled (deleting files may corrupt other tables)", + () -> sparkActions().deleteReachableFiles(metadataLocation(table))); + } + private String metadataLocation(Table tbl) { return ((HasTableOperations) tbl).operations().current().metadataFileLocation(); } diff --git a/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java b/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java index a0685a68434e..95a9e98b0625 100644 --- a/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java +++ b/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java @@ -97,6 +97,10 @@ protected List sql(String query, Object... args) { return ImmutableList.of(); } + return rowsToJava(rows); + } + + protected List rowsToJava(List rows) { return rows.stream().map(this::toJava).collect(Collectors.toList()); } diff --git a/spark/src/test/java/org/apache/iceberg/spark/actions/TestNewRewriteDataFilesAction.java b/spark/src/test/java/org/apache/iceberg/spark/actions/TestNewRewriteDataFilesAction.java new file mode 100644 index 000000000000..0631d037e243 --- /dev/null +++ b/spark/src/test/java/org/apache/iceberg/spark/actions/TestNewRewriteDataFilesAction.java @@ -0,0 +1,778 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.spark.actions; + +import java.io.File; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.hadoop.conf.Configuration; +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.TableProperties; +import org.apache.iceberg.actions.ActionsProvider; +import org.apache.iceberg.actions.BinPackStrategy; +import org.apache.iceberg.actions.RewriteDataFiles; +import org.apache.iceberg.actions.RewriteDataFiles.Result; +import org.apache.iceberg.actions.RewriteDataFilesCommitManager; +import org.apache.iceberg.actions.RewriteFileGroup; +import org.apache.iceberg.exceptions.CommitStateUnknownException; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Streams; +import org.apache.iceberg.spark.SparkTestBase; +import org.apache.iceberg.spark.source.ThreeColumnRecord; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.NestedField; +import org.apache.iceberg.util.Pair; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.mockito.ArgumentMatcher; +import org.mockito.Mockito; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.argThat; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doCallRealMethod; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.spy; + +public abstract class TestNewRewriteDataFilesAction extends SparkTestBase { + + protected abstract ActionsProvider actions(); + protected abstract Set cacheContents(Table table); + + private static final HadoopTables TABLES = new HadoopTables(new Configuration()); + private static final Schema SCHEMA = new Schema( + optional(1, "c1", Types.IntegerType.get()), + optional(2, "c2", Types.StringType.get()), + optional(3, "c3", Types.StringType.get()) + ); + + @Rule + public TemporaryFolder temp = new TemporaryFolder(); + + private String tableLocation = null; + + @Before + public void setupTableLocation() throws Exception { + File tableDir = temp.newFolder(); + this.tableLocation = tableDir.toURI().toString(); + } + + private RewriteDataFiles basicRewrite(Table table) { + // Always compact regardless of input files + table.refresh(); + return actions().rewriteDataFiles(table).option(BinPackStrategy.MIN_INPUT_FILES, "1"); + } + + @Test + public void testEmptyTable() { + PartitionSpec spec = PartitionSpec.unpartitioned(); + Map options = Maps.newHashMap(); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + + Assert.assertNull("Table must be empty", table.currentSnapshot()); + + basicRewrite(table).execute(); + + Assert.assertNull("Table must stay empty", table.currentSnapshot()); + } + + @Test + public void testBinPackUnpartitionedTable() { + Table table = createTable(4); + shouldHaveFiles(table, 4); + List expectedRecords = currentData(); + + Result result = basicRewrite(table).execute(); + Assert.assertEquals("Action should rewrite 4 data files", 4, result.rewrittenDataFilesCount()); + Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFilesCount()); + + shouldHaveFiles(table, 1); + List actual = currentData(); + + assertEquals("Rows must match", expectedRecords, actual); + } + + @Test + public void testBinPackPartitionedTable() { + Table table = createTablePartitioned(4, 2); + shouldHaveFiles(table, 8); + List expectedRecords = currentData(); + + Result result = basicRewrite(table).execute(); + Assert.assertEquals("Action should rewrite 8 data files", 8, result.rewrittenDataFilesCount()); + Assert.assertEquals("Action should add 4 data file", 4, result.addedDataFilesCount()); + + shouldHaveFiles(table, 4); + List actualRecords = currentData(); + + assertEquals("Rows must match", expectedRecords, actualRecords); + } + + @Test + public void testBinPackWithFilter() { + Table table = createTablePartitioned(4, 2); + shouldHaveFiles(table, 8); + List expectedRecords = currentData(); + + Result result = basicRewrite(table) + .filter(Expressions.equal("c1", 1)) + .filter(Expressions.startsWith("c2", "foo")) + .execute(); + + Assert.assertEquals("Action should rewrite 2 data files", 2, result.rewrittenDataFilesCount()); + Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFilesCount()); + + shouldHaveFiles(table, 7); + + List actualRecords = currentData(); + assertEquals("Rows must match", expectedRecords, actualRecords); + } + + @Test + public void testRewriteLargeTableHasResiduals() { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).build(); + Map options = Maps.newHashMap(); + options.put(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, "100"); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + + // all records belong to the same partition + List records = Lists.newArrayList(); + for (int i = 0; i < 100; i++) { + records.add(new ThreeColumnRecord(i, String.valueOf(i), String.valueOf(i % 4))); + } + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class); + writeDF(df); + + List expectedRecords = currentData(); + + table.refresh(); + + CloseableIterable tasks = table.newScan() + .ignoreResiduals() + .filter(Expressions.equal("c3", "0")) + .planFiles(); + for (FileScanTask task : tasks) { + Assert.assertEquals("Residuals must be ignored", Expressions.alwaysTrue(), task.residual()); + } + + shouldHaveFiles(table, 2); + + Result result = basicRewrite(table) + .filter(Expressions.equal("c3", "0")) + .execute(); + Assert.assertEquals("Action should rewrite 2 data files", 2, result.rewrittenDataFilesCount()); + Assert.assertEquals("Action should add 1 data file", 1, result.addedDataFilesCount()); + + List actualRecords = currentData(); + + assertEquals("Rows must match", expectedRecords, actualRecords); + } + + @Test + public void testBinPackSplitLargeFile() { + Table table = createTable(1); + shouldHaveFiles(table, 1); + + List expectedRecords = currentData(); + long targetSize = testDataSize(table) / 2; + + Result result = basicRewrite(table) + .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Long.toString(targetSize)) + .execute(); + + Assert.assertEquals("Action should delete 1 data files", 1, result.rewrittenDataFilesCount()); + Assert.assertEquals("Action should add 2 data files", 2, result.addedDataFilesCount()); + + shouldHaveFiles(table, 2); + + List actualRecords = currentData(); + assertEquals("Rows must match", expectedRecords, actualRecords); + } + + @Test + public void testBinPackCombineMixedFiles() { + // One file too big + Table table = createTable(1); + shouldHaveFiles(table, 1); + + // Two files too small + writeRecords(1, 100); + writeRecords(1, 100); + shouldHaveFiles(table, 3); + + List expectedRecords = currentData(); + + int targetSize = averageFileSize(table); + + Result result = basicRewrite(table) + .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Integer.toString(targetSize)) + .option(BinPackStrategy.MAX_FILE_SIZE_BYTES, Integer.toString(targetSize + 100)) + .option(BinPackStrategy.MIN_FILE_SIZE_BYTES, Integer.toString(targetSize - 100)) + .execute(); + + Assert.assertEquals("Action should delete 3 data files", 3, result.rewrittenDataFilesCount()); + // Should Split the big files into 3 pieces, one of which should be combined with the two smaller files + Assert.assertEquals("Action should add 3 data files", 3, result.addedDataFilesCount()); + + shouldHaveFiles(table, 3); + + List actualRecords = currentData(); + assertEquals("Rows must match", expectedRecords, actualRecords); + } + + @Test + public void testPartialProgressEnabled() { + Table table = createTable(20); + int fileSize = averageFileSize(table); + + List originalData = currentData(); + + // Perform a rewrite but only allow 2 files to be compacted at a time + RewriteDataFiles.Result result = + basicRewrite(table) + .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true") + .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100)) + .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "10") + .execute(); + + Assert.assertEquals("Should have 10 fileGroups", result.rewriteResults().size(), 10); + + table.refresh(); + + shouldHaveSnapshots(table, 11); + shouldHaveACleanCache(table); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + } + + @Test + public void testMultipleGroups() { + Table table = createTable(20); + int fileSize = averageFileSize(table); + + List originalData = currentData(); + + // Perform a rewrite but only allow 2 files to be compacted at a time + RewriteDataFiles.Result result = + basicRewrite(table) + .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100)) + .option(BinPackStrategy.MIN_INPUT_FILES, "1") + .execute(); + + Assert.assertEquals("Should have 10 fileGroups", result.rewriteResults().size(), 10); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 2); + shouldHaveACleanCache(table); + } + + @Test + public void testPartialProgressMaxCommits() { + Table table = createTable(20); + int fileSize = averageFileSize(table); + + List originalData = currentData(); + + // Perform a rewrite but only allow 2 files to be compacted at a time + RewriteDataFiles.Result result = + basicRewrite(table) + .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100)) + .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true") + .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3") + .execute(); + + Assert.assertEquals("Should have 10 fileGroups", result.rewriteResults().size(), 10); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 4); + shouldHaveACleanCache(table); + } + + @Test + public void testSingleCommitWithRewriteFailure() { + Table table = createTable(20); + int fileSize = averageFileSize(table); + + List originalData = currentData(); + + BaseRewriteDataFilesSparkAction realRewrite = + (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction) + basicRewrite(table) + .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100)); + + BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite); + + // Fail groups 1, 3, and 7 during rewrite + GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7); + doThrow(new RuntimeException("Rewrite Failed")) + .when(spyRewrite) + .rewriteFiles(any(), argThat(failGroup)); + + AssertHelpers.assertThrows("Should fail entire rewrite if part fails", RuntimeException.class, + () -> spyRewrite.execute()); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 1); + shouldHaveNoOrphans(table); + shouldHaveACleanCache(table); + } + + @Test + public void testSingleCommitWithCommitFailure() { + Table table = createTable(20); + int fileSize = averageFileSize(table); + + List originalData = currentData(); + + BaseRewriteDataFilesSparkAction realRewrite = + (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction) + basicRewrite(table) + .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100)); + + BaseRewriteDataFilesSparkAction spyRewrite = spy(realRewrite); + RewriteDataFilesCommitManager util = spy(new RewriteDataFilesCommitManager(table)); + + // Fail to commit + doThrow(new RuntimeException("Commit Failure")) + .when(util) + .commitFileGroups(any()); + + doReturn(util) + .when(spyRewrite) + .commitManager(); + + AssertHelpers.assertThrows("Should fail entire rewrite if commit fails", RuntimeException.class, + () -> spyRewrite.execute()); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 1); + shouldHaveNoOrphans(table); + shouldHaveACleanCache(table); + } + + @Test + public void testParallelSingleCommitWithRewriteFailure() { + Table table = createTable(20); + int fileSize = averageFileSize(table); + + List originalData = currentData(); + + BaseRewriteDataFilesSparkAction realRewrite = + (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction) + basicRewrite(table) + .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100)) + .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "3"); + + BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite); + + // Fail groups 1, 3, and 7 during rewrite + GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7); + doThrow(new RuntimeException("Rewrite Failed")) + .when(spyRewrite) + .rewriteFiles(any(), argThat(failGroup)); + + AssertHelpers.assertThrows("Should fail entire rewrite if part fails", RuntimeException.class, + () -> spyRewrite.execute()); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 1); + shouldHaveNoOrphans(table); + shouldHaveACleanCache(table); + } + + @Test + public void testPartialProgressWithRewriteFailure() { + Table table = createTable(20); + int fileSize = averageFileSize(table); + + List originalData = currentData(); + + BaseRewriteDataFilesSparkAction realRewrite = + (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction) + basicRewrite(table) + .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100)) + .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true") + .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3"); + + BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite); + + // Fail groups 1, 3, and 7 during rewrite + GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7); + doThrow(new RuntimeException("Rewrite Failed")) + .when(spyRewrite) + .rewriteFiles(any(), argThat(failGroup)); + + RewriteDataFiles.Result result = spyRewrite.execute(); + + Assert.assertEquals("Should have 7 fileGroups", result.rewriteResults().size(), 7); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + // With 10 original groups and Max Commits of 3, we should have commits with 4, 4, and 2. + // removing 3 groups leaves us with only 2 new commits, 4 and 3 + shouldHaveSnapshots(table, 3); + shouldHaveNoOrphans(table); + shouldHaveACleanCache(table); + } + + @Test + public void testParallelPartialProgressWithRewriteFailure() { + Table table = createTable(20); + int fileSize = averageFileSize(table); + + List originalData = currentData(); + + BaseRewriteDataFilesSparkAction realRewrite = + (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction) + basicRewrite(table) + .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100)) + .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "3") + .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true") + .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3"); + + BaseRewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite); + + // Fail groups 1, 3, and 7 during rewrite + GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7); + doThrow(new RuntimeException("Rewrite Failed")) + .when(spyRewrite) + .rewriteFiles(any(), argThat(failGroup)); + + RewriteDataFiles.Result result = spyRewrite.execute(); + + Assert.assertEquals("Should have 7 fileGroups", result.rewriteResults().size(), 7); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + // With 10 original groups and Max Commits of 3, we should have commits with 4, 4, and 2. + // removing 3 groups leaves us with only 2 new commits, 4 and 3 + shouldHaveSnapshots(table, 3); + shouldHaveNoOrphans(table); + shouldHaveACleanCache(table); + } + + @Test + public void testParallelPartialProgressWithCommitFailure() { + Table table = createTable(20); + int fileSize = averageFileSize(table); + + List originalData = currentData(); + + BaseRewriteDataFilesSparkAction realRewrite = + (org.apache.iceberg.spark.actions.BaseRewriteDataFilesSparkAction) + basicRewrite(table) + .option(RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 100)) + .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "3") + .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true") + .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3"); + + BaseRewriteDataFilesSparkAction spyRewrite = spy(realRewrite); + RewriteDataFilesCommitManager util = spy(new RewriteDataFilesCommitManager(table)); + + // First and Third commits work, second does not + doCallRealMethod() + .doThrow(new RuntimeException("Commit Failed")) + .doCallRealMethod() + .when(util) + .commitFileGroups(any()); + + doReturn(util) + .when(spyRewrite) + .commitManager(); + + RewriteDataFiles.Result result = spyRewrite.execute(); + + // Commit 1: 4/4 + Commit 2 failed 0/4 + Commit 3: 2/2 == 6 out of 10 total groups comitted + Assert.assertEquals("Should have 6 fileGroups", 6, result.rewriteResults().size()); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + // Only 2 new commits because we broke one + shouldHaveSnapshots(table, 3); + shouldHaveNoOrphans(table); + shouldHaveACleanCache(table); + } + + @Test + public void testInvalidOptions() { + Table table = createTable(20); + + AssertHelpers.assertThrows("No negative values for partial progress max commits", + IllegalArgumentException.class, + () -> basicRewrite(table) + .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true") + .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "-5") + .execute()); + + AssertHelpers.assertThrows("No negative values for max concurrent groups", + IllegalArgumentException.class, + () -> basicRewrite(table) + .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "-5") + .execute()); + + AssertHelpers.assertThrows("No unknown options allowed", + IllegalArgumentException.class, + () -> basicRewrite(table) + .option("foobarity", "-5") + .execute()); + } + + @Test + public void testCommitStateUnknownException() { + Table table = createTable(20); + shouldHaveFiles(table, 20); + + List originalData = currentData(); + + BaseRewriteDataFilesSparkAction action = (BaseRewriteDataFilesSparkAction) basicRewrite(table); + BaseRewriteDataFilesSparkAction spyAction = spy(action); + RewriteDataFilesCommitManager util = spy(new RewriteDataFilesCommitManager(table)); + + doAnswer(invocationOnMock -> { + invocationOnMock.callRealMethod(); + throw new CommitStateUnknownException(new RuntimeException("Unknown State")); + }).when(util).commitFileGroups(any()); + + doReturn(util) + .when(spyAction) + .commitManager(); + + AssertHelpers.assertThrows("Should propagate CommitStateUnknown Exception", + CommitStateUnknownException.class, () -> spyAction.execute()); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 2); // Commit actually Succeeded + } + + protected List currentData() { + return rowsToJava(spark.read().format("iceberg").load(tableLocation) + .sort("c1", "c2", "c3") + .collectAsList()); + } + + protected long testDataSize(Table table) { + return Streams.stream(table.newScan().planFiles()).mapToLong(FileScanTask::length).sum(); + } + + protected void shouldHaveFiles(Table table, int numExpected) { + table.refresh(); + int numFiles = Iterables.size(table.newScan().planFiles()); + Assert.assertEquals("Did not have the expected number of files", numExpected, numFiles); + } + + protected void shouldHaveSnapshots(Table table, int expectedSnapshots) { + table.refresh(); + int actualSnapshots = Iterables.size(table.snapshots()); + Assert.assertEquals("Table did not have the expected number of snapshots", + expectedSnapshots, actualSnapshots); + } + + protected void shouldHaveNoOrphans(Table table) { + Assert.assertEquals("Should not have found any orphan files", ImmutableList.of(), + actions().deleteOrphanFiles(table) + .olderThan(System.currentTimeMillis()) + .execute() + .orphanFileLocations()); + } + + protected void shouldHaveACleanCache(Table table) { + Assert.assertEquals("Should not have any entries in cache", ImmutableSet.of(), + cacheContents(table)); + } + + protected void shouldHaveLastCommitSorted(Table table, String column) { + List, Pair>> + overlappingFiles = getOverlappingFiles(table, column); + + Assert.assertEquals("Found overlapping files", Collections.emptyList(), overlappingFiles); + } + + protected void shouldHaveLastCommitUnsorted(Table table, String column) { + List, Pair>> + overlappingFiles = getOverlappingFiles(table, column); + + Assert.assertNotEquals("Found overlapping files", Collections.emptyList(), overlappingFiles); + } + + private List, Pair>> getOverlappingFiles(Table table, String column) { + table.refresh(); + NestedField field = table.schema().caseInsensitiveFindField(column); + int columnId = field.fieldId(); + Class javaClass = (Class) field.type().typeId().javaClass(); + List> columnBounds = + Streams.stream(table.currentSnapshot().addedFiles()) + .map(file -> Pair.of( + javaClass.cast(Conversions.fromByteBuffer(field.type(), file.lowerBounds().get(columnId))), + javaClass.cast(Conversions.fromByteBuffer(field.type(), file.upperBounds().get(columnId))))) + .collect(Collectors.toList()); + + Comparator comparator = Comparators.forType(field.type().asPrimitiveType()); + + List, Pair>> overlappingFiles = columnBounds.stream() + .flatMap(left -> columnBounds.stream().map(right -> Pair.of(left, right))) + .filter(filePair -> { + Pair left = filePair.first(); + T leftLower = left.first(); + T leftUpper = left.second(); + Pair right = filePair.second(); + T rightLower = right.first(); + T rightUpper = right.second(); + boolean boundsOverlap = + (comparator.compare(leftUpper, rightLower) > 0 && comparator.compare(leftUpper, rightUpper) < 0) || + (comparator.compare(leftLower, rightLower) > 0 && comparator.compare(leftLower, rightUpper) < 0); + + return (left != right) && boundsOverlap; + }) + .collect(Collectors.toList()); + return overlappingFiles; + } + + /** + * Create a table with a certain number of files, returns the size of a file + * @param files number of files to create + * @return size of a file + */ + protected Table createTable(int files) { + PartitionSpec spec = PartitionSpec.unpartitioned(); + Map options = Maps.newHashMap(); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + table.updateProperties().set(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, "1024").commit(); + Assert.assertNull("Table must be empty", table.currentSnapshot()); + + writeRecords(files, 2000); + + return table; + } + + protected Table createTablePartitioned(int partitions, int files) { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA) + .identity("c1") + .truncate("c2", 2) + .build(); + Map options = Maps.newHashMap(); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + Assert.assertNull("Table must be empty", table.currentSnapshot()); + + writeRecords(files, 2000, partitions); + return table; + } + + protected int averageFileSize(Table table) { + table.refresh(); + return (int) Streams.stream(table.newScan().planFiles()).mapToLong(FileScanTask::length).average().getAsDouble(); + } + + private void writeRecords(int files, int numRecords) { + writeRecords(files, numRecords, 0); + } + + private void writeRecords(int files, int numRecords, int partitions) { + List records = Lists.newArrayList(); + List data = IntStream.range(0, numRecords).boxed().collect(Collectors.toList()); + Collections.shuffle(data, new Random(42)); + if (partitions > 0) { + data.forEach(i -> records.add(new ThreeColumnRecord(i % partitions, "foo" + i, "bar" + i))); + } else { + data.forEach(i -> records.add(new ThreeColumnRecord(i, "foo" + i, "bar" + i))); + } + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class).repartition(files); + writeDF(df); + } + + private void writeDF(Dataset df) { + df.select("c1", "c2", "c3") + .sortWithinPartitions("c1", "c2") + .write() + .format("iceberg") + .mode("append") + .save(tableLocation); + } + + class GroupInfoMatcher implements ArgumentMatcher { + private final Set groupIDs; + + GroupInfoMatcher(Integer... globalIndex) { + this.groupIDs = ImmutableSet.copyOf(globalIndex); + } + + @Override + public boolean matches(RewriteFileGroup argument) { + return groupIDs.contains(argument.info().globalIndex()); + } + } +} diff --git a/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java b/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java index ae9c7fdc9af4..8cbf6c795956 100644 --- a/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java +++ b/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTest.java @@ -20,15 +20,20 @@ package org.apache.iceberg.spark.data; import java.io.IOException; +import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import org.apache.iceberg.Schema; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.SparkUtil; import org.apache.iceberg.types.TypeUtil; 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.apache.spark.sql.internal.SQLConf; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -185,4 +190,51 @@ public void testMixedTypes() throws IOException { writeAndValidate(schema); } + + @Test + public void testTimestampWithoutZone() throws IOException { + withSQLConf(ImmutableMap.of(SparkUtil.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE, "true"), () -> { + Schema schema = TypeUtil.assignIncreasingFreshIds(new Schema( + required(0, "id", LongType.get()), + optional(1, "ts_without_zone", Types.TimestampType.withoutZone()))); + + writeAndValidate(schema); + }); + } + + protected void withSQLConf(Map conf, Action action) throws IOException { + SQLConf sqlConf = SQLConf.get(); + + Map currentConfValues = Maps.newHashMap(); + conf.keySet().forEach(confKey -> { + if (sqlConf.contains(confKey)) { + String currentConfValue = sqlConf.getConfString(confKey); + currentConfValues.put(confKey, currentConfValue); + } + }); + + conf.forEach((confKey, confValue) -> { + if (SQLConf.staticConfKeys().contains(confKey)) { + throw new RuntimeException("Cannot modify the value of a static config: " + confKey); + } + sqlConf.setConfString(confKey, confValue); + }); + + try { + action.invoke(); + } finally { + conf.forEach((confKey, confValue) -> { + if (currentConfValues.containsKey(confKey)) { + sqlConf.setConfString(confKey, currentConfValues.get(confKey)); + } else { + sqlConf.unsetConf(confKey); + } + }); + } + } + + @FunctionalInterface + protected interface Action { + void invoke() throws IOException; + } } diff --git a/spark/src/test/java/org/apache/iceberg/spark/data/GenericsHelpers.java b/spark/src/test/java/org/apache/iceberg/spark/data/GenericsHelpers.java index 0c4598a209e8..24724d3d12bf 100644 --- a/spark/src/test/java/org/apache/iceberg/spark/data/GenericsHelpers.java +++ b/spark/src/test/java/org/apache/iceberg/spark/data/GenericsHelpers.java @@ -24,7 +24,9 @@ import java.sql.Timestamp; import java.time.Instant; import java.time.LocalDate; +import java.time.LocalDateTime; import java.time.OffsetDateTime; +import java.time.ZoneId; import java.time.ZoneOffset; import java.time.temporal.ChronoUnit; import java.util.Collection; @@ -122,13 +124,19 @@ private static void assertEqualsSafe(Type type, Object expected, Object actual) Assert.assertEquals("ISO-8601 date should be equal", expected.toString(), actual.toString()); break; case TIMESTAMP: - Assert.assertTrue("Should expect an OffsetDateTime", expected instanceof OffsetDateTime); Assert.assertTrue("Should be a Timestamp", actual instanceof Timestamp); Timestamp ts = (Timestamp) actual; // milliseconds from nanos has already been added by getTime OffsetDateTime actualTs = EPOCH.plusNanos( (ts.getTime() * 1_000_000) + (ts.getNanos() % 1_000_000)); - Assert.assertEquals("Timestamp should be equal", expected, actualTs); + Types.TimestampType timestampType = (Types.TimestampType) type; + if (timestampType.shouldAdjustToUTC()) { + Assert.assertTrue("Should expect an OffsetDateTime", expected instanceof OffsetDateTime); + Assert.assertEquals("Timestamp should be equal", expected, actualTs); + } else { + Assert.assertTrue("Should expect an LocalDateTime", expected instanceof LocalDateTime); + Assert.assertEquals("Timestamp should be equal", expected, actualTs.toLocalDateTime()); + } break; case STRING: Assert.assertTrue("Should be a String", actual instanceof String); @@ -241,9 +249,16 @@ private static void assertEqualsUnsafe(Type type, Object expected, Object actual Assert.assertEquals("Primitive value should be equal to expected", expectedDays, actual); break; case TIMESTAMP: - Assert.assertTrue("Should expect an OffsetDateTime", expected instanceof OffsetDateTime); - long expectedMicros = ChronoUnit.MICROS.between(EPOCH, (OffsetDateTime) expected); - Assert.assertEquals("Primitive value should be equal to expected", expectedMicros, actual); + Types.TimestampType timestampType = (Types.TimestampType) type; + if (timestampType.shouldAdjustToUTC()) { + Assert.assertTrue("Should expect an OffsetDateTime", expected instanceof OffsetDateTime); + long expectedMicros = ChronoUnit.MICROS.between(EPOCH, (OffsetDateTime) expected); + Assert.assertEquals("Primitive value should be equal to expected", expectedMicros, actual); + } else { + Assert.assertTrue("Should expect an LocalDateTime", expected instanceof LocalDateTime); + long expectedMicros = ChronoUnit.MICROS.between(EPOCH, ((LocalDateTime) expected).atZone(ZoneId.of("UTC"))); + Assert.assertEquals("Primitive value should be equal to expected", expectedMicros, actual); + } break; case STRING: Assert.assertTrue("Should be a UTF8String", actual instanceof UTF8String); diff --git a/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java b/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java index 965659bc2dd1..6322827c3af5 100644 --- a/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java +++ b/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java @@ -98,7 +98,8 @@ public static void assertEqualsBatch(Types.StructType struct, Iterator e if (checkArrowValidityVector) { ColumnVector columnVector = batch.column(i); ValueVector arrowVector = ((IcebergArrowColumnVector) columnVector).vectorAccessor().getVector(); - Assert.assertFalse("Nullability doesn't match", expectedValue == null ^ arrowVector.isNull(rowId)); + Assert.assertFalse("Nullability doesn't match of " + columnVector.dataType(), + expectedValue == null ^ arrowVector.isNull(rowId)); } } } diff --git a/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReadMetadataColumns.java b/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReadMetadataColumns.java index 410a5f1bd0d7..010f9f5d077c 100644 --- a/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReadMetadataColumns.java +++ b/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReadMetadataColumns.java @@ -69,7 +69,8 @@ public class TestSparkOrcReadMetadataColumns { private static final Schema PROJECTION_SCHEMA = new Schema( required(100, "id", Types.LongType.get()), required(101, "data", Types.StringType.get()), - MetadataColumns.ROW_POSITION + MetadataColumns.ROW_POSITION, + MetadataColumns.IS_DELETED ); private static final int NUM_ROWS = 1000; @@ -91,6 +92,7 @@ public class TestSparkOrcReadMetadataColumns { row.update(0, i); row.update(1, UTF8String.fromString("str" + i)); row.update(2, i); + row.update(3, false); EXPECTED_ROWS.add(row); } } diff --git a/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java b/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java index 8f15db724a3b..b68db024aab3 100644 --- a/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java +++ b/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java @@ -71,7 +71,8 @@ public class TestSparkParquetReadMetadataColumns { private static final Schema PROJECTION_SCHEMA = new Schema( required(100, "id", Types.LongType.get()), required(101, "data", Types.StringType.get()), - MetadataColumns.ROW_POSITION + MetadataColumns.ROW_POSITION, + MetadataColumns.IS_DELETED ); private static final int NUM_ROWS = 1000; @@ -104,6 +105,7 @@ public class TestSparkParquetReadMetadataColumns { } row.update(1, UTF8String.fromString("str" + i)); row.update(2, i); + row.update(3, false); EXPECTED_ROWS.add(row); } } diff --git a/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java b/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java index d33067395985..48dcc94a5fce 100644 --- a/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java +++ b/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java @@ -39,6 +39,7 @@ import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; +import org.apache.parquet.column.ParquetProperties; import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.Type; @@ -110,6 +111,14 @@ FileAppender getParquetWriter(Schema schema, File testFile) .build(); } + FileAppender getParquetV2Writer(Schema schema, File testFile) throws IOException { + return Parquet.write(Files.localOutput(testFile)) + .schema(schema) + .named("test") + .writerVersion(ParquetProperties.WriterVersion.PARQUET_2_0) + .build(); + } + void assertRecordsMatch( Schema schema, int expectedSize, Iterable expected, File testFile, boolean setAndCheckArrowValidityBuffer, boolean reuseContainers, int batchSize) @@ -260,4 +269,41 @@ public void testReadsForTypePromotedColumns() throws Exception { assertRecordsMatch(readSchema, 30000, data, dataFile, false, true, BATCH_SIZE); } + + @Test + public void testSupportedReadsForParquetV2() throws Exception { + // Only float and double column types are written using plain encoding with Parquet V2 + Schema schema = new Schema( + optional(102, "float_data", Types.FloatType.get()), + optional(103, "double_data", Types.DoubleType.get())); + + File dataFile = temp.newFile(); + Assert.assertTrue("Delete should succeed", dataFile.delete()); + Iterable data = generateData(schema, 30000, 0L, + RandomData.DEFAULT_NULL_PERCENTAGE, IDENTITY); + try (FileAppender writer = getParquetV2Writer(schema, dataFile)) { + writer.addAll(data); + } + assertRecordsMatch(schema, 30000, data, dataFile, false, + true, BATCH_SIZE); + } + + @Test + public void testUnsupportedReadsForParquetV2() throws Exception { + // Longs, ints, string types etc use delta encoding and which are not supported for vectorized reads + Schema schema = new Schema(SUPPORTED_PRIMITIVES.fields()); + File dataFile = temp.newFile(); + Assert.assertTrue("Delete should succeed", dataFile.delete()); + Iterable data = generateData(schema, 30000, 0L, + RandomData.DEFAULT_NULL_PERCENTAGE, IDENTITY); + try (FileAppender writer = getParquetV2Writer(schema, dataFile)) { + writer.addAll(data); + } + AssertHelpers.assertThrows("Vectorized reads not supported", + UnsupportedOperationException.class, "Cannot support vectorized reads for column", () -> { + assertRecordsMatch(schema, 30000, data, dataFile, false, + true, BATCH_SIZE); + return null; + }); + } } diff --git a/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java index 015b819fa608..f8f66ca66f28 100644 --- a/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java +++ b/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWrites.java @@ -121,7 +121,7 @@ public TestDataFrameWrites(String format) { @BeforeClass public static void startSpark() { TestDataFrameWrites.spark = SparkSession.builder().master("local[2]").getOrCreate(); - TestDataFrameWrites.sc = new JavaSparkContext(spark.sparkContext()); + TestDataFrameWrites.sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); } @AfterClass diff --git a/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index b3dcd1ea55f6..4244137d338c 100644 --- a/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -880,6 +880,25 @@ public void testPartitionsTable() { Assert.assertEquals("Actual results should have one row", 1, actualAfterFirstCommit.size()); TestHelpers.assertEqualsSafe(partitionsTable.schema().asStruct(), expected.get(0), actualAfterFirstCommit.get(0)); + + // check predicate push down + List filtered = spark.read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "partitions")) + .filter("partition.id < 2") + .collectAsList(); + Assert.assertEquals("Actual results should have one row", 1, filtered.size()); + TestHelpers.assertEqualsSafe(partitionsTable.schema().asStruct(), expected.get(0), filtered.get(0)); + + List nonFiltered = spark.read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "partitions")) + .filter("partition.id < 2 or record_count=1") + .collectAsList(); + Assert.assertEquals("Actual results should have one row", 2, nonFiltered.size()); + for (int i = 0; i < 2; i += 1) { + TestHelpers.assertEqualsSafe(partitionsTable.schema().asStruct(), expected.get(i), actual.get(i)); + } } @Test diff --git a/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java new file mode 100644 index 000000000000..4ab01044046f --- /dev/null +++ b/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.spark.source; + +import java.util.Iterator; +import org.apache.iceberg.RecordWrapperTest; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.data.InternalRecordWrapper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.util.StructLikeWrapper; +import org.apache.spark.sql.catalyst.InternalRow; +import org.junit.Assert; +import org.junit.Ignore; + +public class TestInternalRowWrapper extends RecordWrapperTest { + + @Ignore + @Override + public void testTimestampWithoutZone() { + // Spark does not support timestamp without zone. + } + + @Ignore + @Override + public void testTime() { + // Spark does not support time fields. + } + + @Override + protected void generateAndValidate(Schema schema, AssertMethod assertMethod) { + int numRecords = 100; + Iterable recordList = RandomGenericData.generate(schema, numRecords, 101L); + Iterable rowList = RandomData.generateSpark(schema, numRecords, 101L); + + InternalRecordWrapper recordWrapper = new InternalRecordWrapper(schema.asStruct()); + InternalRowWrapper rowWrapper = new InternalRowWrapper(SparkSchemaUtil.convert(schema)); + + Iterator actual = recordList.iterator(); + Iterator expected = rowList.iterator(); + + StructLikeWrapper actualWrapper = StructLikeWrapper.forType(schema.asStruct()); + StructLikeWrapper expectedWrapper = StructLikeWrapper.forType(schema.asStruct()); + for (int i = 0; i < numRecords; i++) { + Assert.assertTrue("Should have more records", actual.hasNext()); + Assert.assertTrue("Should have more InternalRow", expected.hasNext()); + + StructLike recordStructLike = recordWrapper.wrap(actual.next()); + StructLike rowStructLike = rowWrapper.wrap(expected.next()); + + assertMethod.assertEquals("Should have expected StructLike values", + actualWrapper.set(recordStructLike), expectedWrapper.set(rowStructLike)); + } + + Assert.assertFalse("Shouldn't have more record", actual.hasNext()); + Assert.assertFalse("Shouldn't have more InternalRow", expected.hasNext()); + } +} diff --git a/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java index 79e6603b30d6..3cba747c2e43 100644 --- a/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java +++ b/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java @@ -105,7 +105,7 @@ public TestPartitionPruning(String format, boolean vectorized) { @BeforeClass public static void startSpark() { TestPartitionPruning.spark = SparkSession.builder().master("local[2]").getOrCreate(); - TestPartitionPruning.sparkContext = new JavaSparkContext(spark.sparkContext()); + TestPartitionPruning.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); String optionKey = String.format("fs.%s.impl", CountOpenLocalFileSystem.scheme); CONF.set(optionKey, CountOpenLocalFileSystem.class.getName()); diff --git a/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java index 227e3079ac6b..637ad7c33d82 100644 --- a/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java +++ b/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java @@ -105,7 +105,7 @@ public abstract class TestSparkDataFile { @BeforeClass public static void startSpark() { TestSparkDataFile.spark = SparkSession.builder().master("local[2]").getOrCreate(); - TestSparkDataFile.sparkContext = new JavaSparkContext(spark.sparkContext()); + TestSparkDataFile.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); } @AfterClass diff --git a/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java new file mode 100644 index 000000000000..482a711910ab --- /dev/null +++ b/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java @@ -0,0 +1,287 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.spark.source; + +import java.io.File; +import java.io.IOException; +import java.time.LocalDateTime; +import java.util.List; +import java.util.Locale; +import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.AssertHelpers; +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.data.GenericAppenderFactory; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.SparkTestBase; +import org.apache.iceberg.spark.SparkUtil; +import org.apache.iceberg.spark.data.GenericsHelpers; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.apache.iceberg.Files.localOutput; + +@RunWith(Parameterized.class) +public abstract class TestTimestampWithoutZone extends SparkTestBase { + private static final Configuration CONF = new Configuration(); + private static final HadoopTables TABLES = new HadoopTables(CONF); + + private static final Schema SCHEMA = new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(3, "data", Types.StringType.get()) + ); + + private static SparkSession spark = null; + + @BeforeClass + public static void startSpark() { + TestTimestampWithoutZone.spark = SparkSession.builder().master("local[2]").getOrCreate(); + } + + @AfterClass + public static void stopSpark() { + SparkSession currentSpark = TestTimestampWithoutZone.spark; + TestTimestampWithoutZone.spark = null; + currentSpark.stop(); + } + + @Rule + public TemporaryFolder temp = new TemporaryFolder(); + + private final String format; + private final boolean vectorized; + + @Parameterized.Parameters(name = "format = {0}, vectorized = {1}") + public static Object[][] parameters() { + return new Object[][] { + { "parquet", false }, + { "parquet", true }, + { "avro", false } + }; + } + + public TestTimestampWithoutZone(String format, boolean vectorized) { + this.format = format; + this.vectorized = vectorized; + } + + private File parent = null; + private File unpartitioned = null; + private List records = null; + + @Before + public void writeUnpartitionedTable() throws IOException { + this.parent = temp.newFolder("TestTimestampWithoutZone"); + this.unpartitioned = new File(parent, "unpartitioned"); + File dataFolder = new File(unpartitioned, "data"); + Assert.assertTrue("Mkdir should succeed", dataFolder.mkdirs()); + + Table table = TABLES.create(SCHEMA, PartitionSpec.unpartitioned(), unpartitioned.toString()); + Schema tableSchema = table.schema(); // use the table schema because ids are reassigned + + FileFormat fileFormat = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH)); + + File testFile = new File(dataFolder, fileFormat.addExtension(UUID.randomUUID().toString())); + + // create records using the table's schema + this.records = testRecords(tableSchema); + + try (FileAppender writer = new GenericAppenderFactory(tableSchema).newAppender( + localOutput(testFile), fileFormat)) { + writer.addAll(records); + } + + DataFile file = DataFiles.builder(PartitionSpec.unpartitioned()) + .withRecordCount(records.size()) + .withFileSizeInBytes(testFile.length()) + .withPath(testFile.toString()) + .build(); + + table.newAppend().appendFile(file).commit(); + } + + @Test + public void testUnpartitionedTimestampWithoutZone() { + assertEqualsSafe(SCHEMA.asStruct(), records, read(unpartitioned.toString(), vectorized)); + } + + @Test + public void testUnpartitionedTimestampWithoutZoneProjection() { + Schema projection = SCHEMA.select("id", "ts"); + assertEqualsSafe(projection.asStruct(), + records.stream().map(r -> projectFlat(projection, r)).collect(Collectors.toList()), + read(unpartitioned.toString(), vectorized, "id", "ts")); + } + + @Rule + public ExpectedException exception = ExpectedException.none(); + + @Test + public void testUnpartitionedTimestampWithoutZoneError() { + AssertHelpers.assertThrows(String.format("Read operation performed on a timestamp without timezone field while " + + "'%s' set to false should throw exception", + SparkUtil.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE), + IllegalArgumentException.class, + SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR, + () -> spark.read().format("iceberg") + .option("vectorization-enabled", String.valueOf(vectorized)) + .option(SparkUtil.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE, "false") + .load(unpartitioned.toString()) + .collectAsList()); + } + + @Test + public void testUnpartitionedTimestampWithoutZoneAppend() { + spark.read().format("iceberg") + .option(SparkUtil.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE, "true") + .option("vectorization-enabled", String.valueOf(vectorized)) + .load(unpartitioned.toString()) + .write() + .format("iceberg") + .option(SparkUtil.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE, "true") + .mode(SaveMode.Append) + .save(unpartitioned.toString()); + + assertEqualsSafe(SCHEMA.asStruct(), + Stream.concat(records.stream(), records.stream()).collect(Collectors.toList()), + read(unpartitioned.toString(), vectorized)); + } + + @Test + public void testUnpartitionedTimestampWithoutZoneWriteError() { + String errorMessage = String.format("Write operation performed on a timestamp without timezone field while " + + "'%s' set to false should throw exception", + SparkUtil.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE); + Runnable writeOperation = () -> spark.read().format("iceberg") + .option(SparkUtil.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE, "true") + .option("vectorization-enabled", String.valueOf(vectorized)) + .load(unpartitioned.toString()) + .write() + .format("iceberg") + .option(SparkUtil.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE, "false") + .mode(SaveMode.Append) + .save(unpartitioned.toString()); + + AssertHelpers.assertThrows(errorMessage, IllegalArgumentException.class, + SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR, writeOperation); + + } + + @Test + public void testUnpartitionedTimestampWithoutZoneSessionProperties() { + withSQLConf(ImmutableMap.of(SparkUtil.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE, "true"), () -> { + spark.read().format("iceberg") + .option("vectorization-enabled", String.valueOf(vectorized)) + .load(unpartitioned.toString()) + .write() + .format("iceberg") + .mode(SaveMode.Append) + .save(unpartitioned.toString()); + + assertEqualsSafe(SCHEMA.asStruct(), + Stream.concat(records.stream(), records.stream()).collect(Collectors.toList()), + read(unpartitioned.toString(), vectorized)); + }); + } + + private static Record projectFlat(Schema projection, Record record) { + Record result = GenericRecord.create(projection); + List fields = projection.asStruct().fields(); + for (int i = 0; i < fields.size(); i += 1) { + Types.NestedField field = fields.get(i); + result.set(i, record.getField(field.name())); + } + return result; + } + + public static void assertEqualsSafe(Types.StructType struct, + List expected, List actual) { + Assert.assertEquals("Number of results should match expected", expected.size(), actual.size()); + for (int i = 0; i < expected.size(); i += 1) { + GenericsHelpers.assertEqualsSafe(struct, expected.get(i), actual.get(i)); + } + } + + private List testRecords(Schema schema) { + return Lists.newArrayList( + record(schema, 0L, parseToLocal("2017-12-22T09:20:44.294658"), "junction"), + record(schema, 1L, parseToLocal("2017-12-22T07:15:34.582910"), "alligator"), + record(schema, 2L, parseToLocal("2017-12-22T06:02:09.243857"), "forrest"), + record(schema, 3L, parseToLocal("2017-12-22T03:10:11.134509"), "clapping"), + record(schema, 4L, parseToLocal("2017-12-22T00:34:00.184671"), "brush"), + record(schema, 5L, parseToLocal("2017-12-21T22:20:08.935889"), "trap"), + record(schema, 6L, parseToLocal("2017-12-21T21:55:30.589712"), "element"), + record(schema, 7L, parseToLocal("2017-12-21T17:31:14.532797"), "limited"), + record(schema, 8L, parseToLocal("2017-12-21T15:21:51.237521"), "global"), + record(schema, 9L, parseToLocal("2017-12-21T15:02:15.230570"), "goldfish") + ); + } + + private static List read(String table, boolean vectorized) { + return read(table, vectorized, "*"); + } + + private static List read(String table, boolean vectorized, String select0, String... selectN) { + Dataset dataset = spark.read().format("iceberg") + .option("vectorization-enabled", String.valueOf(vectorized)) + .option(SparkUtil.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE, "true") + .load(table) + .select(select0, selectN); + return dataset.collectAsList(); + } + + private static LocalDateTime parseToLocal(String timestamp) { + return LocalDateTime.parse(timestamp); + } + + private static Record record(Schema schema, Object... values) { + Record rec = GenericRecord.create(schema); + for (int i = 0; i < values.length; i += 1) { + rec.set(i, values[i]); + } + return rec; + } +} diff --git a/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java index 3f6f8efd53e0..9716987ec439 100644 --- a/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java +++ b/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java @@ -82,7 +82,7 @@ public abstract class TestWriteMetricsConfig { @BeforeClass public static void startSpark() { TestWriteMetricsConfig.spark = SparkSession.builder().master("local[2]").getOrCreate(); - TestWriteMetricsConfig.sc = new JavaSparkContext(spark.sparkContext()); + TestWriteMetricsConfig.sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); } @AfterClass diff --git a/spark2/benchmark/.gitkeep b/spark2/benchmark/.gitkeep new file mode 100644 index 000000000000..042f3ce1f396 --- /dev/null +++ b/spark2/benchmark/.gitkeep @@ -0,0 +1,18 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ diff --git a/spark2/src/main/java/org/apache/iceberg/actions/SparkActions.java b/spark2/src/main/java/org/apache/iceberg/actions/SparkActions.java index 30ec698a7856..95b97546ebd0 100644 --- a/spark2/src/main/java/org/apache/iceberg/actions/SparkActions.java +++ b/spark2/src/main/java/org/apache/iceberg/actions/SparkActions.java @@ -22,6 +22,7 @@ import org.apache.iceberg.Table; import org.apache.spark.sql.SparkSession; +@Deprecated class SparkActions extends Actions { protected SparkActions(SparkSession spark, Table table) { super(spark, table); diff --git a/spark2/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java b/spark2/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java index 9b86e004c06d..022b81d4e9b1 100644 --- a/spark2/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java +++ b/spark2/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java @@ -83,6 +83,10 @@ public Optional createWriter(String jobId, StructType dsStruct "Save mode %s is not supported", mode); Configuration conf = new Configuration(lazyBaseConf()); Table table = getTableAndResolveHadoopConfiguration(options, conf); + boolean handleTimestampWithoutZone = + SparkUtil.canHandleTimestampWithoutZone(options.asMap(), lazySparkSession().conf()); + Preconditions.checkArgument(handleTimestampWithoutZone || !SparkUtil.hasTimestampWithoutZone(table.schema()), + SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR); Schema writeSchema = SparkSchemaUtil.convert(table.schema(), dsStruct); TypeUtil.validateWriteSchema(table.schema(), writeSchema, checkNullability(options), checkOrdering(options)); SparkUtil.validatePartitionTransforms(table.spec()); diff --git a/spark2/src/main/java/org/apache/iceberg/spark/source/Reader.java b/spark2/src/main/java/org/apache/iceberg/spark/source/Reader.java index 45a13f2762c4..edc03c3ef7e5 100644 --- a/spark2/src/main/java/org/apache/iceberg/spark/source/Reader.java +++ b/spark2/src/main/java/org/apache/iceberg/spark/source/Reader.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.io.Serializable; +import java.util.Arrays; import java.util.List; import java.util.Locale; import java.util.Map; @@ -50,8 +51,11 @@ import org.apache.iceberg.spark.SparkFilters; import org.apache.iceberg.spark.SparkReadOptions; import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.SparkUtil; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.TableScanUtil; +import org.apache.iceberg.util.Tasks; +import org.apache.iceberg.util.ThreadPools; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.broadcast.Broadcast; import org.apache.spark.sql.RuntimeConfig; @@ -95,6 +99,7 @@ class Reader implements DataSourceReader, SupportsScanColumnarBatch, SupportsPus private Filter[] pushedFilters = NO_FILTERS; private final boolean localityPreferred; private final int batchSize; + private final boolean readTimestampWithoutZone; // lazy variables private Schema schema = null; @@ -157,6 +162,8 @@ class Reader implements DataSourceReader, SupportsScanColumnarBatch, SupportsPus this.batchSize = options.get(SparkReadOptions.VECTORIZATION_BATCH_SIZE).map(Integer::parseInt).orElseGet(() -> PropertyUtil.propertyAsInt(table.properties(), TableProperties.PARQUET_BATCH_SIZE, TableProperties.PARQUET_BATCH_SIZE_DEFAULT)); + RuntimeConfig sessionConf = SparkSession.active().conf(); + this.readTimestampWithoutZone = SparkUtil.canHandleTimestampWithoutZone(options.asMap(), sessionConf); } private Schema lazySchema() { @@ -180,6 +187,8 @@ private Expression filterExpression() { private StructType lazyType() { if (type == null) { + Preconditions.checkArgument(readTimestampWithoutZone || !SparkUtil.hasTimestampWithoutZone(lazySchema()), + SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR); this.type = SparkSchemaUtil.convert(lazySchema()); } return type; @@ -205,15 +214,18 @@ public List> planBatchInputPartitions() { // broadcast the table metadata as input partitions will be sent to executors Broadcast tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table)); - List> readTasks = Lists.newArrayList(); - for (CombinedScanTask task : tasks()) { - readTasks.add(new ReadTask<>( - task, tableBroadcast, expectedSchemaString, caseSensitive, - localityPreferred, new BatchReaderFactory(batchSize))); - } - LOG.info("Batching input partitions with {} tasks.", readTasks.size()); + List scanTasks = tasks(); + InputPartition[] readTasks = new InputPartition[scanTasks.size()]; + + Tasks.range(readTasks.length) + .stopOnFailure() + .executeWith(localityPreferred ? ThreadPools.getWorkerPool() : null) + .run(index -> readTasks[index] = new ReadTask<>( + scanTasks.get(index), tableBroadcast, expectedSchemaString, caseSensitive, + localityPreferred, new BatchReaderFactory(batchSize))); + LOG.info("Batching input partitions with {} tasks.", readTasks.length); - return readTasks; + return Arrays.asList(readTasks); } /** @@ -226,14 +238,17 @@ public List> planInputPartitions() { // broadcast the table metadata as input partitions will be sent to executors Broadcast
      tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table)); - List> readTasks = Lists.newArrayList(); - for (CombinedScanTask task : tasks()) { - readTasks.add(new ReadTask<>( - task, tableBroadcast, expectedSchemaString, caseSensitive, - localityPreferred, InternalRowReaderFactory.INSTANCE)); - } + List scanTasks = tasks(); + InputPartition[] readTasks = new InputPartition[scanTasks.size()]; + + Tasks.range(readTasks.length) + .stopOnFailure() + .executeWith(localityPreferred ? ThreadPools.getWorkerPool() : null) + .run(index -> readTasks[index] = new ReadTask<>( + scanTasks.get(index), tableBroadcast, expectedSchemaString, caseSensitive, + localityPreferred, InternalRowReaderFactory.INSTANCE)); - return readTasks; + return Arrays.asList(readTasks); } @Override diff --git a/spark2/src/test/java/org/apache/iceberg/actions/TestRemoveReachableFilesAction24.java b/spark2/src/test/java/org/apache/iceberg/actions/TestDeleteReachableFilesAction24.java similarity index 94% rename from spark2/src/test/java/org/apache/iceberg/actions/TestRemoveReachableFilesAction24.java rename to spark2/src/test/java/org/apache/iceberg/actions/TestDeleteReachableFilesAction24.java index 134a64eb8512..80c31e9aceea 100644 --- a/spark2/src/test/java/org/apache/iceberg/actions/TestRemoveReachableFilesAction24.java +++ b/spark2/src/test/java/org/apache/iceberg/actions/TestDeleteReachableFilesAction24.java @@ -21,7 +21,7 @@ import org.apache.iceberg.spark.actions.SparkActions; -public class TestRemoveReachableFilesAction24 extends TestRemoveReachableFilesAction { +public class TestDeleteReachableFilesAction24 extends TestDeleteReachableFilesAction { @Override ActionsProvider sparkActions() { diff --git a/spark2/src/test/java/org/apache/iceberg/spark/source/TestSparkTableUtil.java b/spark2/src/test/java/org/apache/iceberg/spark/source/TestSparkTableUtil.java index 967f7b86e298..1c9362a26133 100644 --- a/spark2/src/test/java/org/apache/iceberg/spark/source/TestSparkTableUtil.java +++ b/spark2/src/test/java/org/apache/iceberg/spark/source/TestSparkTableUtil.java @@ -22,6 +22,9 @@ import java.io.File; import java.io.IOException; import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.iceberg.FileFormat; @@ -34,6 +37,7 @@ import org.apache.iceberg.mapping.NameMappingParser; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.SparkTableUtil; import org.apache.iceberg.spark.SparkTableUtil.SparkPartition; @@ -369,6 +373,78 @@ public void testImportUnpartitionedWithWhitespace() throws Exception { } } + public static class GetPartitions { + + @Rule + public TemporaryFolder temp = new TemporaryFolder(); + + // This logic does not really depend on format + private final FileFormat format = FileFormat.PARQUET; + + @Test + public void testPartitionScan() throws Exception { + + List records = Lists.newArrayList( + new ThreeColumnRecord(1, "ab", "data"), + new ThreeColumnRecord(2, "b c", "data"), + new ThreeColumnRecord(1, "b c", "data"), + new ThreeColumnRecord(2, "ab", "data")); + + String tableName = "external_table"; + + spark.createDataFrame(records, ThreeColumnRecord.class) + .write().mode("overwrite").format(format.toString()) + .partitionBy("c1", "c2").saveAsTable(tableName); + + TableIdentifier source = spark.sessionState().sqlParser() + .parseTableIdentifier(tableName); + + Map partition1 = ImmutableMap.of( + "c1", "1", + "c2", "ab"); + Map partition2 = ImmutableMap.of( + "c1", "2", + "c2", "b c"); + Map partition3 = ImmutableMap.of( + "c1", "1", + "c2", "b c"); + Map partition4 = ImmutableMap.of( + "c1", "2", + "c2", "ab"); + + List partitionsC11 = + SparkTableUtil.getPartitions(spark, source, ImmutableMap.of("c1", "1")); + Set> expectedC11 = + Sets.newHashSet(partition1, partition3); + Set> actualC11 = partitionsC11.stream().map( + p -> p.getValues()).collect(Collectors.toSet()); + Assert.assertEquals("Wrong partitions fetched for c1=1", expectedC11, actualC11); + + List partitionsC12 = + SparkTableUtil.getPartitions(spark, source, ImmutableMap.of("c1", "2")); + Set> expectedC12 = Sets.newHashSet(partition2, partition4); + Set> actualC12 = partitionsC12.stream().map( + p -> p.getValues()).collect(Collectors.toSet()); + Assert.assertEquals("Wrong partitions fetched for c1=2", expectedC12, actualC12); + + List partitionsC21 = + SparkTableUtil.getPartitions(spark, source, ImmutableMap.of("c2", "ab")); + Set> expectedC21 = + Sets.newHashSet(partition1, partition4); + Set> actualC21 = partitionsC21.stream().map( + p -> p.getValues()).collect(Collectors.toSet()); + Assert.assertEquals("Wrong partitions fetched for c2=ab", expectedC21, actualC21); + + List partitionsC22 = + SparkTableUtil.getPartitions(spark, source, ImmutableMap.of("c2", "b c")); + Set> expectedC22 = + Sets.newHashSet(partition2, partition3); + Set> actualC22 = partitionsC22.stream().map( + p -> p.getValues()).collect(Collectors.toSet()); + Assert.assertEquals("Wrong partitions fetched for c2=b c", expectedC22, actualC22); + } + } + public static class PartitionScan { @Before diff --git a/spark2/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone24.java b/spark2/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone24.java new file mode 100644 index 000000000000..84c7d1aeb733 --- /dev/null +++ b/spark2/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone24.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.spark.source; + +public class TestTimestampWithoutZone24 extends TestTimestampWithoutZone { + public TestTimestampWithoutZone24(String format, boolean vectorized) { + super(format, vectorized); + } +} diff --git a/spark3-extensions/src/main/antlr/org.apache.spark.sql.catalyst.parser.extensions/IcebergSqlExtensions.g4 b/spark3-extensions/src/main/antlr/org.apache.spark.sql.catalyst.parser.extensions/IcebergSqlExtensions.g4 index 40c8de813cbd..d0b228df0a2f 100644 --- a/spark3-extensions/src/main/antlr/org.apache.spark.sql.catalyst.parser.extensions/IcebergSqlExtensions.g4 +++ b/spark3-extensions/src/main/antlr/org.apache.spark.sql.catalyst.parser.extensions/IcebergSqlExtensions.g4 @@ -71,6 +71,8 @@ statement | ALTER TABLE multipartIdentifier DROP PARTITION FIELD transform #dropPartitionField | ALTER TABLE multipartIdentifier REPLACE PARTITION FIELD transform WITH transform (AS name=identifier)? #replacePartitionField | ALTER TABLE multipartIdentifier WRITE writeSpec #setWriteDistributionAndOrdering + | ALTER TABLE multipartIdentifier SET IDENTIFIER_KW FIELDS fieldList #setIdentifierFields + | ALTER TABLE multipartIdentifier DROP IDENTIFIER_KW FIELDS fieldList #dropIdentifierFields ; writeSpec @@ -157,9 +159,13 @@ quotedIdentifier : BACKQUOTED_IDENTIFIER ; +fieldList + : fields+=multipartIdentifier (',' fields+=multipartIdentifier)* + ; + nonReserved : ADD | ALTER | AS | ASC | BY | CALL | DESC | DROP | FIELD | FIRST | LAST | NULLS | ORDERED | PARTITION | TABLE | WRITE - | DISTRIBUTED | LOCALLY | UNORDERED + | DISTRIBUTED | LOCALLY | UNORDERED | REPLACE | WITH | IDENTIFIER_KW | FIELDS | SET | TRUE | FALSE | MAP ; @@ -174,6 +180,7 @@ DESC: 'DESC'; DISTRIBUTED: 'DISTRIBUTED'; DROP: 'DROP'; FIELD: 'FIELD'; +FIELDS: 'FIELDS'; FIRST: 'FIRST'; LAST: 'LAST'; LOCALLY: 'LOCALLY'; @@ -181,6 +188,8 @@ NULLS: 'NULLS'; ORDERED: 'ORDERED'; PARTITION: 'PARTITION'; REPLACE: 'REPLACE'; +IDENTIFIER_KW: 'IDENTIFIER'; +SET: 'SET'; TABLE: 'TABLE'; UNORDERED: 'UNORDERED'; WITH: 'WITH'; diff --git a/spark3-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala b/spark3-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala index 78b17d355f2a..30b5df5317db 100644 --- a/spark3-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala +++ b/spark3-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala @@ -41,7 +41,7 @@ class IcebergSparkSessionExtensions extends (SparkSessionExtensions => Unit) { // analyzer extensions extensions.injectResolutionRule { spark => ResolveProcedures(spark) } extensions.injectResolutionRule { _ => ProcedureArgumentCoercion } - extensions.injectPostHocResolutionRule { spark => AlignRowLevelOperations(spark.sessionState.conf)} + extensions.injectPostHocResolutionRule { spark => AlignRowLevelOperations } extensions.injectCheckRule { _ => RowLevelOperationsPredicateCheck } // optimizer extensions diff --git a/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignRowLevelOperations.scala b/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignRowLevelOperations.scala index e6864b29ec66..6da3ba6323e2 100644 --- a/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignRowLevelOperations.scala +++ b/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignRowLevelOperations.scala @@ -31,7 +31,10 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.utils.PlanUtils.isIcebergRelation import org.apache.spark.sql.internal.SQLConf -case class AlignRowLevelOperations(conf: SQLConf) extends Rule[LogicalPlan] with AssignmentAlignmentSupport { +case object AlignRowLevelOperations extends Rule[LogicalPlan] + with AssignmentAlignmentSupport with CastSupport { + + override def conf: SQLConf = SQLConf.get override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case u: UpdateTable if u.resolved && isIcebergRelation(u.table)=> diff --git a/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala b/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala index 673d28706486..c1140df1d295 100644 --- a/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala +++ b/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala @@ -32,13 +32,17 @@ import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.expressions.NamedExpression import org.apache.spark.sql.catalyst.plans.logical.Assignment import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.utils.RewriteRowLevelOperationHelper.createAlias +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy import org.apache.spark.sql.types.DataType import org.apache.spark.sql.types.StructField import org.apache.spark.sql.types.StructType import scala.collection.mutable -trait AssignmentAlignmentSupport extends CastSupport { +trait AssignmentAlignmentSupport { + + def conf: SQLConf private case class ColumnUpdate(ref: Seq[String], expr: Expression) @@ -96,7 +100,7 @@ trait AssignmentAlignmentSupport extends CastSupport { case StructType(fields) => // build field expressions val fieldExprs = fields.zipWithIndex.map { case (field, ordinal) => - Alias(GetStructField(col, ordinal, Some(field.name)), field.name)() + createAlias(GetStructField(col, ordinal, Some(field.name)), field.name) } // recursively apply this method on nested fields diff --git a/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDelete.scala b/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDelete.scala index 32257ff4e05f..e5a826506edc 100644 --- a/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDelete.scala +++ b/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDelete.scala @@ -28,7 +28,6 @@ import org.apache.spark.sql.catalyst.expressions.EqualNullSafe import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.expressions.Not -import org.apache.spark.sql.catalyst.expressions.SortOrder import org.apache.spark.sql.catalyst.expressions.SubqueryExpression import org.apache.spark.sql.catalyst.plans.logical.DeleteFromTable import org.apache.spark.sql.catalyst.plans.logical.Filter @@ -38,6 +37,7 @@ import org.apache.spark.sql.catalyst.plans.logical.RepartitionByExpression import org.apache.spark.sql.catalyst.plans.logical.ReplaceData import org.apache.spark.sql.catalyst.plans.logical.Sort import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.utils.DistributionAndOrderingUtils import org.apache.spark.sql.catalyst.utils.PlanUtils.isIcebergRelation import org.apache.spark.sql.catalyst.utils.RewriteRowLevelOperationHelper import org.apache.spark.sql.connector.catalog.Table @@ -52,6 +52,9 @@ case class RewriteDelete(spark: SparkSession) extends Rule[LogicalPlan] with Rew import ExtendedDataSourceV2Implicits._ import RewriteRowLevelOperationHelper._ + import DistributionAndOrderingUtils._ + + override def conf: SQLConf = SQLConf.get override def apply(plan: LogicalPlan): LogicalPlan = plan transform { // don't rewrite deletes that can be answered by passing filters to deleteWhere in SupportsDelete @@ -66,7 +69,7 @@ case class RewriteDelete(spark: SparkSession) extends Rule[LogicalPlan] with Rew val mergeBuilder = r.table.asMergeable.newMergeBuilder("delete", writeInfo) val matchingRowsPlanBuilder = scanRelation => Filter(cond, scanRelation) - val scanPlan = buildDynamicFilterScanPlan(spark, r.table, r.output, mergeBuilder, cond, matchingRowsPlanBuilder) + val scanPlan = buildDynamicFilterScanPlan(spark, r, r.output, mergeBuilder, cond, matchingRowsPlanBuilder) val remainingRowFilter = Not(EqualNullSafe(cond, Literal(true, BooleanType))) val remainingRowsPlan = Filter(remainingRowFilter, scanPlan) @@ -91,11 +94,11 @@ case class RewriteDelete(spark: SparkSession) extends Rule[LogicalPlan] with Rew remainingRowsPlan case _ => // apply hash partitioning by file if the distribution mode is hash or range - val numShufflePartitions = SQLConf.get.numShufflePartitions + val numShufflePartitions = conf.numShufflePartitions RepartitionByExpression(Seq(fileNameCol), remainingRowsPlan, numShufflePartitions) } - val order = Seq(SortOrder(fileNameCol, Ascending), SortOrder(rowPosCol, Ascending)) + val order = Seq(createSortOrder(fileNameCol, Ascending), createSortOrder(rowPosCol, Ascending)) val sort = Sort(order, global = false, planWithDistribution) Project(output, sort) } diff --git a/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala b/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala index c30e60332ee9..a08adfbf5913 100644 --- a/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala +++ b/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala @@ -55,9 +55,12 @@ import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implici import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.BooleanType -case class RewriteMergeInto(spark: SparkSession) extends Rule[LogicalPlan] with RewriteRowLevelOperationHelper { +case class RewriteMergeInto(spark: SparkSession) extends Rule[LogicalPlan] with RewriteRowLevelOperationHelper { import ExtendedDataSourceV2Implicits._ import RewriteMergeInto._ + import RewriteRowLevelOperationHelper._ + + override def conf: SQLConf = SQLConf.get override def apply(plan: LogicalPlan): LogicalPlan = { plan transform { @@ -79,7 +82,7 @@ case class RewriteMergeInto(spark: SparkSession) extends Rule[LogicalPlan] with val outputExprs = insertAction.assignments.map(_.value) val outputColNames = target.output.map(_.name) - val outputCols = outputExprs.zip(outputColNames).map { case (expr, name) => Alias(expr, name)() } + val outputCols = outputExprs.zip(outputColNames).map { case (expr, name) => createAlias(expr, name) } val mergePlan = Project(outputCols, joinPlan) val writePlan = buildWritePlan(mergePlan, target.table) @@ -121,7 +124,7 @@ case class RewriteMergeInto(spark: SparkSession) extends Rule[LogicalPlan] with // when there are no not-matched actions, use a right outer join to ignore source rows that do not match, but // keep all unmatched target rows that must be preserved. - val sourceTableProj = source.output ++ Seq(Alias(TRUE_LITERAL, ROW_FROM_SOURCE)()) + val sourceTableProj = source.output ++ Seq(createAlias(TRUE_LITERAL, ROW_FROM_SOURCE)) val newSourceTableScan = Project(sourceTableProj, source) val targetTableScan = buildDynamicFilterTargetScan(mergeBuilder, target, source, cond, matchedActions) val joinPlan = Join(newSourceTableScan, targetTableScan, RightOuter, Some(cond), JoinHint.NONE) @@ -151,10 +154,10 @@ case class RewriteMergeInto(spark: SparkSession) extends Rule[LogicalPlan] with val (matchedConditions, matchedOutputs) = rewriteMatchedActions(matchedActions, target.output) // use a full outer join because there are both matched and not matched actions - val sourceTableProj = source.output ++ Seq(Alias(TRUE_LITERAL, ROW_FROM_SOURCE)()) + val sourceTableProj = source.output ++ Seq(createAlias(TRUE_LITERAL, ROW_FROM_SOURCE)) val newSourceTableScan = Project(sourceTableProj, source) val targetTableScan = buildDynamicFilterTargetScan(mergeBuilder, target, source, cond, matchedActions) - val targetTableProj = targetTableScan.output ++ Seq(Alias(TRUE_LITERAL, ROW_FROM_TARGET)()) + val targetTableProj = targetTableScan.output ++ Seq(createAlias(TRUE_LITERAL, ROW_FROM_TARGET)) val newTargetTableScan = Project(targetTableProj, targetTableScan) val joinPlan = Join(newSourceTableScan, newTargetTableScan, FullOuter, Some(cond), JoinHint.NONE) @@ -202,7 +205,7 @@ case class RewriteMergeInto(spark: SparkSession) extends Rule[LogicalPlan] with val output = target.output val matchingRowsPlanBuilder = rel => Join(source, rel, Inner, Some(cond), JoinHint.NONE) val runCardinalityCheck = isCardinalityCheckEnabled(table) && isCardinalityCheckNeeded(matchedActions) - buildDynamicFilterScanPlan(spark, table, output, mergeBuilder, cond, matchingRowsPlanBuilder, runCardinalityCheck) + buildDynamicFilterScanPlan(spark, target, output, mergeBuilder, cond, matchingRowsPlanBuilder, runCardinalityCheck) } private def rewriteMatchedActions( diff --git a/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteUpdate.scala b/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteUpdate.scala index fce761807532..de5984cbcc34 100644 --- a/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteUpdate.scala +++ b/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteUpdate.scala @@ -40,11 +40,15 @@ import org.apache.spark.sql.catalyst.utils.PlanUtils.isIcebergRelation import org.apache.spark.sql.catalyst.utils.RewriteRowLevelOperationHelper import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.BooleanType case class RewriteUpdate(spark: SparkSession) extends Rule[LogicalPlan] with RewriteRowLevelOperationHelper { import ExtendedDataSourceV2Implicits._ + import RewriteRowLevelOperationHelper._ + + override def conf: SQLConf = SQLConf.get // TODO: can we do any better for no-op updates? when conditions evaluate to false/true? override def apply(plan: LogicalPlan): LogicalPlan = plan transform { @@ -59,7 +63,7 @@ case class RewriteUpdate(spark: SparkSession) extends Rule[LogicalPlan] with Rew // so the first job uses DynamicFileFilter and the second one uses the underlying scan plan // both jobs share the same SparkMergeScan instance to ensure they operate on same files val matchingRowsPlanBuilder = scanRelation => Filter(cond, scanRelation) - val scanPlan = buildDynamicFilterScanPlan(spark, r.table, r.output, mergeBuilder, cond, matchingRowsPlanBuilder) + val scanPlan = buildDynamicFilterScanPlan(spark, r, r.output, mergeBuilder, cond, matchingRowsPlanBuilder) val underlyingScanPlan = scanPlan match { case DynamicFileFilter(plan, _, _) => plan.clone() case _ => scanPlan.clone() @@ -85,7 +89,7 @@ case class RewriteUpdate(spark: SparkSession) extends Rule[LogicalPlan] with Rew val mergeBuilder = r.table.asMergeable.newMergeBuilder("update", writeInfo) val matchingRowsPlanBuilder = scanRelation => Filter(cond, scanRelation) - val scanPlan = buildDynamicFilterScanPlan(spark, r.table, r.output, mergeBuilder, cond, matchingRowsPlanBuilder) + val scanPlan = buildDynamicFilterScanPlan(spark, r, r.output, mergeBuilder, cond, matchingRowsPlanBuilder) val updateProjection = buildUpdateProjection(r, scanPlan, assignments, cond) @@ -110,10 +114,10 @@ case class RewriteUpdate(spark: SparkSession) extends Rule[LogicalPlan] with Rew if (attr.semanticEquals(assignedExpr)) { attr } else if (cond == Literal.TrueLiteral) { - Alias(assignedExpr, attr.name)() + createAlias(assignedExpr, attr.name) } else { val updatedExpr = If(cond, assignedExpr, attr) - Alias(updatedExpr, attr.name)() + createAlias(updatedExpr, attr.name) } } diff --git a/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala b/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala index 11a8017bb1a5..1379dacec51f 100644 --- a/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala +++ b/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala @@ -25,6 +25,8 @@ import org.antlr.v4.runtime.atn.PredictionMode import org.antlr.v4.runtime.misc.Interval import org.antlr.v4.runtime.misc.ParseCancellationException import org.antlr.v4.runtime.tree.TerminalNodeImpl +import org.apache.iceberg.common.DynConstructors +import org.apache.iceberg.spark.Spark3VersionUtil import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.TableIdentifier @@ -41,7 +43,9 @@ import org.apache.spark.sql.types.StructType class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserInterface { - private lazy val substitutor = new VariableSubstitution(SQLConf.get) + import IcebergSparkSqlExtensionsParser._ + + private lazy val substitutor = substitutorCtor.newInstance(SQLConf.get) private lazy val astBuilder = new IcebergSqlExtensionsAstBuilder(delegate) /** @@ -54,9 +58,7 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI /** * Parse a string to a raw DataType without CHAR/VARCHAR replacement. */ - override def parseRawDataType(sqlText: String): DataType = { - delegate.parseRawDataType(sqlText) - } + def parseRawDataType(sqlText: String): DataType = throw new UnsupportedOperationException() /** * Parse a string to an Expression. @@ -107,7 +109,7 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI } private def isIcebergCommand(sqlText: String): Boolean = { - val normalized = sqlText.toLowerCase(Locale.ROOT).trim() + val normalized = sqlText.toLowerCase(Locale.ROOT).trim().replaceAll("\\s+", " ") normalized.startsWith("call") || ( normalized.startsWith("alter table") && ( normalized.contains("add partition field") || @@ -116,7 +118,9 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI normalized.contains("write ordered by") || normalized.contains("write locally ordered by") || normalized.contains("write distributed by") || - normalized.contains("write unordered"))) + normalized.contains("write unordered") || + normalized.contains("set identifier fields") || + normalized.contains("drop identifier fields"))) } protected def parse[T](command: String)(toResult: IcebergSqlExtensionsParser => T): T = { @@ -159,6 +163,14 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI } } +object IcebergSparkSqlExtensionsParser { + private val substitutorCtor: DynConstructors.Ctor[VariableSubstitution] = + DynConstructors.builder() + .impl(classOf[VariableSubstitution]) + .impl(classOf[VariableSubstitution], classOf[SQLConf]) + .build() +} + /* Copied from Apache Spark's to avoid dependency on Spark Internals */ class UpperCaseCharStream(wrapped: CodePointCharStream) extends CharStream { override def consume(): Unit = wrapped.consume diff --git a/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala b/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala index 8ecd1f02c8cc..678da9bfc345 100644 --- a/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala +++ b/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala @@ -37,11 +37,13 @@ import org.apache.spark.sql.catalyst.parser.extensions.IcebergSqlExtensionsParse import org.apache.spark.sql.catalyst.plans.logical.AddPartitionField import org.apache.spark.sql.catalyst.plans.logical.CallArgument import org.apache.spark.sql.catalyst.plans.logical.CallStatement +import org.apache.spark.sql.catalyst.plans.logical.DropIdentifierFields import org.apache.spark.sql.catalyst.plans.logical.DropPartitionField import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.logical.NamedArgument import org.apache.spark.sql.catalyst.plans.logical.PositionalArgument import org.apache.spark.sql.catalyst.plans.logical.ReplacePartitionField +import org.apache.spark.sql.catalyst.plans.logical.SetIdentifierFields import org.apache.spark.sql.catalyst.plans.logical.SetWriteDistributionAndOrdering import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.catalyst.trees.Origin @@ -85,7 +87,7 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS /** - * Create an CHANGE PARTITION FIELD logical command. + * Create an REPLACE PARTITION FIELD logical command. */ override def visitReplacePartitionField(ctx: ReplacePartitionFieldContext): ReplacePartitionField = withOrigin(ctx) { ReplacePartitionField( @@ -95,6 +97,24 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS Option(ctx.name).map(_.getText)) } + /** + * Create an SET IDENTIFIER FIELDS logical command. + */ + override def visitSetIdentifierFields(ctx: SetIdentifierFieldsContext): SetIdentifierFields = withOrigin(ctx) { + SetIdentifierFields( + typedVisit[Seq[String]](ctx.multipartIdentifier), + ctx.fieldList.fields.asScala.map(_.getText)) + } + + /** + * Create an DROP IDENTIFIER FIELDS logical command. + */ + override def visitDropIdentifierFields(ctx: DropIdentifierFieldsContext): DropIdentifierFields = withOrigin(ctx) { + DropIdentifierFields( + typedVisit[Seq[String]](ctx.multipartIdentifier), + ctx.fieldList.fields.asScala.map(_.getText)) + } + /** * Create a [[SetWriteDistributionAndOrdering]] for changing the write distribution and ordering. */ diff --git a/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala b/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala new file mode 100644 index 000000000000..115af1586d7a --- /dev/null +++ b/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.catalyst.plans.logical + +import org.apache.spark.sql.catalyst.expressions.Attribute + +case class DropIdentifierFields( + table: Seq[String], + fields: Seq[String]) extends Command { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override lazy val output: Seq[Attribute] = Nil + + override def simpleString(maxFields: Int): String = { + s"DropIdentifierFields ${table.quoted} (${fields.quoted})" + } +} diff --git a/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplaceData.scala b/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplaceData.scala index 2535e70d9dee..16fd559c05ab 100644 --- a/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplaceData.scala +++ b/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplaceData.scala @@ -25,4 +25,12 @@ import org.apache.spark.sql.connector.write.BatchWrite case class ReplaceData( table: NamedRelation, write: BatchWrite, - query: LogicalPlan) extends V2WriteCommand + query: LogicalPlan) extends V2WriteCommand { + + def isByName: Boolean = false + + def withNewQuery(newQuery: LogicalPlan): ReplaceData = copy(query = newQuery) + + def withNewTable(newTable: NamedRelation): ReplaceData = copy(table = newTable) + +} diff --git a/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala b/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala new file mode 100644 index 000000000000..2e9a34b87204 --- /dev/null +++ b/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.catalyst.plans.logical + +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.expressions.Transform + +case class SetIdentifierFields( + table: Seq[String], + fields: Seq[String]) extends Command { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override lazy val output: Seq[Attribute] = Nil + + override def simpleString(maxFields: Int): String = { + s"SetIdentifierFields ${table.quoted} (${fields.quoted})" + } +} diff --git a/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/DistributionAndOrderingUtils.scala b/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/DistributionAndOrderingUtils.scala index ad4673056b4e..bcef977ca90c 100644 --- a/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/DistributionAndOrderingUtils.scala +++ b/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/DistributionAndOrderingUtils.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.catalyst.utils +import org.apache.iceberg.common.DynConstructors +import org.apache.iceberg.spark.Spark3VersionUtil import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst import org.apache.spark.sql.catalyst.analysis.Resolver @@ -58,6 +60,18 @@ import org.apache.spark.sql.types.IntegerType object DistributionAndOrderingUtils { + private val repartitionByExpressionCtor: DynConstructors.Ctor[RepartitionByExpression] = + DynConstructors.builder() + .impl(classOf[RepartitionByExpression], + classOf[Seq[catalyst.expressions.Expression]], + classOf[LogicalPlan], + classOf[Option[Int]]) + .impl(classOf[RepartitionByExpression], + classOf[Seq[catalyst.expressions.Expression]], + classOf[LogicalPlan], + Integer.TYPE) + .build() + def prepareQuery( requiredDistribution: Distribution, requiredOrdering: Seq[SortOrder], @@ -80,7 +94,11 @@ object DistributionAndOrderingUtils { // the conversion to catalyst expressions above produces SortOrder expressions // for OrderedDistribution and generic expressions for ClusteredDistribution // this allows RepartitionByExpression to pick either range or hash partitioning - RepartitionByExpression(distribution, query, numShufflePartitions) + if (Spark3VersionUtil.isSpark30) { + repartitionByExpressionCtor.newInstance(distribution.toSeq, query, Integer.valueOf(numShufflePartitions)) + } else { + repartitionByExpressionCtor.newInstance(distribution.toSeq, query, Some(numShufflePartitions)) + } } else { query } @@ -98,6 +116,37 @@ object DistributionAndOrderingUtils { queryWithDistributionAndOrdering } + private val sortOrderCtor: DynConstructors.Ctor[catalyst.expressions.SortOrder] = + DynConstructors.builder() + .impl(classOf[catalyst.expressions.SortOrder], + classOf[catalyst.expressions.Expression], + classOf[catalyst.expressions.SortDirection], + classOf[catalyst.expressions.NullOrdering], + classOf[Seq[catalyst.expressions.Expression]]) + .impl(classOf[catalyst.expressions.SortOrder], + classOf[catalyst.expressions.Expression], + classOf[catalyst.expressions.SortDirection], + classOf[catalyst.expressions.NullOrdering], + classOf[Set[catalyst.expressions.Expression]]) + .build() + + def createSortOrder( + child: catalyst.expressions.Expression, + direction: catalyst.expressions.SortDirection): catalyst.expressions.SortOrder = { + createSortOrder(child, direction, direction.defaultNullOrdering) + } + + def createSortOrder( + child: catalyst.expressions.Expression, + direction: catalyst.expressions.SortDirection, + nullOrdering: catalyst.expressions.NullOrdering): catalyst.expressions.SortOrder = { + if (Spark3VersionUtil.isSpark30) { + sortOrderCtor.newInstance(child, direction, nullOrdering, Set.empty) + } else { + sortOrderCtor.newInstance(child, direction, nullOrdering, Seq.empty) + } + } + private def toCatalyst( expr: Expression, query: LogicalPlan, @@ -118,7 +167,7 @@ object DistributionAndOrderingUtils { expr match { case s: SortOrder => val catalystChild = toCatalyst(s.expression(), query, resolver) - catalyst.expressions.SortOrder(catalystChild, toCatalyst(s.direction), toCatalyst(s.nullOrdering), Set.empty) + createSortOrder(catalystChild, toCatalyst(s.direction), toCatalyst(s.nullOrdering)) case it: IdentityTransform => resolve(it.ref.fieldNames) case BucketTransform(numBuckets, ref) => diff --git a/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/RewriteRowLevelOperationHelper.scala b/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/RewriteRowLevelOperationHelper.scala index 19a950aff57e..910e9ce8c2f5 100644 --- a/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/RewriteRowLevelOperationHelper.scala +++ b/spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/RewriteRowLevelOperationHelper.scala @@ -19,7 +19,9 @@ package org.apache.spark.sql.catalyst.utils import java.util.UUID +import org.apache.iceberg.common.DynConstructors import org.apache.iceberg.spark.Spark3Util +import org.apache.iceberg.spark.Spark3VersionUtil import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.SparkSession @@ -31,8 +33,10 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.expressions.AttributeSet import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.ExprId import org.apache.spark.sql.catalyst.expressions.GreaterThan import org.apache.spark.sql.catalyst.expressions.Literal +import org.apache.spark.sql.catalyst.expressions.NamedExpression import org.apache.spark.sql.catalyst.expressions.PredicateHelper import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.expressions.aggregate.Complete @@ -48,6 +52,7 @@ import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.connector.iceberg.distributions.OrderedDistribution import org.apache.spark.sql.connector.iceberg.read.SupportsFileFilter import org.apache.spark.sql.connector.iceberg.write.MergeBuilder +import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.connector.read.ScanBuilder import org.apache.spark.sql.connector.write.LogicalWriteInfo import org.apache.spark.sql.connector.write.LogicalWriteInfoImpl @@ -59,6 +64,7 @@ import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implici import org.apache.spark.sql.execution.datasources.v2.PushDownUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources +import org.apache.spark.sql.types.Metadata import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -69,7 +75,7 @@ trait RewriteRowLevelOperationHelper extends PredicateHelper with Logging { import ExtendedDataSourceV2Implicits.ScanBuilderHelper protected def spark: SparkSession - protected lazy val conf: SQLConf = spark.sessionState.conf + def conf: SQLConf protected lazy val resolver: Resolver = conf.resolver protected def buildSimpleScanPlan( @@ -83,14 +89,14 @@ trait RewriteRowLevelOperationHelper extends PredicateHelper with Logging { val scan = scanBuilder.asIceberg.withMetadataColumns(FILE_NAME_COL, ROW_POS_COL).build() val outputAttrs = toOutputAttrs(scan.readSchema(), relation.output) val predicates = extractFilters(cond, relation.output).reduceLeftOption(And) - val scanRelation = DataSourceV2ScanRelation(relation.table, scan, outputAttrs) + val scanRelation = createScanRelation(relation, scan, outputAttrs) predicates.map(Filter(_, scanRelation)).getOrElse(scanRelation) } protected def buildDynamicFilterScanPlan( spark: SparkSession, - table: Table, + relation: DataSourceV2Relation, tableAttrs: Seq[AttributeReference], mergeBuilder: MergeBuilder, cond: Expression, @@ -103,7 +109,7 @@ trait RewriteRowLevelOperationHelper extends PredicateHelper with Logging { val scan = scanBuilder.asIceberg.withMetadataColumns(FILE_NAME_COL, ROW_POS_COL).build() val outputAttrs = toOutputAttrs(scan.readSchema(), tableAttrs) - val scanRelation = DataSourceV2ScanRelation(table, scan, outputAttrs) + val scanRelation = createScanRelation(relation, scan, outputAttrs) scan match { case filterable: SupportsFileFilter if runCardinalityCheck => @@ -171,11 +177,11 @@ trait RewriteRowLevelOperationHelper extends PredicateHelper with Logging { prunedTargetPlan: LogicalPlan): LogicalPlan = { val fileAttr = findOutputAttr(prunedTargetPlan.output, FILE_NAME_COL) val rowPosAttr = findOutputAttr(prunedTargetPlan.output, ROW_POS_COL) - val accumulatorExpr = Alias(AccumulateFiles(filesAccumulator, fileAttr), AFFECTED_FILES_ACC_ALIAS_NAME)() + val accumulatorExpr = createAlias(AccumulateFiles(filesAccumulator, fileAttr), AFFECTED_FILES_ACC_ALIAS_NAME) val projectList = Seq(fileAttr, rowPosAttr, accumulatorExpr) val projectPlan = Project(projectList, prunedTargetPlan) val affectedFilesAttr = findOutputAttr(projectPlan.output, AFFECTED_FILES_ACC_ALIAS_NAME) - val aggSumCol = Alias(AggregateExpression(Sum(affectedFilesAttr), Complete, false), SUM_ROW_ID_ALIAS_NAME)() + val aggSumCol = createAlias(AggregateExpression(Sum(affectedFilesAttr), Complete, false), SUM_ROW_ID_ALIAS_NAME) // Group by the rows by row id while collecting the files that need to be over written via accumulator. val aggPlan = Aggregate(Seq(fileAttr, rowPosAttr), Seq(aggSumCol), projectPlan) val sumAttr = findOutputAttr(aggPlan.output, SUM_ROW_ID_ALIAS_NAME) @@ -229,4 +235,48 @@ object RewriteRowLevelOperationHelper { private final val AFFECTED_FILES_ACC_NAME = "internal.metrics.merge.affectedFiles" private final val AFFECTED_FILES_ACC_ALIAS_NAME = "_affectedFiles_" private final val SUM_ROW_ID_ALIAS_NAME = "_sum_" + + private val scanRelationCtor: DynConstructors.Ctor[DataSourceV2ScanRelation] = + DynConstructors.builder() + .impl(classOf[DataSourceV2ScanRelation], + classOf[DataSourceV2Relation], + classOf[Scan], + classOf[Seq[AttributeReference]]) + .impl(classOf[DataSourceV2ScanRelation], + classOf[Table], + classOf[Scan], + classOf[Seq[AttributeReference]]) + .build() + + def createScanRelation( + relation: DataSourceV2Relation, + scan: Scan, + outputAttrs: Seq[AttributeReference]): DataSourceV2ScanRelation = { + if (Spark3VersionUtil.isSpark30) { + scanRelationCtor.newInstance(relation.table, scan, outputAttrs) + } else { + scanRelationCtor.newInstance(relation, scan, outputAttrs) + } + } + + private val aliasCtor: DynConstructors.Ctor[Alias] = + DynConstructors.builder() + .impl(classOf[Alias], + classOf[Expression], + classOf[String], + classOf[ExprId], + classOf[Seq[String]], + classOf[Option[Metadata]], + classOf[Seq[String]]) + .impl(classOf[Alias], + classOf[Expression], + classOf[String], + classOf[ExprId], + classOf[Seq[String]], + classOf[Option[Metadata]]) + .build() + + def createAlias(child: Expression, name: String): Alias = { + aliasCtor.newInstance(child, name, NamedExpression.newExprId, Seq.empty, None, Seq.empty) + } } diff --git a/spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala b/spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala new file mode 100644 index 000000000000..525ed77437a5 --- /dev/null +++ b/spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.execution.datasources.v2 + +import org.apache.iceberg.relocated.com.google.common.base.Preconditions +import org.apache.iceberg.relocated.com.google.common.collect.Sets +import org.apache.iceberg.spark.source.SparkTable +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.connector.catalog.TableCatalog + +case class DropIdentifierFieldsExec( + catalog: TableCatalog, + ident: Identifier, + fields: Seq[String]) extends V2CommandExec { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override lazy val output: Seq[Attribute] = Nil + + override protected def run(): Seq[InternalRow] = { + catalog.loadTable(ident) match { + case iceberg: SparkTable => + val schema = iceberg.table.schema + val identifierFieldNames = Sets.newHashSet(schema.identifierFieldNames) + + for (name <- fields) { + Preconditions.checkArgument(schema.findField(name) != null, + "Cannot complete drop identifier fields operation: field %s not found", name) + Preconditions.checkArgument(identifierFieldNames.contains(name), + "Cannot complete drop identifier fields operation: %s is not an identifier field", name) + identifierFieldNames.remove(name) + } + + iceberg.table.updateSchema() + .setIdentifierFields(identifierFieldNames) + .commit(); + case table => + throw new UnsupportedOperationException(s"Cannot drop identifier fields in non-Iceberg table: $table") + } + + Nil + } + + override def simpleString(maxFields: Int): String = { + s"DropIdentifierFields ${catalog.name}.${ident.quoted} (${fields.quoted})"; + } +} diff --git a/spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala b/spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala index d5901a8446ce..6f0361fdb150 100644 --- a/spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala +++ b/spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.expressions.NamedExpression import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.AddPartitionField import org.apache.spark.sql.catalyst.plans.logical.Call +import org.apache.spark.sql.catalyst.plans.logical.DropIdentifierFields import org.apache.spark.sql.catalyst.plans.logical.DropPartitionField import org.apache.spark.sql.catalyst.plans.logical.DynamicFileFilter import org.apache.spark.sql.catalyst.plans.logical.DynamicFileFilterWithCardinalityCheck @@ -40,6 +41,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.logical.MergeInto import org.apache.spark.sql.catalyst.plans.logical.ReplaceData import org.apache.spark.sql.catalyst.plans.logical.ReplacePartitionField +import org.apache.spark.sql.catalyst.plans.logical.SetIdentifierFields import org.apache.spark.sql.catalyst.plans.logical.SetWriteDistributionAndOrdering import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.TableCatalog @@ -66,6 +68,12 @@ case class ExtendedDataSourceV2Strategy(spark: SparkSession) extends Strategy { case ReplacePartitionField(IcebergCatalogAndIdentifier(catalog, ident), transformFrom, transformTo, name) => ReplacePartitionFieldExec(catalog, ident, transformFrom, transformTo, name) :: Nil + case SetIdentifierFields(IcebergCatalogAndIdentifier(catalog, ident), fields) => + SetIdentifierFieldsExec(catalog, ident, fields) :: Nil + + case DropIdentifierFields(IcebergCatalogAndIdentifier(catalog, ident), fields) => + DropIdentifierFieldsExec(catalog, ident, fields) :: Nil + case SetWriteDistributionAndOrdering( IcebergCatalogAndIdentifier(catalog, ident), distributionMode, ordering) => SetWriteDistributionAndOrderingExec(catalog, ident, distributionMode, ordering) :: Nil diff --git a/spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala b/spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala new file mode 100644 index 000000000000..7fad2dc016d2 --- /dev/null +++ b/spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.execution.datasources.v2 + +import org.apache.iceberg.spark.source.SparkTable +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.connector.catalog.TableCatalog + +case class SetIdentifierFieldsExec( + catalog: TableCatalog, + ident: Identifier, + fields: Seq[String]) extends V2CommandExec { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override lazy val output: Seq[Attribute] = Nil + + override protected def run(): Seq[InternalRow] = { + catalog.loadTable(ident) match { + case iceberg: SparkTable => + iceberg.table.updateSchema() + .setIdentifierFields(scala.collection.JavaConverters.seqAsJavaList(fields)) + .commit(); + case table => + throw new UnsupportedOperationException(s"Cannot set identifier fields in non-Iceberg table: $table") + } + + Nil + } + + override def simpleString(maxFields: Int): String = { + s"SetIdentifierFields ${catalog.name}.${ident.quoted} (${fields.quoted})"; + } +} diff --git a/spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java b/spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java index edc47f64c320..dea01a10a647 100644 --- a/spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java +++ b/spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java @@ -318,6 +318,26 @@ public void addFilteredPartitionsToPartitioned() { sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName)); } + @Test + public void addFilteredPartitionsToPartitioned2() { + createCompositePartitionedTable("parquet"); + + String createIceberg = + "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg " + + "PARTITIONED BY (id, dept)"; + + sql(createIceberg, tableName); + + Object result = scalarSql("CALL %s.system.add_files('%s', '`parquet`.`%s`', map('dept', 'hr'))", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + Assert.assertEquals(6L, result); + + assertEquals("Iceberg table contains correct data", + sql("SELECT id, name, dept, subdept FROM %s WHERE dept = 'hr' ORDER BY id", sourceTableName), + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName)); + } + @Test public void addWeirdCaseHiveTable() { createWeirdCaseTable(); diff --git a/spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java b/spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java index af9b837b3039..9d630508b6e4 100644 --- a/spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java +++ b/spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java @@ -265,7 +265,8 @@ public void testDropPartitionByName() { Assert.assertEquals("Table should have 1 partition field", 1, table.spec().fields().size()); - sql("ALTER TABLE %s DROP PARTITION FIELD shard", tableName); + // Should be recognized as iceberg command even with extra white spaces + sql("ALTER TABLE %s DROP PARTITION \n FIELD shard", tableName); table.refresh(); diff --git a/spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java b/spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java new file mode 100644 index 000000000000..ac12953d0a7e --- /dev/null +++ b/spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.spark.extensions; + +import java.util.Map; +import org.apache.iceberg.AssertHelpers; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +public class TestAlterTableSchema extends SparkExtensionsTestBase { + public TestAlterTableSchema(String catalogName, String implementation, Map config) { + super(catalogName, implementation, config); + } + + @After + public void removeTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @Test + public void testSetIdentifierFields() { + sql("CREATE TABLE %s (id bigint NOT NULL, " + + "location struct NOT NULL) USING iceberg", tableName); + Table table = validationCatalog.loadTable(tableIdent); + Assert.assertTrue("Table should start without identifier", table.schema().identifierFieldIds().isEmpty()); + + sql("ALTER TABLE %s SET IDENTIFIER FIELDS id", tableName); + table.refresh(); + Assert.assertEquals("Should have new identifier field", + Sets.newHashSet(table.schema().findField("id").fieldId()), + table.schema().identifierFieldIds()); + + sql("ALTER TABLE %s SET IDENTIFIER FIELDS id, location.lon", tableName); + table.refresh(); + Assert.assertEquals("Should have new identifier field", + Sets.newHashSet( + table.schema().findField("id").fieldId(), + table.schema().findField("location.lon").fieldId()), + table.schema().identifierFieldIds()); + + sql("ALTER TABLE %s SET IDENTIFIER FIELDS location.lon", tableName); + table.refresh(); + Assert.assertEquals("Should have new identifier field", + Sets.newHashSet(table.schema().findField("location.lon").fieldId()), + table.schema().identifierFieldIds()); + } + + @Test + public void testSetInvalidIdentifierFields() { + sql("CREATE TABLE %s (id bigint NOT NULL, id2 bigint) USING iceberg", tableName); + Table table = validationCatalog.loadTable(tableIdent); + Assert.assertTrue("Table should start without identifier", table.schema().identifierFieldIds().isEmpty()); + AssertHelpers.assertThrows("should not allow setting unknown fields", + IllegalArgumentException.class, + "not found in current schema or added columns", + () -> sql("ALTER TABLE %s SET IDENTIFIER FIELDS unknown", tableName)); + + AssertHelpers.assertThrows("should not allow setting optional fields", + IllegalArgumentException.class, + "not a required field", + () -> sql("ALTER TABLE %s SET IDENTIFIER FIELDS id2", tableName)); + } + + @Test + public void testDropIdentifierFields() { + sql("CREATE TABLE %s (id bigint NOT NULL, " + + "location struct NOT NULL) USING iceberg", tableName); + Table table = validationCatalog.loadTable(tableIdent); + Assert.assertTrue("Table should start without identifier", table.schema().identifierFieldIds().isEmpty()); + + sql("ALTER TABLE %s SET IDENTIFIER FIELDS id, location.lon", tableName); + table.refresh(); + Assert.assertEquals("Should have new identifier fields", + Sets.newHashSet( + table.schema().findField("id").fieldId(), + table.schema().findField("location.lon").fieldId()), + table.schema().identifierFieldIds()); + + sql("ALTER TABLE %s DROP IDENTIFIER FIELDS id", tableName); + table.refresh(); + Assert.assertEquals("Should removed identifier field", + Sets.newHashSet(table.schema().findField("location.lon").fieldId()), + table.schema().identifierFieldIds()); + + sql("ALTER TABLE %s SET IDENTIFIER FIELDS id, location.lon", tableName); + table.refresh(); + Assert.assertEquals("Should have new identifier fields", + Sets.newHashSet( + table.schema().findField("id").fieldId(), + table.schema().findField("location.lon").fieldId()), + table.schema().identifierFieldIds()); + + sql("ALTER TABLE %s DROP IDENTIFIER FIELDS id, location.lon", tableName); + table.refresh(); + Assert.assertEquals("Should have no identifier field", + Sets.newHashSet(), + table.schema().identifierFieldIds()); + } + + @Test + public void testDropInvalidIdentifierFields() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string NOT NULL, " + + "location struct NOT NULL) USING iceberg", tableName); + Table table = validationCatalog.loadTable(tableIdent); + Assert.assertTrue("Table should start without identifier", table.schema().identifierFieldIds().isEmpty()); + AssertHelpers.assertThrows("should not allow dropping unknown fields", + IllegalArgumentException.class, + "field unknown not found", + () -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS unknown", tableName)); + + sql("ALTER TABLE %s SET IDENTIFIER FIELDS id", tableName); + AssertHelpers.assertThrows("should not allow dropping a field that is not an identifier", + IllegalArgumentException.class, + "data is not an identifier field", + () -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS data", tableName)); + + AssertHelpers.assertThrows("should not allow dropping a nested field that is not an identifier", + IllegalArgumentException.class, + "location.lon is not an identifier field", + () -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS location.lon", tableName)); + } +} diff --git a/spark3/benchmark/.gitkeep b/spark3/benchmark/.gitkeep new file mode 100644 index 000000000000..042f3ce1f396 --- /dev/null +++ b/spark3/benchmark/.gitkeep @@ -0,0 +1,18 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ diff --git a/spark3/src/main/java/org/apache/iceberg/actions/SparkActions.java b/spark3/src/main/java/org/apache/iceberg/actions/SparkActions.java index 960feb5887d8..ac1a1efc6276 100644 --- a/spark3/src/main/java/org/apache/iceberg/actions/SparkActions.java +++ b/spark3/src/main/java/org/apache/iceberg/actions/SparkActions.java @@ -26,15 +26,18 @@ import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.connector.catalog.CatalogPlugin; +@Deprecated public class SparkActions extends Actions { protected SparkActions(SparkSession spark, Table table) { super(spark, table); } + @Deprecated public static CreateAction migrate(String tableName) { return migrate(SparkSession.active(), tableName); } + @Deprecated public static CreateAction migrate(SparkSession spark, String tableName) { CatalogPlugin defaultCatalog = spark.sessionState().catalogManager().currentCatalog(); CatalogAndIdentifier catalogAndIdentifier; @@ -42,10 +45,12 @@ public static CreateAction migrate(SparkSession spark, String tableName) { return new Spark3MigrateAction(spark, catalogAndIdentifier.catalog(), catalogAndIdentifier.identifier()); } + @Deprecated public static SnapshotAction snapshot(String sourceId, String destId) { return snapshot(SparkSession.active(), sourceId, destId); } + @Deprecated public static SnapshotAction snapshot(SparkSession spark, String sourceId, String destId) { CatalogPlugin defaultCatalog = spark.sessionState().catalogManager().currentCatalog(); CatalogAndIdentifier sourceIdent = Spark3Util.catalogAndIdentifier("snapshot source", spark, sourceId, diff --git a/spark3/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java b/spark3/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java index e35b533edb56..02a272c0412d 100644 --- a/spark3/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java +++ b/spark3/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java @@ -23,6 +23,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.HasTableOperations; @@ -71,6 +72,8 @@ public void commitRewrite(Table table, Set fileSetIDs) { table.newRewrite() .rewriteFiles(rewrittenDataFiles, newDataFiles) .commit(); + + fileSetIDs.stream().map(id -> toID(table, id)).forEach(resultMap::remove); } private Set fetchRewrittenDataFiles(Table table, Set fileSetIDs) { @@ -93,7 +96,7 @@ private Set fetchRewrittenDataFiles(Table table, Set fileSetID return Collections.unmodifiableSet(rewrittenDataFiles); } - private Set fetchNewDataFiles(Table table, Set fileSetIDs) { + public Set fetchNewDataFiles(Table table, Set fileSetIDs) { List> results = Lists.newArrayList(); for (String fileSetID : fileSetIDs) { @@ -114,6 +117,11 @@ private Set fetchNewDataFiles(Table table, Set fileSetIDs) { return newDataFiles; } + public void clearRewrite(Table table, String fileSetID) { + Pair id = toID(table, fileSetID); + resultMap.remove(id); + } + public void abortRewrite(Table table, String fileSetID) { Pair id = toID(table, fileSetID); Set dataFiles = resultMap.remove(id); @@ -129,6 +137,13 @@ public void abortRewrite(Table table, Set fileSetIDs) { } } + public Set fetchSetIDs(Table table) { + return resultMap.keySet().stream() + .filter(e -> e.first().equals(tableUUID(table))) + .map(Pair::second) + .collect(Collectors.toSet()); + } + private void deleteFiles(FileIO io, Iterable dataFiles) { Tasks.foreach(dataFiles) .noRetry() diff --git a/spark3/src/main/java/org/apache/iceberg/spark/FileScanTaskSetManager.java b/spark3/src/main/java/org/apache/iceberg/spark/FileScanTaskSetManager.java index b43f1c6bab3e..827b674ca16d 100644 --- a/spark3/src/main/java/org/apache/iceberg/spark/FileScanTaskSetManager.java +++ b/spark3/src/main/java/org/apache/iceberg/spark/FileScanTaskSetManager.java @@ -21,6 +21,8 @@ import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.Table; @@ -58,9 +60,19 @@ public List removeTasks(Table table, String setID) { return tasksMap.remove(id); } - private Pair toID(Table table, String setID) { + public Set fetchSetIDs(Table table) { + return tasksMap.keySet().stream() + .filter(e -> e.first().equals(tableUUID(table))) + .map(Pair::second) + .collect(Collectors.toSet()); + } + + private String tableUUID(Table table) { TableOperations ops = ((HasTableOperations) table).operations(); - String tableUUID = ops.current().uuid(); - return Pair.of(tableUUID, setID); + return ops.current().uuid(); + } + + private Pair toID(Table table, String setID) { + return Pair.of(tableUUID(table), setID); } } diff --git a/spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java b/spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java index 9a24b3e5ffb6..7b52ba118890 100644 --- a/spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java +++ b/spark3/src/main/java/org/apache/iceberg/spark/Spark3Util.java @@ -562,6 +562,19 @@ public static Integer propertyAsInt(CaseInsensitiveStringMap options, String pro return null; } + public static Boolean propertyAsBoolean(CaseInsensitiveStringMap options, String property, Boolean defaultValue) { + if (defaultValue != null) { + return options.getBoolean(property, defaultValue); + } + + String value = options.get(property); + if (value != null) { + return Boolean.parseBoolean(value); + } + + return null; + } + public static class DescribeSchemaVisitor extends TypeUtil.SchemaVisitor { private static final Joiner COMMA = Joiner.on(','); private static final DescribeSchemaVisitor INSTANCE = new DescribeSchemaVisitor(); diff --git a/spark3/src/main/java/org/apache/iceberg/spark/Spark3VersionUtil.java b/spark3/src/main/java/org/apache/iceberg/spark/Spark3VersionUtil.java new file mode 100644 index 000000000000..984c66d22a82 --- /dev/null +++ b/spark3/src/main/java/org/apache/iceberg/spark/Spark3VersionUtil.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.spark; + +import org.apache.spark.package$; + +public class Spark3VersionUtil { + + private Spark3VersionUtil() { + } + + public static boolean isSpark30() { + return package$.MODULE$.SPARK_VERSION_SHORT().startsWith("3.0"); + } + + public static boolean isSpark31() { + return package$.MODULE$.SPARK_VERSION_SHORT().startsWith("3.1"); + } +} diff --git a/spark3/src/main/java/org/apache/iceberg/spark/SparkCatalog.java b/spark3/src/main/java/org/apache/iceberg/spark/SparkCatalog.java index ac7e6e989c68..d099291fc353 100644 --- a/spark3/src/main/java/org/apache/iceberg/spark/SparkCatalog.java +++ b/spark3/src/main/java/org/apache/iceberg/spark/SparkCatalog.java @@ -72,6 +72,7 @@ *
    • uri - the Hive Metastore URI (Hive catalog only)
    • *
    • warehouse - the warehouse path (Hadoop catalog only)
    • *
    • default-namespace - a namespace to use as the default
    • + *
    • cache-enabled - whether to enable catalog cache
    • * *

      * To use a custom catalog that is not a Hive or Hadoop catalog, extend this class and override @@ -86,6 +87,7 @@ public class SparkCatalog extends BaseCatalog { private SupportsNamespaces asNamespaceCatalog = null; private String[] defaultNamespace = null; private HadoopTables tables; + private boolean useTimestampsWithoutZone; /** * Build an Iceberg {@link Catalog} to be used by this Spark catalog adapter. @@ -127,7 +129,7 @@ public SparkTable loadTable(Identifier ident) throws NoSuchTableException { public SparkTable createTable(Identifier ident, StructType schema, Transform[] transforms, Map properties) throws TableAlreadyExistsException { - Schema icebergSchema = SparkSchemaUtil.convert(schema); + Schema icebergSchema = SparkSchemaUtil.convert(schema, useTimestampsWithoutZone); try { Catalog.TableBuilder builder = newBuilder(ident, icebergSchema); Table icebergTable = builder @@ -144,7 +146,7 @@ public SparkTable createTable(Identifier ident, StructType schema, @Override public StagedTable stageCreate(Identifier ident, StructType schema, Transform[] transforms, Map properties) throws TableAlreadyExistsException { - Schema icebergSchema = SparkSchemaUtil.convert(schema); + Schema icebergSchema = SparkSchemaUtil.convert(schema, useTimestampsWithoutZone); try { Catalog.TableBuilder builder = newBuilder(ident, icebergSchema); Transaction transaction = builder.withPartitionSpec(Spark3Util.toPartitionSpec(icebergSchema, transforms)) @@ -160,7 +162,7 @@ public StagedTable stageCreate(Identifier ident, StructType schema, Transform[] @Override public StagedTable stageReplace(Identifier ident, StructType schema, Transform[] transforms, Map properties) throws NoSuchTableException { - Schema icebergSchema = SparkSchemaUtil.convert(schema); + Schema icebergSchema = SparkSchemaUtil.convert(schema, useTimestampsWithoutZone); try { Catalog.TableBuilder builder = newBuilder(ident, icebergSchema); Transaction transaction = builder.withPartitionSpec(Spark3Util.toPartitionSpec(icebergSchema, transforms)) @@ -176,7 +178,7 @@ public StagedTable stageReplace(Identifier ident, StructType schema, Transform[] @Override public StagedTable stageCreateOrReplace(Identifier ident, StructType schema, Transform[] transforms, Map properties) { - Schema icebergSchema = SparkSchemaUtil.convert(schema); + Schema icebergSchema = SparkSchemaUtil.convert(schema, useTimestampsWithoutZone); Catalog.TableBuilder builder = newBuilder(ident, icebergSchema); Transaction transaction = builder.withPartitionSpec(Spark3Util.toPartitionSpec(icebergSchema, transforms)) .withLocation(properties.get("location")) @@ -386,7 +388,9 @@ public final void initialize(String name, CaseInsensitiveStringMap options) { Catalog catalog = buildIcebergCatalog(name, options); this.catalogName = name; - this.tables = new HadoopTables(SparkSession.active().sessionState().newHadoopConf()); + SparkSession sparkSession = SparkSession.active(); + this.useTimestampsWithoutZone = SparkUtil.useTimestampWithoutZoneInNewTables(sparkSession.conf()); + this.tables = new HadoopTables(sparkSession.sessionState().newHadoopConf()); this.icebergCatalog = cacheEnabled ? CachingCatalog.wrap(catalog) : catalog; if (catalog instanceof SupportsNamespaces) { this.asNamespaceCatalog = (SupportsNamespaces) catalog; diff --git a/spark3/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSpark3Action.java b/spark3/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSpark3Action.java new file mode 100644 index 000000000000..ac2224f43e3f --- /dev/null +++ b/spark3/src/main/java/org/apache/iceberg/spark/actions/BaseRewriteDataFilesSpark3Action.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.spark.actions; + +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.BinPackStrategy; +import org.apache.iceberg.actions.RewriteDataFiles; +import org.apache.spark.sql.SparkSession; + +public class BaseRewriteDataFilesSpark3Action extends BaseRewriteDataFilesSparkAction { + + protected BaseRewriteDataFilesSpark3Action(SparkSession spark, Table table) { + super(spark, table); + } + + @Override + protected BinPackStrategy binPackStrategy() { + return new Spark3BinPackStrategy(table(), spark()); + } + + @Override + protected RewriteDataFiles self() { + return this; + } +} diff --git a/spark3/src/main/java/org/apache/iceberg/spark/actions/Spark3BinPackStrategy.java b/spark3/src/main/java/org/apache/iceberg/spark/actions/Spark3BinPackStrategy.java new file mode 100644 index 000000000000..b8e5079530c7 --- /dev/null +++ b/spark3/src/main/java/org/apache/iceberg/spark/actions/Spark3BinPackStrategy.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.spark.actions; + +import java.util.List; +import java.util.Set; +import java.util.UUID; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.BinPackStrategy; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.spark.FileRewriteCoordinator; +import org.apache.iceberg.spark.FileScanTaskSetManager; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.internal.SQLConf; + +public class Spark3BinPackStrategy extends BinPackStrategy { + private final Table table; + private final SparkSession spark; + private final FileScanTaskSetManager manager = FileScanTaskSetManager.get(); + private final FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get(); + + public Spark3BinPackStrategy(Table table, SparkSession spark) { + this.table = table; + this.spark = spark; + } + + @Override + public Table table() { + return table; + } + + @Override + public Set rewriteFiles(List filesToRewrite) { + String groupID = UUID.randomUUID().toString(); + try { + manager.stageTasks(table, groupID, filesToRewrite); + + // Disable Adaptive Query Execution as this may change the output partitioning of our write + SparkSession cloneSession = spark.cloneSession(); + cloneSession.conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false); + + Dataset scanDF = cloneSession.read().format("iceberg") + .option(SparkReadOptions.FILE_SCAN_TASK_SET_ID, groupID) + .option(SparkReadOptions.SPLIT_SIZE, splitSize(inputFileSize(filesToRewrite))) + .option(SparkReadOptions.FILE_OPEN_COST, "0") + .load(table.name()); + + // write the packed data into new files where each split becomes a new file + scanDF.write() + .format("iceberg") + .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, groupID) + .option(SparkWriteOptions.TARGET_FILE_SIZE_BYTES, writeMaxFileSize()) + .mode("append") + .save(table.name()); + + return rewriteCoordinator.fetchNewDataFiles(table, ImmutableSet.of(groupID)); + } finally { + manager.removeTasks(table, groupID); + rewriteCoordinator.clearRewrite(table, groupID); + } + } +} diff --git a/spark3/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java b/spark3/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java index 149784575edb..3230728261c9 100644 --- a/spark3/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java +++ b/spark3/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java @@ -19,8 +19,10 @@ package org.apache.iceberg.spark.actions; +import org.apache.iceberg.Table; import org.apache.iceberg.actions.ActionsProvider; import org.apache.iceberg.actions.MigrateTable; +import org.apache.iceberg.actions.RewriteDataFiles; import org.apache.iceberg.actions.SnapshotTable; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.Spark3Util.CatalogAndIdentifier; @@ -62,4 +64,9 @@ public MigrateTable migrateTable(String tableIdent) { CatalogAndIdentifier catalogAndIdent = Spark3Util.catalogAndIdentifier(ctx, spark(), tableIdent, defaultCatalog); return new BaseMigrateTableSparkAction(spark(), catalogAndIdent.catalog(), catalogAndIdent.identifier()); } + + @Override + public RewriteDataFiles rewriteDataFiles(Table table) { + return new BaseRewriteDataFilesSpark3Action(spark(), table); + } } diff --git a/spark3/src/main/java/org/apache/iceberg/spark/procedures/ExpireSnapshotsProcedure.java b/spark3/src/main/java/org/apache/iceberg/spark/procedures/ExpireSnapshotsProcedure.java index ba1e6b087750..88803f0dc21b 100644 --- a/spark3/src/main/java/org/apache/iceberg/spark/procedures/ExpireSnapshotsProcedure.java +++ b/spark3/src/main/java/org/apache/iceberg/spark/procedures/ExpireSnapshotsProcedure.java @@ -22,8 +22,8 @@ import org.apache.iceberg.actions.Actions; import org.apache.iceberg.actions.ExpireSnapshots; import org.apache.iceberg.spark.procedures.SparkProcedures.ProcedureBuilder; +import org.apache.iceberg.util.DateTimeUtil; import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.util.DateTimeUtils; import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.catalog.TableCatalog; import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter; @@ -77,7 +77,7 @@ public StructType outputType() { @Override public InternalRow[] call(InternalRow args) { Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name()); - Long olderThanMillis = args.isNullAt(1) ? null : DateTimeUtils.toMillis(args.getLong(1)); + Long olderThanMillis = args.isNullAt(1) ? null : DateTimeUtil.microsToMillis(args.getLong(1)); Integer retainLastNum = args.isNullAt(2) ? null : args.getInt(2); return modifyIcebergTable(tableIdent, table -> { diff --git a/spark3/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java b/spark3/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java index eea4951cfd08..84703db662d2 100644 --- a/spark3/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java +++ b/spark3/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java @@ -21,11 +21,11 @@ import java.util.concurrent.TimeUnit; import org.apache.iceberg.actions.Actions; -import org.apache.iceberg.actions.RemoveOrphanFiles; +import org.apache.iceberg.actions.DeleteOrphanFiles; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.spark.procedures.SparkProcedures.ProcedureBuilder; +import org.apache.iceberg.util.DateTimeUtil; import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.util.DateTimeUtils; import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.catalog.TableCatalog; import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter; @@ -79,12 +79,12 @@ public StructType outputType() { @Override public InternalRow[] call(InternalRow args) { Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name()); - Long olderThanMillis = args.isNullAt(1) ? null : DateTimeUtils.toMillis(args.getLong(1)); + Long olderThanMillis = args.isNullAt(1) ? null : DateTimeUtil.microsToMillis(args.getLong(1)); String location = args.isNullAt(2) ? null : args.getString(2); boolean dryRun = args.isNullAt(3) ? false : args.getBoolean(3); return withIcebergTable(tableIdent, table -> { - RemoveOrphanFiles action = actions().removeOrphanFiles(table); + DeleteOrphanFiles action = actions().deleteOrphanFiles(table); if (olderThanMillis != null) { boolean isTesting = Boolean.parseBoolean(spark().conf().get("spark.testing", "false")); @@ -102,13 +102,13 @@ public InternalRow[] call(InternalRow args) { action.deleteWith(file -> { }); } - RemoveOrphanFiles.Result result = action.execute(); + DeleteOrphanFiles.Result result = action.execute(); return toOutputRows(result); }); } - private InternalRow[] toOutputRows(RemoveOrphanFiles.Result result) { + private InternalRow[] toOutputRows(DeleteOrphanFiles.Result result) { Iterable orphanFileLocations = result.orphanFileLocations(); int orphanFileLocationsCount = Iterables.size(orphanFileLocations); diff --git a/spark3/src/main/java/org/apache/iceberg/spark/procedures/RollbackToTimestampProcedure.java b/spark3/src/main/java/org/apache/iceberg/spark/procedures/RollbackToTimestampProcedure.java index 3e38c3ebd305..94e894928c9b 100644 --- a/spark3/src/main/java/org/apache/iceberg/spark/procedures/RollbackToTimestampProcedure.java +++ b/spark3/src/main/java/org/apache/iceberg/spark/procedures/RollbackToTimestampProcedure.java @@ -21,8 +21,8 @@ import org.apache.iceberg.Snapshot; import org.apache.iceberg.spark.procedures.SparkProcedures.ProcedureBuilder; +import org.apache.iceberg.util.DateTimeUtil; import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.util.DateTimeUtils; import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.catalog.TableCatalog; import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter; @@ -77,7 +77,7 @@ public StructType outputType() { public InternalRow[] call(InternalRow args) { Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name()); // timestamps in Spark have nanosecond precision so this conversion is lossy - long timestampMillis = DateTimeUtils.toMillis(args.getLong(1)); + long timestampMillis = DateTimeUtil.microsToMillis(args.getLong(1)); return modifyIcebergTable(tableIdent, table -> { Snapshot previousSnapshot = table.currentSnapshot(); diff --git a/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchScan.java b/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchScan.java index 1999b568b9e0..acd02037aced 100644 --- a/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchScan.java +++ b/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchScan.java @@ -36,10 +36,14 @@ import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.hadoop.HadoopInputFile; import org.apache.iceberg.hadoop.Util; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.SparkUtil; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.TableScanUtil; +import org.apache.iceberg.util.Tasks; +import org.apache.iceberg.util.ThreadPools; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.broadcast.Broadcast; import org.apache.spark.sql.RuntimeConfig; @@ -52,6 +56,7 @@ import org.apache.spark.sql.connector.read.Scan; import org.apache.spark.sql.connector.read.Statistics; import org.apache.spark.sql.connector.read.SupportsReportStatistics; +import org.apache.spark.sql.connector.read.streaming.MicroBatchStream; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; import org.apache.spark.sql.vectorized.ColumnarBatch; @@ -68,6 +73,7 @@ abstract class SparkBatchScan implements Scan, Batch, SupportsReportStatistics { private final Schema expectedSchema; private final List filterExpressions; private final int batchSize; + private final boolean readTimestampWithoutZone; private final CaseInsensitiveStringMap options; // lazy variables @@ -83,6 +89,9 @@ abstract class SparkBatchScan implements Scan, Batch, SupportsReportStatistics { this.localityPreferred = Spark3Util.isLocalityEnabled(table.io(), table.location(), options); this.batchSize = Spark3Util.batchSize(table.properties(), options); this.options = options; + + RuntimeConfig sessionConf = SparkSession.active().conf(); + this.readTimestampWithoutZone = SparkUtil.canHandleTimestampWithoutZone(options, sessionConf); } protected Table table() { @@ -108,9 +117,17 @@ public Batch toBatch() { return this; } + @Override + public MicroBatchStream toMicroBatchStream(String checkpointLocation) { + return new SparkMicroBatchStream( + sparkContext, table, caseSensitive, expectedSchema, options, checkpointLocation); + } + @Override public StructType readSchema() { if (readSchema == null) { + Preconditions.checkArgument(readTimestampWithoutZone || !SparkUtil.hasTimestampWithoutZone(expectedSchema), + SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR); this.readSchema = SparkSchemaUtil.convert(expectedSchema); } return readSchema; @@ -125,11 +142,13 @@ public InputPartition[] planInputPartitions() { List scanTasks = tasks(); InputPartition[] readTasks = new InputPartition[scanTasks.size()]; - for (int i = 0; i < scanTasks.size(); i++) { - readTasks[i] = new ReadTask( - scanTasks.get(i), tableBroadcast, expectedSchemaString, - caseSensitive, localityPreferred); - } + + Tasks.range(readTasks.length) + .stopOnFailure() + .executeWith(localityPreferred ? ThreadPools.getWorkerPool() : null) + .run(index -> readTasks[index] = new ReadTask( + scanTasks.get(index), tableBroadcast, expectedSchemaString, + caseSensitive, localityPreferred)); return readTasks; } @@ -213,10 +232,10 @@ public String description() { return String.format("%s [filters=%s]", table, filters); } - private static class ReaderFactory implements PartitionReaderFactory { + static class ReaderFactory implements PartitionReaderFactory { private final int batchSize; - private ReaderFactory(int batchSize) { + ReaderFactory(int batchSize) { this.batchSize = batchSize; } @@ -256,7 +275,7 @@ private static class BatchReader extends BatchDataReader implements PartitionRea } } - private static class ReadTask implements InputPartition, Serializable { + static class ReadTask implements InputPartition, Serializable { private final CombinedScanTask task; private final Broadcast

      tableBroadcast; private final String expectedSchemaString; diff --git a/spark3/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java b/spark3/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java new file mode 100644 index 000000000000..700793271967 --- /dev/null +++ b/spark3/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java @@ -0,0 +1,271 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.spark.source; + +import java.io.BufferedWriter; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.OutputStreamWriter; +import java.io.UncheckedIOException; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Locale; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataOperations; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.MicroBatches; +import org.apache.iceberg.MicroBatches.MicroBatch; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.spark.source.SparkBatchScan.ReadTask; +import org.apache.iceberg.spark.source.SparkBatchScan.ReaderFactory; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.iceberg.util.TableScanUtil; +import org.apache.iceberg.util.Tasks; +import org.apache.iceberg.util.ThreadPools; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.broadcast.Broadcast; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.PartitionReaderFactory; +import org.apache.spark.sql.connector.read.streaming.MicroBatchStream; +import org.apache.spark.sql.connector.read.streaming.Offset; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.iceberg.TableProperties.SPLIT_LOOKBACK; +import static org.apache.iceberg.TableProperties.SPLIT_LOOKBACK_DEFAULT; +import static org.apache.iceberg.TableProperties.SPLIT_OPEN_FILE_COST; +import static org.apache.iceberg.TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT; +import static org.apache.iceberg.TableProperties.SPLIT_SIZE; +import static org.apache.iceberg.TableProperties.SPLIT_SIZE_DEFAULT; + +public class SparkMicroBatchStream implements MicroBatchStream { + private static final Joiner SLASH = Joiner.on("/"); + private static final Logger LOG = LoggerFactory.getLogger(SparkMicroBatchStream.class); + + private final Table table; + private final boolean caseSensitive; + private final String expectedSchema; + private final Broadcast
      tableBroadcast; + private final Long splitSize; + private final Integer splitLookback; + private final Long splitOpenFileCost; + private final boolean localityPreferred; + private final StreamingOffset initialOffset; + private final boolean skipDelete; + + SparkMicroBatchStream(JavaSparkContext sparkContext, Table table, boolean caseSensitive, + Schema expectedSchema, CaseInsensitiveStringMap options, String checkpointLocation) { + this.table = table; + this.caseSensitive = caseSensitive; + this.expectedSchema = SchemaParser.toJson(expectedSchema); + this.localityPreferred = Spark3Util.isLocalityEnabled(table.io(), table.location(), options); + this.tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table)); + + long tableSplitSize = PropertyUtil.propertyAsLong(table.properties(), SPLIT_SIZE, SPLIT_SIZE_DEFAULT); + this.splitSize = Spark3Util.propertyAsLong(options, SparkReadOptions.SPLIT_SIZE, tableSplitSize); + + int tableSplitLookback = PropertyUtil.propertyAsInt(table.properties(), SPLIT_LOOKBACK, SPLIT_LOOKBACK_DEFAULT); + this.splitLookback = Spark3Util.propertyAsInt(options, SparkReadOptions.LOOKBACK, tableSplitLookback); + + long tableSplitOpenFileCost = PropertyUtil.propertyAsLong( + table.properties(), SPLIT_OPEN_FILE_COST, SPLIT_OPEN_FILE_COST_DEFAULT); + this.splitOpenFileCost = Spark3Util.propertyAsLong( + options, SparkReadOptions.FILE_OPEN_COST, tableSplitOpenFileCost); + + InitialOffsetStore initialOffsetStore = new InitialOffsetStore(table, checkpointLocation); + this.initialOffset = initialOffsetStore.initialOffset(); + + this.skipDelete = Spark3Util.propertyAsBoolean(options, SparkReadOptions.STREAMING_SKIP_DELETE_SNAPSHOTS, false); + } + + @Override + public Offset latestOffset() { + table.refresh(); + Snapshot latestSnapshot = table.currentSnapshot(); + if (latestSnapshot == null) { + return StreamingOffset.START_OFFSET; + } + + return new StreamingOffset(latestSnapshot.snapshotId(), Iterables.size(latestSnapshot.addedFiles()), false); + } + + @Override + public InputPartition[] planInputPartitions(Offset start, Offset end) { + Preconditions.checkArgument(end instanceof StreamingOffset, "Invalid end offset: %s is not a StreamingOffset", end); + Preconditions.checkArgument( + start instanceof StreamingOffset, "Invalid start offset: %s is not a StreamingOffset", start); + + if (end.equals(StreamingOffset.START_OFFSET)) { + return new InputPartition[0]; + } + + StreamingOffset endOffset = (StreamingOffset) end; + StreamingOffset startOffset = (StreamingOffset) start; + + List fileScanTasks = planFiles(startOffset, endOffset); + + CloseableIterable splitTasks = TableScanUtil.splitFiles( + CloseableIterable.withNoopClose(fileScanTasks), + splitSize); + List combinedScanTasks = Lists.newArrayList( + TableScanUtil.planTasks(splitTasks, splitSize, splitLookback, splitOpenFileCost)); + InputPartition[] readTasks = new InputPartition[combinedScanTasks.size()]; + + Tasks.range(readTasks.length) + .stopOnFailure() + .executeWith(localityPreferred ? ThreadPools.getWorkerPool() : null) + .run(index -> readTasks[index] = new ReadTask( + combinedScanTasks.get(index), tableBroadcast, expectedSchema, + caseSensitive, localityPreferred)); + + return readTasks; + } + + @Override + public PartitionReaderFactory createReaderFactory() { + return new ReaderFactory(0); + } + + @Override + public Offset initialOffset() { + return initialOffset; + } + + @Override + public Offset deserializeOffset(String json) { + return StreamingOffset.fromJson(json); + } + + @Override + public void commit(Offset end) { + } + + @Override + public void stop() { + } + + private List planFiles(StreamingOffset startOffset, StreamingOffset endOffset) { + List fileScanTasks = Lists.newArrayList(); + StreamingOffset batchStartOffset = StreamingOffset.START_OFFSET.equals(startOffset) ? + new StreamingOffset(SnapshotUtil.oldestSnapshot(table).snapshotId(), 0, false) : + startOffset; + + StreamingOffset currentOffset = null; + + do { + if (currentOffset == null) { + currentOffset = batchStartOffset; + } else { + Snapshot snapshotAfter = SnapshotUtil.snapshotAfter(table, currentOffset.snapshotId()); + currentOffset = new StreamingOffset(snapshotAfter.snapshotId(), 0L, false); + } + + if (!shouldProcess(table.snapshot(currentOffset.snapshotId()))) { + LOG.debug("Skipping snapshot: {} of table {}", currentOffset.snapshotId(), table.name()); + continue; + } + + MicroBatch latestMicroBatch = MicroBatches.from(table.snapshot(currentOffset.snapshotId()), table.io()) + .caseSensitive(caseSensitive) + .specsById(table.specs()) + .generate(currentOffset.position(), Long.MAX_VALUE, currentOffset.shouldScanAllFiles()); + + fileScanTasks.addAll(latestMicroBatch.tasks()); + } while (currentOffset.snapshotId() != endOffset.snapshotId()); + + return fileScanTasks; + } + + private boolean shouldProcess(Snapshot snapshot) { + String op = snapshot.operation(); + Preconditions.checkState(!op.equals(DataOperations.DELETE) || skipDelete, + "Cannot process delete snapshot: %s", snapshot.snapshotId()); + Preconditions.checkState( + op.equals(DataOperations.DELETE) || op.equals(DataOperations.APPEND) || op.equals(DataOperations.REPLACE), + "Cannot process %s snapshot: %s", op.toLowerCase(Locale.ROOT), snapshot.snapshotId()); + return op.equals(DataOperations.APPEND); + } + + private static class InitialOffsetStore { + private final Table table; + private final FileIO io; + private final String initialOffsetLocation; + + InitialOffsetStore(Table table, String checkpointLocation) { + this.table = table; + this.io = table.io(); + this.initialOffsetLocation = SLASH.join(checkpointLocation, "offsets/0"); + } + + public StreamingOffset initialOffset() { + InputFile inputFile = io.newInputFile(initialOffsetLocation); + if (inputFile.exists()) { + return readOffset(inputFile); + } + + table.refresh(); + StreamingOffset offset = table.currentSnapshot() == null ? + StreamingOffset.START_OFFSET : + new StreamingOffset(SnapshotUtil.oldestSnapshot(table).snapshotId(), 0, false); + + OutputFile outputFile = io.newOutputFile(initialOffsetLocation); + writeOffset(offset, outputFile); + + return offset; + } + + private void writeOffset(StreamingOffset offset, OutputFile file) { + try (OutputStream outputStream = file.create()) { + BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(outputStream, StandardCharsets.UTF_8)); + writer.write(offset.json()); + writer.flush(); + } catch (IOException ioException) { + throw new UncheckedIOException( + String.format("Failed writing offset to: %s", initialOffsetLocation), ioException); + } + } + + private StreamingOffset readOffset(InputFile file) { + try (InputStream in = file.newStream()) { + return StreamingOffset.fromJson(in); + } catch (IOException ioException) { + throw new UncheckedIOException( + String.format("Failed reading offset from: %s", initialOffsetLocation), ioException); + } + } + } +} diff --git a/spark3/src/main/java/org/apache/iceberg/spark/source/SparkTable.java b/spark3/src/main/java/org/apache/iceberg/spark/source/SparkTable.java index 52fbf7a1bf2d..c535a3534954 100644 --- a/spark3/src/main/java/org/apache/iceberg/spark/source/SparkTable.java +++ b/spark3/src/main/java/org/apache/iceberg/spark/source/SparkTable.java @@ -70,6 +70,7 @@ public class SparkTable implements org.apache.spark.sql.connector.catalog.Table, private static final Set CAPABILITIES = ImmutableSet.of( TableCapability.BATCH_READ, TableCapability.BATCH_WRITE, + TableCapability.MICRO_BATCH_READ, TableCapability.STREAMING_WRITE, TableCapability.OVERWRITE_BY_FILTER, TableCapability.OVERWRITE_DYNAMIC); diff --git a/spark3/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java b/spark3/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java index 19debd53c9cb..b23e0a7935cf 100644 --- a/spark3/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java +++ b/spark3/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java @@ -51,6 +51,7 @@ class SparkWriteBuilder implements WriteBuilder, SupportsDynamicOverwrite, Suppo private final StructType dsSchema; private final CaseInsensitiveStringMap options; private final String overwriteMode; + private final boolean canHandleTimestampWithoutZone; private boolean overwriteDynamic = false; private boolean overwriteByFilter = false; private Expression overwriteExpr = null; @@ -66,6 +67,7 @@ class SparkWriteBuilder implements WriteBuilder, SupportsDynamicOverwrite, Suppo this.options = info.options(); this.overwriteMode = options.containsKey("overwrite-mode") ? options.get("overwrite-mode").toLowerCase(Locale.ROOT) : null; + this.canHandleTimestampWithoutZone = SparkUtil.canHandleTimestampWithoutZone(options, spark.conf()); } public WriteBuilder overwriteFiles(Scan scan, IsolationLevel writeIsolationLevel) { @@ -103,6 +105,9 @@ public WriteBuilder overwrite(Filter[] filters) { @Override public BatchWrite buildForBatch() { // Validate + Preconditions.checkArgument(canHandleTimestampWithoutZone || !SparkUtil.hasTimestampWithoutZone(table.schema()), + SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR); + Schema writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema); TypeUtil.validateWriteSchema(table.schema(), writeSchema, checkNullability(spark, options), checkOrdering(spark, options)); diff --git a/spark3/src/main/java/org/apache/iceberg/spark/source/StreamingOffset.java b/spark3/src/main/java/org/apache/iceberg/spark/source/StreamingOffset.java index f43578ada310..64277ecf3be5 100644 --- a/spark3/src/main/java/org/apache/iceberg/spark/source/StreamingOffset.java +++ b/spark3/src/main/java/org/apache/iceberg/spark/source/StreamingOffset.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.databind.JsonNode; import java.io.IOException; +import java.io.InputStream; import java.io.StringWriter; import java.io.UncheckedIOException; import org.apache.iceberg.relocated.com.google.common.base.Objects; @@ -62,20 +63,23 @@ static StreamingOffset fromJson(String json) { try { JsonNode node = JsonUtil.mapper().readValue(json, JsonNode.class); - // The version of StreamingOffset. The offset was created with a version number - // used to validate when deserializing from json string. - int version = JsonUtil.getInt(VERSION, node); - Preconditions.checkArgument(version == CURR_VERSION, - "Cannot parse offset JSON: offset version %s is not supported", version); + return fromJsonNode(node); + } catch (IOException e) { + throw new UncheckedIOException(String.format("Failed to parse StreamingOffset from JSON string %s", json), e); + } + } - long snapshotId = JsonUtil.getLong(SNAPSHOT_ID, node); - int position = JsonUtil.getInt(POSITION, node); - boolean shouldScanAllFiles = JsonUtil.getBool(SCAN_ALL_FILES, node); + static StreamingOffset fromJson(InputStream inputStream) { + Preconditions.checkNotNull(inputStream, "Cannot parse StreamingOffset from inputStream: null"); - return new StreamingOffset(snapshotId, position, shouldScanAllFiles); + JsonNode node; + try { + node = JsonUtil.mapper().readValue(inputStream, JsonNode.class); } catch (IOException e) { - throw new IllegalArgumentException(String.format("Failed to parse StreamingOffset from JSON string %s", json), e); + throw new UncheckedIOException("Failed to read StreamingOffset from json", e); } + + return fromJsonNode(node); } @Override @@ -132,4 +136,19 @@ public String toString() { return String.format("Streaming Offset[%d: position (%d) scan_all_files (%b)]", snapshotId, position, scanAllFiles); } + + private static StreamingOffset fromJsonNode(JsonNode node) { + // The version of StreamingOffset. The offset was created with a version number + // used to validate when deserializing from json string. + int version = JsonUtil.getInt(VERSION, node); + Preconditions.checkArgument(version == CURR_VERSION, + "This version of Iceberg source only supports version %s. Version %s is not supported.", + CURR_VERSION, version); + + long snapshotId = JsonUtil.getLong(SNAPSHOT_ID, node); + int position = JsonUtil.getInt(POSITION, node); + boolean shouldScanAllFiles = JsonUtil.getBool(SCAN_ALL_FILES, node); + + return new StreamingOffset(snapshotId, position, shouldScanAllFiles); + } } diff --git a/spark3/src/test/java/org/apache/iceberg/actions/TestNewRewriteDataFilesAction3.java b/spark3/src/test/java/org/apache/iceberg/actions/TestNewRewriteDataFilesAction3.java new file mode 100644 index 000000000000..19737f0c40b9 --- /dev/null +++ b/spark3/src/test/java/org/apache/iceberg/actions/TestNewRewriteDataFilesAction3.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.actions; + +import java.util.Set; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.spark.FileRewriteCoordinator; +import org.apache.iceberg.spark.FileScanTaskSetManager; +import org.apache.iceberg.spark.actions.SparkActions; +import org.apache.iceberg.spark.actions.TestNewRewriteDataFilesAction; + +public class TestNewRewriteDataFilesAction3 extends TestNewRewriteDataFilesAction { + private final FileRewriteCoordinator coordinator = FileRewriteCoordinator.get(); + private final FileScanTaskSetManager manager = FileScanTaskSetManager.get(); + + @Override + protected ActionsProvider actions() { + return SparkActions.get(); + } + + @Override + protected Set cacheContents(Table table) { + return ImmutableSet.builder() + .addAll(manager.fetchSetIDs(table)) + .addAll(coordinator.fetchSetIDs(table)) + .build(); + } +} diff --git a/spark3/src/test/java/org/apache/iceberg/actions/TestRemoveFilesAction3.java b/spark3/src/test/java/org/apache/iceberg/actions/TestRemoveFilesAction3.java index a7b4203e110f..838741a580b2 100644 --- a/spark3/src/test/java/org/apache/iceberg/actions/TestRemoveFilesAction3.java +++ b/spark3/src/test/java/org/apache/iceberg/actions/TestRemoveFilesAction3.java @@ -21,7 +21,7 @@ import org.apache.iceberg.spark.actions.SparkActions; -public class TestRemoveFilesAction3 extends TestRemoveReachableFilesAction { +public class TestRemoveFilesAction3 extends TestDeleteReachableFilesAction { @Override ActionsProvider sparkActions() { return SparkActions.get(); diff --git a/spark3/src/test/java/org/apache/iceberg/actions/TestRemoveOrphanFilesAction3.java b/spark3/src/test/java/org/apache/iceberg/actions/TestRemoveOrphanFilesAction3.java index fa429ff0cc1c..02a66424b07d 100644 --- a/spark3/src/test/java/org/apache/iceberg/actions/TestRemoveOrphanFilesAction3.java +++ b/spark3/src/test/java/org/apache/iceberg/actions/TestRemoveOrphanFilesAction3.java @@ -29,6 +29,7 @@ import org.apache.iceberg.spark.source.SparkTable; import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.expressions.Transform; +import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -131,10 +132,6 @@ public void testSparkSessionCatalogHadoopTable() throws Exception { .olderThan(System.currentTimeMillis() + 1000).execute(); Assert.assertTrue("trash file should be removed", results.contains("file:" + location + "/data/trashfile")); - // reset spark_catalog to default - spark.conf().unset("spark.sql.catalog.spark_catalog"); - spark.conf().unset("spark.sql.catalog.spark_catalog.type"); - spark.conf().unset("spark.sql.catalog.spark_catalog.warehouse"); } @Test @@ -160,9 +157,13 @@ public void testSparkSessionCatalogHiveTable() throws Exception { .olderThan(System.currentTimeMillis() + 1000).execute(); Assert.assertTrue("trash file should be removed", results.contains("file:" + location + "/data/trashfile")); - // reset spark_catalog to default + } + + @After + public void resetSparkSessionCatalog() throws Exception { spark.conf().unset("spark.sql.catalog.spark_catalog"); spark.conf().unset("spark.sql.catalog.spark_catalog.type"); + spark.conf().unset("spark.sql.catalog.spark_catalog.warehouse"); } } diff --git a/spark3/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java b/spark3/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java new file mode 100644 index 000000000000..4e50cf2a3e21 --- /dev/null +++ b/spark3/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java @@ -0,0 +1,431 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.spark.source; + +import java.io.File; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeoutException; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.AssertHelpers; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.DataOperations; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.data.FileHelpers; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.streaming.DataStreamWriter; +import org.apache.spark.sql.streaming.OutputMode; +import org.apache.spark.sql.streaming.StreamingQuery; +import org.apache.spark.sql.streaming.StreamingQueryException; +import org.apache.spark.sql.streaming.Trigger; +import org.assertj.core.api.Assertions; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.apache.iceberg.expressions.Expressions.ref; +import static org.apache.iceberg.types.Types.NestedField.optional; + +@RunWith(Parameterized.class) +public final class TestStructuredStreamingRead3 extends SparkCatalogTestBase { + public TestStructuredStreamingRead3( + String catalogName, String implementation, Map config) { + super(catalogName, implementation, config); + } + + private Table table; + private String tableIdentifier; + + private static final Configuration CONF = new Configuration(); + private static final Schema SCHEMA = new Schema( + optional(1, "id", Types.IntegerType.get()), + optional(2, "data", Types.StringType.get()) + ); + + /** + * test data to be used by multiple writes + * each write creates a snapshot and writes a list of records + */ + private static final List> TEST_DATA_MULTIPLE_SNAPSHOTS = Lists.newArrayList( + Lists.newArrayList( + new SimpleRecord(1, "one"), + new SimpleRecord(2, "two"), + new SimpleRecord(3, "three")), + Lists.newArrayList( + new SimpleRecord(4, "four"), + new SimpleRecord(5, "five")), + Lists.newArrayList( + new SimpleRecord(6, "six"), + new SimpleRecord(7, "seven"))); + + /** + * test data - to be used for multiple write batches + * each batch inturn will have multiple snapshots + */ + private static final List>> TEST_DATA_MULTIPLE_WRITES_MULTIPLE_SNAPSHOTS = Lists.newArrayList( + Lists.newArrayList( + Lists.newArrayList( + new SimpleRecord(1, "one"), + new SimpleRecord(2, "two"), + new SimpleRecord(3, "three")), + Lists.newArrayList( + new SimpleRecord(4, "four"), + new SimpleRecord(5, "five"))), + Lists.newArrayList( + Lists.newArrayList( + new SimpleRecord(6, "six"), + new SimpleRecord(7, "seven")), + Lists.newArrayList( + new SimpleRecord(8, "eight"), + new SimpleRecord(9, "nine"))), + Lists.newArrayList( + Lists.newArrayList( + new SimpleRecord(10, "ten"), + new SimpleRecord(11, "eleven"), + new SimpleRecord(12, "twelve")), + Lists.newArrayList( + new SimpleRecord(13, "thirteen"), + new SimpleRecord(14, "fourteen")), + Lists.newArrayList( + new SimpleRecord(15, "fifteen"), + new SimpleRecord(16, "sixteen")))); + + @Before + public void setupTable() { + sql("CREATE TABLE %s " + + "(id INT, data STRING) " + + "USING iceberg " + + "PARTITIONED BY (bucket(3, id))", tableName); + this.table = validationCatalog.loadTable(tableIdent); + this.tableIdentifier = tableName; + } + + @After + public void stopStreams() throws TimeoutException { + for (StreamingQuery query : spark.streams().active()) { + query.stop(); + } + } + + @After + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @SuppressWarnings("unchecked") + @Test + public void testReadStreamOnIcebergTableWithMultipleSnapshots() throws Exception { + List> expected = TEST_DATA_MULTIPLE_SNAPSHOTS; + appendDataAsMultipleSnapshots(expected, tableIdentifier); + + table.refresh(); + + Dataset df = spark.readStream() + .format("iceberg") + .load(tableIdentifier); + List actual = processAvailable(df); + + Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + } + + @SuppressWarnings("unchecked") + @Test + public void testResumingStreamReadFromCheckpoint() throws Exception { + File writerCheckpointFolder = temp.newFolder("writer-checkpoint-folder"); + File writerCheckpoint = new File(writerCheckpointFolder, "writer-checkpoint"); + final String tempView = "microBatchView"; + + Dataset df = spark.readStream() + .format("iceberg") + .load(tableIdentifier); + + // Trigger.Once with the combination of StreamingQuery.awaitTermination, which succeeds after this code + // will result in stopping the stream. + // This is how Stream STOP and RESUME is simulated in this Test Case. + DataStreamWriter singleBatchWriter = df.writeStream() + .trigger(Trigger.Once()) + .option("checkpointLocation", writerCheckpoint.toString()) + .foreachBatch((batchDF, batchId) -> { + batchDF.createOrReplaceGlobalTempView(tempView); + }); + + String globalTempView = "global_temp." + tempView; + + List processStreamOnEmptyIcebergTable = processMicroBatch(singleBatchWriter, globalTempView); + Assert.assertEquals(Collections.emptyList(), processStreamOnEmptyIcebergTable); + + for (List> expectedCheckpoint : TEST_DATA_MULTIPLE_WRITES_MULTIPLE_SNAPSHOTS) { + appendDataAsMultipleSnapshots(expectedCheckpoint, tableIdentifier); + table.refresh(); + + List actualDataInCurrentMicroBatch = processMicroBatch(singleBatchWriter, globalTempView); + Assertions.assertThat(actualDataInCurrentMicroBatch) + .containsExactlyInAnyOrderElementsOf(Iterables.concat(expectedCheckpoint)); + } + } + + @SuppressWarnings("unchecked") + @Test + public void testParquetOrcAvroDataInOneTable() throws Exception { + List parquetFileRecords = Lists.newArrayList( + new SimpleRecord(1, "one"), + new SimpleRecord(2, "two"), + new SimpleRecord(3, "three")); + + List orcFileRecords = Lists.newArrayList( + new SimpleRecord(4, "four"), + new SimpleRecord(5, "five")); + + List avroFileRecords = Lists.newArrayList( + new SimpleRecord(6, "six"), + new SimpleRecord(7, "seven")); + + appendData(parquetFileRecords, tableIdentifier, "parquet"); + appendData(orcFileRecords, tableIdentifier, "orc"); + appendData(avroFileRecords, tableIdentifier, "avro"); + + table.refresh(); + + Dataset ds = spark.readStream() + .format("iceberg") + .load(tableIdentifier); + Assertions.assertThat(processAvailable(ds)) + .containsExactlyInAnyOrderElementsOf(Iterables.concat(parquetFileRecords, orcFileRecords, avroFileRecords)); + } + + @SuppressWarnings("unchecked") + @Test + public void testReadStreamFromEmptyTable() throws Exception { + table.refresh(); + + Dataset df = spark.readStream() + .format("iceberg") + .load(tableIdentifier); + + List actual = processAvailable(df); + Assert.assertEquals(Collections.emptyList(), actual); + } + + @SuppressWarnings("unchecked") + @Test + public void testReadStreamWithSnapshotTypeOverwriteErrorsOut() throws Exception { + // upgrade table to version 2 - to facilitate creation of Snapshot of type OVERWRITE. + TableOperations ops = ((BaseTable) table).operations(); + TableMetadata meta = ops.current(); + ops.commit(meta, meta.upgradeToFormatVersion(2)); + + // fill table with some initial data + List> dataAcrossSnapshots = TEST_DATA_MULTIPLE_SNAPSHOTS; + appendDataAsMultipleSnapshots(dataAcrossSnapshots, tableIdentifier); + + Schema deleteRowSchema = table.schema().select("data"); + Record dataDelete = GenericRecord.create(deleteRowSchema); + List dataDeletes = Lists.newArrayList( + dataDelete.copy("data", "one") // id = 1 + ); + + DeleteFile eqDeletes = FileHelpers.writeDeleteFile( + table, Files.localOutput(temp.newFile()), TestHelpers.Row.of(0), dataDeletes, deleteRowSchema); + + table.newRowDelta() + .addDeletes(eqDeletes) + .commit(); + + // check pre-condition - that the above Delete file write - actually resulted in snapshot of type OVERWRITE + Assert.assertEquals(DataOperations.OVERWRITE, table.currentSnapshot().operation()); + + Dataset df = spark.readStream() + .format("iceberg") + .load(tableIdentifier); + StreamingQuery streamingQuery = df.writeStream() + .format("memory") + .queryName("testtablewithoverwrites") + .outputMode(OutputMode.Append()) + .start(); + + AssertHelpers.assertThrowsCause( + "Streaming should fail with IllegalStateException, as the snapshot is not of type APPEND", + IllegalStateException.class, + "Cannot process overwrite snapshot", + () -> streamingQuery.processAllAvailable() + ); + } + + @SuppressWarnings("unchecked") + @Test + public void testReadStreamWithSnapshotTypeReplaceIgnoresReplace() throws Exception { + // fill table with some data + List> expected = TEST_DATA_MULTIPLE_SNAPSHOTS; + appendDataAsMultipleSnapshots(expected, tableIdentifier); + + table.refresh(); + + // this should create a snapshot with type Replace. + table.rewriteManifests() + .clusterBy(f -> 1) + .commit(); + + // check pre-condition + Assert.assertEquals(DataOperations.REPLACE, table.currentSnapshot().operation()); + + Dataset df = spark.readStream() + .format("iceberg") + .load(tableIdentifier); + + List actual = processAvailable(df); + Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + } + + @SuppressWarnings("unchecked") + @Test + public void testReadStreamWithSnapshotTypeDeleteErrorsOut() throws Exception { + table.updateSpec() + .removeField("id_bucket") + .addField(ref("id")) + .commit(); + + table.refresh(); + + // fill table with some data + List> dataAcrossSnapshots = TEST_DATA_MULTIPLE_SNAPSHOTS; + appendDataAsMultipleSnapshots(dataAcrossSnapshots, tableIdentifier); + + table.refresh(); + + // this should create a snapshot with type delete. + table.newDelete() + .deleteFromRowFilter(Expressions.equal("id", 4)) + .commit(); + + // check pre-condition - that the above delete operation on table resulted in Snapshot of Type DELETE. + table.refresh(); + Assert.assertEquals(DataOperations.DELETE, table.currentSnapshot().operation()); + + Dataset df = spark.readStream() + .format("iceberg") + .load(tableIdentifier); + StreamingQuery streamingQuery = df.writeStream() + .format("memory") + .queryName("testtablewithdelete") + .outputMode(OutputMode.Append()) + .start(); + + AssertHelpers.assertThrowsCause( + "Streaming should fail with IllegalStateException, as the snapshot is not of type APPEND", + IllegalStateException.class, + "Cannot process delete snapshot", + () -> streamingQuery.processAllAvailable() + ); + } + + @SuppressWarnings("unchecked") + @Test + public void testReadStreamWithSnapshotTypeDeleteAndSkipDeleteOption() throws Exception { + table.updateSpec() + .removeField("id_bucket") + .addField(ref("id")) + .commit(); + + table.refresh(); + + // fill table with some data + List> dataAcrossSnapshots = TEST_DATA_MULTIPLE_SNAPSHOTS; + appendDataAsMultipleSnapshots(dataAcrossSnapshots, tableIdentifier); + + table.refresh(); + + // this should create a snapshot with type delete. + table.newDelete() + .deleteFromRowFilter(Expressions.equal("id", 4)) + .commit(); + + // check pre-condition - that the above delete operation on table resulted in Snapshot of Type DELETE. + table.refresh(); + Assert.assertEquals(DataOperations.DELETE, table.currentSnapshot().operation()); + + Dataset df = spark.readStream() + .format("iceberg") + .option(SparkReadOptions.STREAMING_SKIP_DELETE_SNAPSHOTS, "true") + .load(tableIdentifier); + Assertions.assertThat(processAvailable(df)) + .containsExactlyInAnyOrderElementsOf(Iterables.concat(dataAcrossSnapshots)); + } + + private static List processMicroBatch(DataStreamWriter singleBatchWriter, String viewName) + throws TimeoutException, StreamingQueryException { + StreamingQuery streamingQuery = singleBatchWriter.start(); + streamingQuery.awaitTermination(); + + return spark.sql(String.format("select * from %s", viewName)) + .as(Encoders.bean(SimpleRecord.class)) + .collectAsList(); + } + + /** + * appends each list as a Snapshot on the iceberg table at the given location. + * accepts a list of lists - each list representing data per snapshot. + */ + private static void appendDataAsMultipleSnapshots(List> data, String tableIdentifier) { + for (List l : data) { + appendData(l, tableIdentifier, "parquet"); + } + } + + private static void appendData(List data, String tableIdentifier, String fileFormat) { + Dataset df = spark.createDataFrame(data, SimpleRecord.class); + df.select("id", "data").write() + .format("iceberg") + .option("write-format", fileFormat) + .mode("append") + .save(tableIdentifier); + } + + private static List processAvailable(Dataset df) throws TimeoutException { + StreamingQuery streamingQuery = df.writeStream() + .format("memory") + .queryName("test12") + .outputMode(OutputMode.Append()) + .start(); + streamingQuery.processAllAvailable(); + return spark.sql("select * from test12") + .as(Encoders.bean(SimpleRecord.class)) + .collectAsList(); + } +} diff --git a/spark3/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone3.java b/spark3/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone3.java new file mode 100644 index 000000000000..4216aec02789 --- /dev/null +++ b/spark3/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone3.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.spark.source; + +public class TestTimestampWithoutZone3 extends TestTimestampWithoutZone { + public TestTimestampWithoutZone3(String format, boolean vectorized) { + super(format, vectorized); + } +} diff --git a/spark3/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java b/spark3/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java index 43b3bf9f9756..eb852fb2d32f 100644 --- a/spark3/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java +++ b/spark3/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java @@ -24,9 +24,11 @@ import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.spark.Spark3VersionUtil; import org.apache.iceberg.spark.SparkCatalogTestBase; import org.junit.After; import org.junit.Assert; +import org.junit.Assume; import org.junit.Test; public class TestDeleteFrom extends SparkCatalogTestBase { @@ -41,6 +43,10 @@ public void removeTables() { @Test public void testDeleteFromUnpartitionedTable() { + // This test fails in Spark 3.1. `canDeleteWhere` was added to `SupportsDelete` in Spark 3.1, + // but logic to rewrite the query if `canDeleteWhere` returns false was left to be implemented + // later. + Assume.assumeTrue(Spark3VersionUtil.isSpark30()); // set the shuffle partitions to 1 to force the write to use a single task and produce 1 file String originalParallelism = spark.conf().get("spark.sql.shuffle.partitions"); spark.conf().set("spark.sql.shuffle.partitions", "1"); @@ -68,6 +74,10 @@ public void testDeleteFromUnpartitionedTable() { @Test public void testDeleteFromPartitionedTable() { + // This test fails in Spark 3.1. `canDeleteWhere` was added to `SupportsDelete` in Spark 3.1, + // but logic to rewrite the query if `canDeleteWhere` returns false was left to be implemented + // later. + Assume.assumeTrue(Spark3VersionUtil.isSpark30()); // set the shuffle partitions to 1 to force the write to use a single task and produce 1 file per partition String originalParallelism = spark.conf().get("spark.sql.shuffle.partitions"); spark.conf().set("spark.sql.shuffle.partitions", "1"); diff --git a/spark3/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java b/spark3/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java new file mode 100644 index 000000000000..b6e92c82abd7 --- /dev/null +++ b/spark3/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java @@ -0,0 +1,195 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.spark.sql; + +import java.sql.Timestamp; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.iceberg.AssertHelpers; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.apache.iceberg.spark.SparkSessionCatalog; +import org.apache.iceberg.spark.SparkUtil; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; +import org.joda.time.DateTime; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runners.Parameterized; + +public class TestTimestampWithoutZone extends SparkCatalogTestBase { + + private static final String newTableName = "created_table"; + private final Map config; + + private static final Schema schema = new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required(2, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.required(3, "tsz", Types.TimestampType.withZone()) + ); + + private final List values = ImmutableList.of( + row(1L, toTimestamp("2021-01-01T00:00:00.0"), toTimestamp("2021-02-01T00:00:00.0")), + row(2L, toTimestamp("2021-01-01T00:00:00.0"), toTimestamp("2021-02-01T00:00:00.0")), + row(3L, toTimestamp("2021-01-01T00:00:00.0"), toTimestamp("2021-02-01T00:00:00.0")) + ); + + @Parameterized.Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + public static Object[][] parameters() { + return new Object[][]{{"spark_catalog", + SparkSessionCatalog.class.getName(), + ImmutableMap.of( + "type", "hive", + "default-namespace", "default", + "parquet-enabled", "true", + "cache-enabled", "false" + )} + }; + } + + public TestTimestampWithoutZone(String catalogName, String implementation, Map config) { + super(catalogName, implementation, config); + this.config = config; + } + + @Before + public void createTables() { + validationCatalog.createTable(tableIdent, schema); + } + + @After + public void removeTables() { + validationCatalog.dropTable(tableIdent, true); + sql("DROP TABLE IF EXISTS %s", newTableName); + } + + @Test + public void testWriteTimestampWithoutZoneError() { + AssertHelpers.assertThrows( + String.format("Write operation performed on a timestamp without timezone field while " + + "'%s' set to false should throw exception", SparkUtil.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE), + IllegalArgumentException.class, + SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR, + () -> sql("INSERT INTO %s VALUES %s", tableName, rowToSqlValues(values))); + } + + @Test + public void testAppendTimestampWithoutZone() { + withSQLConf(ImmutableMap.of(SparkUtil.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE, "true"), () -> { + sql("INSERT INTO %s VALUES %s", tableName, rowToSqlValues(values)); + + Assert.assertEquals("Should have " + values.size() + " row", + (long) values.size(), scalarSql("SELECT count(*) FROM %s", tableName)); + + assertEquals("Row data should match expected", + values, sql("SELECT * FROM %s ORDER BY id", tableName)); + }); + } + + @Test + public void testCreateAsSelectWithTimestampWithoutZone() { + withSQLConf(ImmutableMap.of(SparkUtil.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE, "true"), () -> { + sql("INSERT INTO %s VALUES %s", tableName, rowToSqlValues(values)); + + sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", newTableName, tableName); + + Assert.assertEquals("Should have " + values.size() + " row", (long) values.size(), + scalarSql("SELECT count(*) FROM %s", newTableName)); + + assertEquals("Row data should match expected", + sql("SELECT * FROM %s ORDER BY id", tableName), + sql("SELECT * FROM %s ORDER BY id", newTableName)); + }); + } + + @Test + public void testCreateNewTableShouldHaveTimestampWithZoneIcebergType() { + withSQLConf(ImmutableMap.of(SparkUtil.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE, "true"), () -> { + sql("INSERT INTO %s VALUES %s", tableName, rowToSqlValues(values)); + + sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", newTableName, tableName); + + Assert.assertEquals("Should have " + values.size() + " row", (long) values.size(), + scalarSql("SELECT count(*) FROM %s", newTableName)); + + assertEquals("Data from created table should match data from base table", + sql("SELECT * FROM %s ORDER BY id", tableName), + sql("SELECT * FROM %s ORDER BY id", newTableName)); + + Table createdTable = validationCatalog.loadTable(TableIdentifier.of("default", newTableName)); + assertFieldsType(createdTable.schema(), Types.TimestampType.withZone(), "ts", "tsz"); + }); + } + + @Test + public void testCreateNewTableShouldHaveTimestampWithoutZoneIcebergType() { + withSQLConf(ImmutableMap.of( + SparkUtil.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE, "true", + SparkUtil.USE_TIMESTAMP_WITHOUT_TIME_ZONE_IN_NEW_TABLES, "true"), () -> { + spark.sessionState().catalogManager().currentCatalog() + .initialize(catalog.name(), new CaseInsensitiveStringMap(config)); + sql("INSERT INTO %s VALUES %s", tableName, rowToSqlValues(values)); + + sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", newTableName, tableName); + + Assert.assertEquals("Should have " + values.size() + " row", (long) values.size(), + scalarSql("SELECT count(*) FROM %s", newTableName)); + + assertEquals("Row data should match expected", + sql("SELECT * FROM %s ORDER BY id", tableName), + sql("SELECT * FROM %s ORDER BY id", newTableName)); + Table createdTable = validationCatalog.loadTable(TableIdentifier.of("default", newTableName)); + assertFieldsType(createdTable.schema(), Types.TimestampType.withoutZone(), "ts", "tsz"); + }); + } + + private Timestamp toTimestamp(String value) { + return new Timestamp(DateTime.parse(value).getMillis()); + } + + private String rowToSqlValues(List rows) { + List rowValues = rows.stream().map(row -> { + List columns = Arrays.stream(row).map(value -> { + if (value instanceof Long) { + return value.toString(); + } else if (value instanceof Timestamp) { + return String.format("timestamp '%s'", value); + } + throw new RuntimeException("Type is not supported"); + }).collect(Collectors.toList()); + return "(" + Joiner.on(",").join(columns) + ")"; + }).collect(Collectors.toList()); + return Joiner.on(",").join(rowValues); + } + + private void assertFieldsType(Schema actual, Type.PrimitiveType expected, String... fields) { + actual.select(fields).asStruct().fields().forEach(field -> Assert.assertEquals(expected, field.type())); + } +} diff --git a/versions.props b/versions.props index 1948f5cbd0e5..e889b0f5d5c8 100644 --- a/versions.props +++ b/versions.props @@ -1,15 +1,14 @@ org.slf4j:* = 1.7.25 -org.apache.avro:avro = 1.9.2 +org.apache.avro:avro = 1.10.1 org.apache.calcite:* = 1.10.0 org.apache.flink:* = 1.12.1 org.apache.hadoop:* = 2.7.3 org.apache.hive:hive-metastore = 2.3.8 org.apache.hive:hive-serde = 2.3.8 org.apache.orc:* = 1.6.8 -org.apache.parquet:* = 1.11.1 -org.apache.spark:spark-hive_2.11 = 2.4.7 -org.apache.spark:spark-avro_2.11 = 2.4.7 -org.apache.spark:spark-hive_2.12 = 3.0.1 +org.apache.parquet:* = 1.12.0 +org.apache.spark:spark-hive_2.11 = 2.4.8 +org.apache.spark:spark-avro_2.11 = 2.4.8 org.apache.pig:pig = 0.14.0 com.fasterxml.jackson.*:* = 2.11.4 com.google.guava:guava = 28.0-jre @@ -24,10 +23,13 @@ javax.ws.rs:javax.ws.rs-api = 2.1.1 io.quarkus:* = 1.13.1.Final # test deps -junit:junit = 4.12 +org.junit.vintage:junit-vintage-engine = 5.7.2 +org.junit.jupiter:junit-jupiter = 5.7.2 org.mockito:mockito-core = 3.7.7 org.apache.hive:hive-exec = 2.3.8 org.apache.hive:hive-service = 2.3.8 org.apache.tez:tez-dag = 0.8.4 org.apache.tez:tez-mapreduce = 0.8.4 com.adobe.testing:s3mock-junit4 = 2.1.28 +org.assertj:assertj-core = 3.19.0 +org.xerial:sqlite-jdbc = 3.34.0