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 extends Exception> 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, ? extends Throwable> 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 extends Exception> 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, ? extends Throwable> 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 extends Exception> 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 extends Exception> 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