diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowFormatModels.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowFormatModels.java new file mode 100644 index 000000000000..4996c9533c11 --- /dev/null +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowFormatModels.java @@ -0,0 +1,39 @@ +/* + * 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.arrow.vectorized; + +import org.apache.arrow.vector.NullCheckingForGet; +import org.apache.iceberg.formats.FormatModelRegistry; +import org.apache.iceberg.parquet.ParquetFormatModel; + +public class ArrowFormatModels { + public static void register() { + FormatModelRegistry.register( + new ParquetFormatModel<>( + ColumnarBatch.class, + Object.class, + (schema, messageType, idToConstant) -> + ArrowReader.VectorizedCombinedScanIterator.buildReader( + schema, + messageType, /* setArrowValidityVector */ + NullCheckingForGet.NULL_CHECKING_ENABLED))); + } + + private ArrowFormatModels() {} +} 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 06b7baec27d5..68a27bdfb8eb 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 @@ -29,7 +29,6 @@ import java.util.stream.Collectors; import java.util.stream.Stream; import java.util.stream.StreamSupport; -import org.apache.arrow.vector.NullCheckingForGet; import org.apache.arrow.vector.VectorSchemaRoot; import org.apache.arrow.vector.types.Types.MinorType; import org.apache.iceberg.CombinedScanTask; @@ -40,13 +39,14 @@ import org.apache.iceberg.encryption.EncryptedFiles; import org.apache.iceberg.encryption.EncryptedInputFile; import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.formats.FormatModelRegistry; +import org.apache.iceberg.formats.ReadBuilder; import org.apache.iceberg.io.CloseableGroup; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.CloseableIterator; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.mapping.NameMappingParser; -import org.apache.iceberg.parquet.Parquet; 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; @@ -189,8 +189,7 @@ public void close() throws IOException { * Reads the data file and returns an iterator of {@link VectorSchemaRoot}. Only Parquet data file * format is supported. */ - private static final class VectorizedCombinedScanIterator - implements CloseableIterator { + static final class VectorizedCombinedScanIterator implements CloseableIterator { private final Iterator fileItr; private final Map inputFiles; @@ -324,19 +323,8 @@ CloseableIterator open(FileScanTask task) { InputFile location = getInputFile(task); Preconditions.checkNotNull(location, "Could not find InputFile associated with FileScanTask"); if (task.file().format() == FileFormat.PARQUET) { - Parquet.ReadBuilder builder = - Parquet.read(location) - .project(expectedSchema) - .split(task.start(), task.length()) - .createBatchedReaderFunc( - fileSchema -> - buildReader( - expectedSchema, - fileSchema, /* setArrowValidityVector */ - NullCheckingForGet.NULL_CHECKING_ENABLED)) - .recordsPerBatch(batchSize) - .filter(task.residual()) - .caseSensitive(caseSensitive); + ReadBuilder builder = + FormatModelRegistry.readBuilder(FileFormat.PARQUET, ColumnarBatch.class, location); if (reuseContainers) { builder.reuseContainers(); @@ -345,7 +333,14 @@ CloseableIterator open(FileScanTask task) { builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); } - iter = builder.build(); + iter = + builder + .project(expectedSchema) + .split(task.start(), task.length()) + .recordsPerBatch(batchSize) + .caseSensitive(caseSensitive) + .filter(task.residual()) + .build(); } else { throw new UnsupportedOperationException( "Format: " + task.file().format() + " not supported for batched reads"); @@ -376,7 +371,7 @@ private InputFile getInputFile(FileScanTask task) { * @param fileSchema Schema of the data file. * @param setArrowValidityVector Indicates whether to set the validity vector in Arrow vectors. */ - private static ArrowBatchReader buildReader( + static ArrowBatchReader buildReader( Schema expectedSchema, MessageType fileSchema, boolean setArrowValidityVector) { return (ArrowBatchReader) TypeWithSchemaVisitor.visit( diff --git a/core/src/main/java/org/apache/iceberg/avro/Avro.java b/core/src/main/java/org/apache/iceberg/avro/Avro.java index 6c7edc25b691..4a5136f58e71 100644 --- a/core/src/main/java/org/apache/iceberg/avro/Avro.java +++ b/core/src/main/java/org/apache/iceberg/avro/Avro.java @@ -182,8 +182,7 @@ public WriteBuilder overwrite(boolean enabled) { } // supposed to always be a private method used strictly by data and delete write builders - private WriteBuilder createContextFunc( - Function, Context> newCreateContextFunc) { + WriteBuilder createContextFunc(Function, Context> newCreateContextFunc) { this.createContextFunc = newCreateContextFunc; return this; } @@ -217,7 +216,7 @@ public FileAppender build() throws IOException { overwrite); } - private static class Context { + static class Context { private final CodecFactory codec; private Context(CodecFactory codec) { @@ -568,7 +567,7 @@ public PositionDeleteWriter buildPositionWriter() throws IOException { } /** A {@link DatumWriter} implementation that wraps another to produce position deletes. */ - private static class PositionDatumWriter implements MetricsAwareDatumWriter> { + static class PositionDatumWriter implements MetricsAwareDatumWriter> { private static final ValueWriter PATH_WRITER = ValueWriters.strings(); private static final ValueWriter POS_WRITER = ValueWriters.longs(); diff --git a/core/src/main/java/org/apache/iceberg/avro/AvroFormatModel.java b/core/src/main/java/org/apache/iceberg/avro/AvroFormatModel.java new file mode 100644 index 000000000000..3b6355a5104b --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/avro/AvroFormatModel.java @@ -0,0 +1,260 @@ +/* + * 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.avro; + +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.function.BiFunction; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.DatumWriter; +import org.apache.iceberg.FileContent; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetricsConfig; +import org.apache.iceberg.Schema; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.formats.FormatModel; +import org.apache.iceberg.formats.ReadBuilder; +import org.apache.iceberg.formats.WriteBuilder; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.DeleteSchemaUtil; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; + +public class AvroFormatModel implements FormatModel { + private final Class type; + private final Class schemaType; + private final BiFunction, DatumReader> readerFunction; + private final BiFunction> writerFunction; + + public AvroFormatModel(Class type) { + this(type, null, null, null); + } + + public AvroFormatModel( + Class type, + Class schemaType, + BiFunction, DatumReader> readerFunction, + BiFunction> writerFunction) { + this.type = type; + this.schemaType = schemaType; + this.readerFunction = readerFunction; + this.writerFunction = writerFunction; + } + + @Override + public FileFormat format() { + return FileFormat.AVRO; + } + + @Override + public Class type() { + return type; + } + + @Override + public Class schemaType() { + return schemaType; + } + + @Override + public WriteBuilder writeBuilder(EncryptedOutputFile outputFile) { + return new WriteBuilderWrapper<>(outputFile, writerFunction); + } + + @Override + public ReadBuilder readBuilder(InputFile inputFile) { + return new ReadBuilderWrapper<>(inputFile, readerFunction); + } + + private static class ReadBuilderWrapper implements ReadBuilder { + private final Avro.ReadBuilder internal; + private final BiFunction, DatumReader> readerFunction; + private Schema icebergSchema; + private Map idToConstant = ImmutableMap.of(); + + private ReadBuilderWrapper( + InputFile inputFile, BiFunction, DatumReader> readerFunction) { + this.internal = Avro.read(inputFile); + this.readerFunction = readerFunction; + } + + @Override + public ReadBuilder split(long newStart, long newLength) { + internal.split(newStart, newLength); + return this; + } + + @Override + public ReadBuilder project(Schema schema) { + this.icebergSchema = schema; + internal.project(schema); + return this; + } + + @Override + public ReadBuilder caseSensitive(boolean caseSensitive) { + // Filtering is not supported in Avro reader, so case sensitivity does not matter + return this; + } + + @Override + public ReadBuilder filter(Expression filter) { + // Filtering is not supported in Avro reader + return this; + } + + @Override + public ReadBuilder set(String key, String value) { + // Configuration is not used for Avro reader creation + return this; + } + + @Override + public ReadBuilder reuseContainers() { + internal.reuseContainers(); + return this; + } + + @Override + public ReadBuilder recordsPerBatch(int numRowsPerBatch) { + throw new UnsupportedOperationException("Batch reading is not supported in Avro reader"); + } + + @Override + public ReadBuilder idToConstant(Map newIdToConstant) { + this.idToConstant = newIdToConstant; + return this; + } + + @Override + public ReadBuilder withNameMapping(org.apache.iceberg.mapping.NameMapping nameMapping) { + internal.withNameMapping(nameMapping); + return this; + } + + @Override + public CloseableIterable build() { + return internal + .createResolvingReader(unused -> readerFunction.apply(icebergSchema, idToConstant)) + .build(); + } + } + + private static class WriteBuilderWrapper implements WriteBuilder { + private final Avro.WriteBuilder internal; + private final BiFunction> writerFunction; + private S inputSchema; + private FileContent content; + + private WriteBuilderWrapper( + EncryptedOutputFile outputFile, + BiFunction> writerFunction) { + this.internal = Avro.write(outputFile.encryptingOutputFile()); + this.writerFunction = writerFunction; + } + + @Override + public WriteBuilder schema(Schema schema) { + internal.schema(schema); + return this; + } + + @Override + public WriteBuilder inputSchema(S schema) { + this.inputSchema = schema; + return this; + } + + @Override + public WriteBuilder set(String property, String value) { + internal.set(property, value); + return this; + } + + @Override + public WriteBuilder setAll(Map properties) { + internal.setAll(properties); + return this; + } + + @Override + public WriteBuilder meta(String property, String value) { + internal.meta(property, value); + return this; + } + + @Override + public WriteBuilder meta(Map properties) { + internal.meta(properties); + return this; + } + + @Override + public WriteBuilder content(FileContent newContent) { + this.content = newContent; + return this; + } + + @Override + public WriteBuilder metricsConfig(MetricsConfig metricsConfig) { + internal.metricsConfig(metricsConfig); + return this; + } + + @Override + public WriteBuilder overwrite() { + internal.overwrite(); + return this; + } + + @Override + public WriteBuilder withFileEncryptionKey(ByteBuffer encryptionKey) { + throw new UnsupportedOperationException("Avro does not support file encryption keys"); + } + + @Override + public WriteBuilder withAADPrefix(ByteBuffer aadPrefix) { + throw new UnsupportedOperationException("Avro does not support AAD prefix"); + } + + @Override + public org.apache.iceberg.io.FileAppender build() throws java.io.IOException { + switch (content) { + case DATA: + internal.createContextFunc(Avro.WriteBuilder.Context::dataContext); + internal.createWriterFunc(avroSchema -> writerFunction.apply(avroSchema, inputSchema)); + break; + case EQUALITY_DELETES: + internal.createContextFunc(Avro.WriteBuilder.Context::deleteContext); + internal.createWriterFunc(avroSchema -> writerFunction.apply(avroSchema, inputSchema)); + break; + case POSITION_DELETES: + internal.createContextFunc(Avro.WriteBuilder.Context::deleteContext); + internal.createWriterFunc(unused -> new Avro.PositionDatumWriter()); + internal.schema(DeleteSchemaUtil.pathPosSchema()); + break; + default: + throw new IllegalArgumentException("Unknown file content: " + content); + } + + return internal.build(); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/formats/CommonWriteBuilder.java b/core/src/main/java/org/apache/iceberg/formats/CommonWriteBuilder.java new file mode 100644 index 000000000000..b37e755926f1 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/formats/CommonWriteBuilder.java @@ -0,0 +1,122 @@ +/* + * 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.formats; + +import java.nio.ByteBuffer; +import java.util.Map; +import org.apache.iceberg.MetricsConfig; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.deletes.EqualityDeleteWriter; +import org.apache.iceberg.deletes.PositionDeleteWriter; +import org.apache.iceberg.encryption.EncryptionKeyMetadata; +import org.apache.iceberg.io.DataWriter; + +/** + * A generic builder interface for creating specialized file writers in the Iceberg ecosystem. + * + *

This builder provides a unified configuration API for generating various types of content + * writers: + * + *

    + *
  • {@link DataWriter} for creating data files with table records + *
  • {@link EqualityDeleteWriter} for creating files with equality-based delete records + *
  • {@link PositionDeleteWriter} for creating files with position-based delete records + *
+ * + *

Each concrete implementation configures the underlying file format writer while adding + * content-specific metadata and behaviors. + * + * @param the concrete builder type for method chaining + */ +interface CommonWriteBuilder> { + + /** + * Set a writer configuration property which affects the writer behavior. + * + * @param property a writer config property name + * @param value config value + * @return this for method chaining + */ + B set(String property, String value); + + /** + * Adds the new properties to the writer configuration. + * + * @param properties a map of writer config properties + * @return this for method chaining + */ + default B setAll(Map properties) { + properties.forEach(this::set); + return self(); + } + + /** + * Set a file metadata property in the created file. + * + * @param property a file metadata property name + * @param value config value + * @return this for method chaining + */ + B meta(String property, String value); + + /** + * Add the new properties to file metadata for the created file. + * + * @param properties a map of file metadata properties + * @return this for method chaining + */ + default B meta(Map properties) { + properties.forEach(this::meta); + return self(); + } + + /** Sets the metrics configuration used for collecting column metrics for the created file. */ + B metricsConfig(MetricsConfig metricsConfig); + + /** Overwrite the file if it already exists. By default, overwrite is disabled. */ + B overwrite(); + + /** + * Sets the encryption key used for writing the file. If the writer does not support encryption, + * then an exception should be thrown. + */ + B withFileEncryptionKey(ByteBuffer encryptionKey); + + /** + * Sets the additional authentication data (AAD) prefix used for writing the file. If the writer + * does not support encryption, then an exception should be thrown. + */ + B withAADPrefix(ByteBuffer aadPrefix); + + /** Sets the partition specification for the Iceberg metadata. */ + B spec(PartitionSpec newSpec); + + /** Sets the partition value for the Iceberg metadata. */ + B partition(StructLike partition); + + /** Sets the encryption key metadata for Iceberg metadata. */ + B keyMetadata(EncryptionKeyMetadata keyMetadata); + + /** Sets the sort order for the Iceberg metadata. */ + B sortOrder(SortOrder sortOrder); + + B self(); +} diff --git a/core/src/main/java/org/apache/iceberg/formats/CommonWriteBuilderImpl.java b/core/src/main/java/org/apache/iceberg/formats/CommonWriteBuilderImpl.java new file mode 100644 index 000000000000..9e5d9b1605cb --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/formats/CommonWriteBuilderImpl.java @@ -0,0 +1,337 @@ +/* + * 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.formats; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.iceberg.FileContent; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.MetricsConfig; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.deletes.EqualityDeleteWriter; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.deletes.PositionDeleteWriter; +import org.apache.iceberg.encryption.EncryptionKeyMetadata; +import org.apache.iceberg.io.DataWriter; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * An internal implementation that handles all {@link CommonWriteBuilder} interface variants. + * + *

This unified implementation serves as a backend for multiple specialized content writers: + * + *

    + *
  • {@link DataWriteBuilder} for creating data files + *
  • {@link EqualityDeleteWriteBuilder} for creating equality delete files + *
  • {@link PositionDeleteWriteBuilder} for creating position delete files + *
+ * + *

The implementation delegates to a format-specific {@link WriteBuilder} while enriching it with + * content-specific functionality. When building a writer, the implementation configures the + * underlying builder and calls its {@link WriteBuilder#build()} method to create the appropriate + * specialized writer for the requested content type. + * + * @param the concrete builder type for method chaining + * @param the type of data records the writer will accept + * @param the type of the schema for the input data + */ +abstract class CommonWriteBuilderImpl, D, S> + implements CommonWriteBuilder { + private final WriteBuilder writeBuilder; + private final String location; + private final FileFormat format; + private PartitionSpec spec = null; + private StructLike partition = null; + private EncryptionKeyMetadata keyMetadata = null; + private SortOrder sortOrder = null; + + static DataWriteBuilder forDataFile( + WriteBuilder writeBuilder, String location, FileFormat format) { + return new DataFileWriteBuilder<>(writeBuilder.content(FileContent.DATA), location, format); + } + + static EqualityDeleteWriteBuilder forEqualityDelete( + WriteBuilder writeBuilder, String location, FileFormat format) { + return new EqualityDeleteFileWriteBuilder<>( + writeBuilder.content(FileContent.EQUALITY_DELETES), location, format); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + static PositionDeleteWriteBuilder forPositionDelete( + WriteBuilder writeBuilder, String location, FileFormat format) { + return new PositionDeleteFileWriteBuilder( + (WriteBuilder) writeBuilder.content(FileContent.POSITION_DELETES), + location, + format); + } + + private CommonWriteBuilderImpl( + WriteBuilder writeBuilder, String location, FileFormat format) { + this.writeBuilder = writeBuilder; + this.location = location; + this.format = format; + } + + @Override + public B set(String property, String value) { + writeBuilder.set(property, value); + return self(); + } + + @Override + public B meta(String property, String value) { + writeBuilder.meta(property, value); + return self(); + } + + @Override + public B metricsConfig(MetricsConfig metricsConfig) { + writeBuilder.metricsConfig(metricsConfig); + return self(); + } + + @Override + public B overwrite() { + writeBuilder.overwrite(); + return self(); + } + + @Override + public B withFileEncryptionKey(ByteBuffer encryptionKey) { + writeBuilder.withFileEncryptionKey(encryptionKey); + return self(); + } + + @Override + public B withAADPrefix(ByteBuffer aadPrefix) { + writeBuilder.withAADPrefix(aadPrefix); + return self(); + } + + @Override + public B spec(PartitionSpec newSpec) { + this.spec = newSpec; + return self(); + } + + @Override + public B partition(StructLike newPartition) { + this.partition = newPartition; + return self(); + } + + @Override + public B keyMetadata(EncryptionKeyMetadata newKeyMetadata) { + this.keyMetadata = newKeyMetadata; + return self(); + } + + @Override + public B sortOrder(SortOrder newSortOrder) { + this.sortOrder = newSortOrder; + return self(); + } + + private static class DataFileWriteBuilder + extends CommonWriteBuilderImpl, D, S> + implements DataWriteBuilder { + private DataFileWriteBuilder( + WriteBuilder writeBuilder, String location, FileFormat format) { + super(writeBuilder, location, format); + } + + @Override + public DataFileWriteBuilder schema(Schema schema) { + super.writeBuilder.schema(schema); + return this; + } + + @Override + public DataFileWriteBuilder inputSchema(S schema) { + super.writeBuilder.inputSchema(schema); + return this; + } + + @Override + public DataFileWriteBuilder self() { + return this; + } + + @Override + public DataWriter build() throws IOException { + Preconditions.checkArgument(super.spec != null, "Cannot create data writer without spec"); + Preconditions.checkArgument( + super.spec.isUnpartitioned() || super.partition != null, + "Partition must not be null when creating data writer for partitioned spec"); + + return new DataWriter<>( + super.writeBuilder.build(), + super.format, + super.location, + super.spec, + super.partition, + super.keyMetadata, + super.sortOrder); + } + } + + private static class EqualityDeleteFileWriteBuilder + extends CommonWriteBuilderImpl, D, S> + implements EqualityDeleteWriteBuilder { + private Schema rowSchema = null; + private int[] equalityFieldIds = null; + + private EqualityDeleteFileWriteBuilder( + WriteBuilder writeBuilder, String location, FileFormat format) { + super(writeBuilder, location, format); + } + + @Override + public EqualityDeleteFileWriteBuilder inputSchema(S schema) { + super.writeBuilder.inputSchema(schema); + return this; + } + + @Override + public EqualityDeleteFileWriteBuilder self() { + return this; + } + + @Override + public EqualityDeleteFileWriteBuilder rowSchema(Schema schema) { + this.rowSchema = schema; + return this; + } + + @Override + public EqualityDeleteFileWriteBuilder equalityFieldIds(int... fieldIds) { + this.equalityFieldIds = fieldIds; + return this; + } + + @Override + public EqualityDeleteWriter build() throws IOException { + Preconditions.checkState( + rowSchema != null, "Cannot create equality delete file without a schema"); + Preconditions.checkState( + equalityFieldIds != null, "Cannot create equality delete file without delete field ids"); + Preconditions.checkArgument( + super.spec != null, "Spec must not be null when creating equality delete writer"); + Preconditions.checkArgument( + super.spec.isUnpartitioned() || super.partition != null, + "Partition must not be null for partitioned writes"); + + return new EqualityDeleteWriter<>( + super.writeBuilder + .schema(rowSchema) + .meta("delete-type", "equality") + .meta( + "delete-field-ids", + IntStream.of(equalityFieldIds) + .mapToObj(Objects::toString) + .collect(Collectors.joining(", "))) + .build(), + super.format, + super.location, + super.spec, + super.partition, + super.keyMetadata, + super.sortOrder, + equalityFieldIds); + } + } + + @SuppressWarnings({"rawtypes", "unchecked"}) + private static class PositionDeleteFileWriteBuilder + extends CommonWriteBuilderImpl + implements PositionDeleteWriteBuilder { + + private PositionDeleteFileWriteBuilder( + WriteBuilder writeBuilder, String location, FileFormat format) { + super(writeBuilder, location, format); + } + + @Override + public PositionDeleteFileWriteBuilder self() { + return this; + } + + @Override + @SuppressWarnings("unchecked") + public PositionDeleteWriter build() throws IOException { + Preconditions.checkArgument( + super.spec != null, "Spec must not be null when creating position delete writer"); + Preconditions.checkArgument( + super.spec.isUnpartitioned() || super.partition != null, + "Partition must not be null for partitioned writes"); + + return new PositionDeleteWriter<>( + new PositionDeleteFileAppender( + super.writeBuilder.meta("delete-type", "position").build()), + super.format, + super.location, + super.spec, + super.partition, + super.keyMetadata); + } + } + + @SuppressWarnings("rawtypes") + private static class PositionDeleteFileAppender implements FileAppender { + private final FileAppender appender; + + PositionDeleteFileAppender(FileAppender appender) { + this.appender = appender; + } + + @Override + public void add(StructLike positionDelete) { + appender.add((PositionDelete) positionDelete); + } + + @Override + public Metrics metrics() { + return appender.metrics(); + } + + @Override + public long length() { + return appender.length(); + } + + @Override + public void close() throws IOException { + appender.close(); + } + + @Override + public List splitOffsets() { + return appender.splitOffsets(); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/formats/DataWriteBuilder.java b/core/src/main/java/org/apache/iceberg/formats/DataWriteBuilder.java new file mode 100644 index 000000000000..d81734794874 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/formats/DataWriteBuilder.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.formats; + +import java.io.IOException; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.DataWriter; + +/** + * A specialized builder for creating data content file writers. + * + *

This builder extends the generic {@link CommonWriteBuilder} interface with functionality + * specific to creating {@link DataWriter} instances. Data writers produce table content files + * containing actual data records stored in an Iceberg table, configured according to the table's + * schema and partition specification. + * + * @param the type of data records the writer will accept + * @param the type of the schema for the input data + */ +public interface DataWriteBuilder extends CommonWriteBuilder> { + + /** Set the file schema. */ + DataWriteBuilder schema(Schema schema); + + /** + * Sets the input schema accepted by the writer. If not provided derived from the {@link + * #schema(Schema)}. + */ + DataWriteBuilder inputSchema(S schema); + + /** + * Creates a data file writer configured with the current builder settings. + * + *

The returned {@link DataWriter} produces files that conform to the Iceberg table format, + * generating proper {@link DataFile} metadata on completion. The writer accepts input records + * exactly matching the Iceberg schema specified via {@link #schema(Schema)} for writing. + * + * @return a fully configured {@link DataWriter} instance + * @throws IOException if the writer cannot be created due to I/O errors + */ + DataWriter build() throws IOException; +} diff --git a/core/src/main/java/org/apache/iceberg/formats/EqualityDeleteWriteBuilder.java b/core/src/main/java/org/apache/iceberg/formats/EqualityDeleteWriteBuilder.java new file mode 100644 index 000000000000..25b38da22159 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/formats/EqualityDeleteWriteBuilder.java @@ -0,0 +1,74 @@ +/* + * 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.formats; + +import java.io.IOException; +import java.util.List; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.Schema; +import org.apache.iceberg.deletes.EqualityDeleteWriter; +import org.apache.iceberg.util.ArrayUtil; + +/** + * A specialized builder for creating equality-based delete file writers. + * + *

This builder extends the generic {@link CommonWriteBuilder} interface with functionality + * specific to creating {@link EqualityDeleteWriter} instances. + * + *

The builder provides methods to configure which fields should be used for equality comparison + * through {@link #equalityFieldIds(List)} or {@link #equalityFieldIds(int...)}, along with schema + * configuration for the delete records. + * + * @param the type of data records the writer will accept + * @param the type of the schema for the input data + */ +public interface EqualityDeleteWriteBuilder + extends CommonWriteBuilder> { + + /** + * Sets the input schema accepted by the writer. If not provided derived from the {@link + * #rowSchema(Schema)}. + */ + EqualityDeleteWriteBuilder inputSchema(S schema); + + /** Sets the row schema for the delete writers. */ + EqualityDeleteWriteBuilder rowSchema(Schema rowSchema); + + /** Sets the equality field ids for the equality delete writer. */ + default EqualityDeleteWriteBuilder equalityFieldIds(List fieldIds) { + return equalityFieldIds(ArrayUtil.toIntArray(fieldIds)); + } + + /** Sets the equality field ids for the equality delete writer. */ + EqualityDeleteWriteBuilder equalityFieldIds(int... fieldIds); + + /** + * Creates an equality-based delete file writer configured with the current builder settings. + * + *

The returned {@link EqualityDeleteWriter} produces files that identify records to be deleted + * based on field equality, generating proper {@link DeleteFile} metadata on completion. + * + *

The writer accepts input records exactly matching the input schema specified via {@link + * #rowSchema(Schema)} for deletion. + * + * @return a fully configured {@link EqualityDeleteWriter} instance + * @throws IOException if the writer cannot be created due to I/O errors + */ + EqualityDeleteWriter build() throws IOException; +} diff --git a/core/src/main/java/org/apache/iceberg/formats/FormatModel.java b/core/src/main/java/org/apache/iceberg/formats/FormatModel.java new file mode 100644 index 000000000000..c8164aba1d8f --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/formats/FormatModel.java @@ -0,0 +1,91 @@ +/* + * 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.formats; + +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.io.InputFile; + +/** + * Interface that provides a unified abstraction for converting between data file formats and + * input/output data representations. + * + *

{@link FormatModel} serves as a bridge between storage formats ({@link FileFormat}) and + * expected input/output data structures, optimizing performance through direct conversion without + * intermediate representations. File format implementations handle the low-level parsing details + * while the object model determines the in-memory representation used for the parsed data. + * Together, these provide a consistent API for consuming data files while optimizing for specific + * processing engines. + * + *

Iceberg provides some built-in object models and processing engines can implement custom + * object models to integrate with Iceberg's file reading and writing capabilities. + * + * @param output type used for reading data, and input type for writing data and deletes + * @param the type of the schema for the input/output data + */ +public interface FormatModel { + /** The file format which is read/written by the object model. */ + FileFormat format(); + + /** + * Return the row type class for the object model implementation processed by this factory. + * + *

The model types act as a contract specifying the expected data structures for both reading + * (converting file formats into output objects) and writing (converting input objects into file + * formats). This ensures proper integration between Iceberg's storage layer and processing + * engines. + * + *

Processing engines can define their own object models by implementing this interface and + * using their own model name. They can register these models with Iceberg by using the {@link + * FormatModelRegistry}. This allows custom data representations to be seamlessly integrated with + * Iceberg's file format handlers. + * + * @return the type of the data structures handled by this model implementation + */ + Class type(); + + /** + * Return the schema type class for the object model implementation processed by this factory. + * + * @return the type of the schema for the data structures handled by this model implementation + */ + Class schemaType(); + + /** + * Creates a writer builder for data files. + * + *

The returned {@link WriteBuilder} configures and creates a writer that converts input + * objects into the file format supported by this factory. + * + * @param outputFile destination for the written data + * @return configured writer builder + */ + WriteBuilder writeBuilder(EncryptedOutputFile outputFile); + + /** + * Creates a file reader builder for the specified input file. + * + *

The returned {@link ReadBuilder} configures and creates a reader that converts data from the + * file format into output objects supported by this factory. + * + * @param inputFile source file to read from + * @return configured reader builder for the specified input + */ + ReadBuilder readBuilder(InputFile inputFile); +} diff --git a/core/src/main/java/org/apache/iceberg/formats/FormatModelRegistry.java b/core/src/main/java/org/apache/iceberg/formats/FormatModelRegistry.java new file mode 100644 index 000000000000..d1b2ddaac46b --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/formats/FormatModelRegistry.java @@ -0,0 +1,223 @@ +/* + * 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.formats; + +import java.util.List; +import java.util.Map; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.common.DynMethods; +import org.apache.iceberg.deletes.EqualityDeleteWriter; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.deletes.PositionDeleteWriter; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.io.DataWriter; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A registry that manages file-format-specific readers and writers through a unified object model + * factory interface. + * + *

This registry provides access to {@link ReadBuilder}s for data consumption and various writer + * builders: + * + *

    + *
  • {@link WriteBuilder} for basic file writing, + *
  • {@link DataWriteBuilder} for data files, + *
  • {@link EqualityDeleteWriteBuilder} for equality deletes, + *
  • {@link PositionDeleteWriteBuilder} for position deletes. + *
+ * + * The appropriate builder is selected based on {@link FileFormat} and object model name. + * + *

{@link FormatModel} objects are registered through {@link #register(FormatModel)} and used for + * creating readers and writers. Read builders are returned directly from the factory. Write + * builders may be wrapped in specialized content file writer implementations depending on the + * requested builder type. + */ +public final class FormatModelRegistry { + private static final Logger LOG = LoggerFactory.getLogger(FormatModelRegistry.class); + // The list of classes which are used for registering the reader and writer builders + private static final List CLASSES_TO_REGISTER = + ImmutableList.of( + "org.apache.iceberg.data.GenericFormatModels", + "org.apache.iceberg.arrow.vectorized.ArrowFormatModels", + "org.apache.iceberg.flink.data.FlinkFormatModels", + "org.apache.iceberg.spark.source.SparkFormatModels"); + + // Format models indexed by file format and object model class + private static final Map>, FormatModel> MODELS = + Maps.newConcurrentMap(); + + static { + registerSupportedFormats(); + } + + /** + * Registers an {@link FormatModel} in this registry. + * + *

The {@link FormatModel} creates readers and writers for a specific combinations of file + * format (Parquet, ORC, Avro) and object model (for example: "generic", "spark", "flink", etc.). + * Registering custom factories allows integration of new data processing engines for the + * supported file formats with Iceberg's file access mechanisms. + * + *

Each factory must be uniquely identified by its combination of file format and object model + * name. This uniqueness constraint prevents ambiguity when selecting factories for read and write + * operations. + * + * @param formatModel the factory implementation to register + * @throws IllegalArgumentException if a factory is already registered for the combination of + * {@link FormatModel#format()} and {@link FormatModel#type()} + */ + public static synchronized void register(FormatModel formatModel) { + Pair> key = Pair.of(formatModel.format(), formatModel.type()); + + FormatModel existing = MODELS.get(key); + Preconditions.checkArgument( + existing == null, + "Cannot register %s: %s is registered for format=%s type=%s schemaType=%s", + formatModel.getClass(), + existing == null ? null : existing.getClass(), + key.first(), + key.second(), + existing == null ? null : existing.schemaType()); + + MODELS.put(key, formatModel); + } + + /** + * Returns a reader builder for the specified file format and object model. + * + *

The returned {@link ReadBuilder} provides a fluent interface for configuring how data is + * read from the input file and converted to the output objects. + * + * @param format the file format (Parquet, Avro, ORC) that determines the parsing implementation + * @param type the output type + * @param inputFile source file to read data from + * @param the type of data records the reader will produce + * @param the type of the output schema for the reader + * @return a configured reader builder for the specified format and object model + */ + public static ReadBuilder readBuilder( + FileFormat format, Class type, InputFile inputFile) { + FormatModel factory = factoryFor(format, type); + return factory.readBuilder(inputFile); + } + + /** + * Returns a writer builder for generating a {@link DataFile}. + * + *

The returned builder produces a writer that accepts records defined by the specified object + * model and persists them using the provided file format. Unlike basic writers, this writer + * collects file metadata during the writing process and generates a {@link DataFile} that can be + * used for table operations. + * + * @param format the file format used for writing + * @param type the input type + * @param outputFile destination for the written data + * @param the type of data records the writer will accept + * @param the type of the input schema for the writer + * @return a configured data write builder for creating a {@link DataWriter} + */ + public static DataWriteBuilder dataWriteBuilder( + FileFormat format, Class type, EncryptedOutputFile outputFile) { + FormatModel factory = factoryFor(format, type); + return CommonWriteBuilderImpl.forDataFile( + factory.writeBuilder(outputFile), outputFile.encryptingOutputFile().location(), format); + } + + /** + * Creates a writer builder for generating a {@link DeleteFile} with equality deletes. + * + *

The returned builder produces a writer that accepts records defined by the specified object + * model and persists them using the given file format. The writer persists equality delete + * records that identify rows to be deleted based on the configured equality fields, producing a + * {@link DeleteFile} that can be used for table operations. + * + * @param format the file format used for writing + * @param type the input type + * @param outputFile destination for the written data + * @param the type of data records the writer will accept + * @param the type of the input schema for the writer + * @return a configured delete write builder for creating an {@link EqualityDeleteWriter} + */ + public static EqualityDeleteWriteBuilder equalityDeleteWriteBuilder( + FileFormat format, Class type, EncryptedOutputFile outputFile) { + FormatModel factory = factoryFor(format, type); + return CommonWriteBuilderImpl.forEqualityDelete( + factory.writeBuilder(outputFile), outputFile.encryptingOutputFile().location(), format); + } + + /** + * Creates a writer builder for generating a {@link DeleteFile} with position-based deletes. + * + *

The returned builder produces a writer that accepts records defined by the specified object + * model and persists them using the given file format. The writer accepts {@link PositionDelete} + * records that identify rows to be deleted by file path and position, producing a {@link + * DeleteFile} that can be used for table operations. + * + * @param format the file format used for writing + * @param outputFile destination for the written data + * @return a configured delete write builder for creating a {@link PositionDeleteWriter} + */ + @SuppressWarnings("rawtypes") + public static PositionDeleteWriteBuilder positionDeleteWriteBuilder( + FileFormat format, EncryptedOutputFile outputFile) { + FormatModel factory = factoryFor(format, PositionDelete.class); + return CommonWriteBuilderImpl.forPositionDelete( + factory.writeBuilder(outputFile), outputFile.encryptingOutputFile().location(), format); + } + + @VisibleForTesting + static Map>, FormatModel> models() { + return MODELS; + } + + @SuppressWarnings("unchecked") + private static FormatModel factoryFor(FileFormat format, Class type) { + FormatModel model = (FormatModel) MODELS.get(Pair.of(format, type)); + Preconditions.checkArgument( + model != null, "Format model is not registered for format %s and type %s", format, type); + return model; + } + + @SuppressWarnings("CatchBlockLogException") + private static void registerSupportedFormats() { + // Uses dynamic methods to call the `register` for the listed classes + for (String classToRegister : CLASSES_TO_REGISTER) { + try { + DynMethods.builder("register").impl(classToRegister).buildStaticChecked().invoke(); + } catch (NoSuchMethodException e) { + // failing to register a factory is normal and does not require a stack trace + LOG.info( + "Skip registration of {}. Likely the jar is not in the classpath", classToRegister); + } + } + } + + private FormatModelRegistry() {} +} diff --git a/core/src/main/java/org/apache/iceberg/formats/PositionDeleteWriteBuilder.java b/core/src/main/java/org/apache/iceberg/formats/PositionDeleteWriteBuilder.java new file mode 100644 index 000000000000..ee379bfc249d --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/formats/PositionDeleteWriteBuilder.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.formats; + +import java.io.IOException; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.deletes.PositionDeleteWriter; + +/** + * A specialized builder for creating position-based delete file writers. + * + *

This builder extends the generic {@link CommonWriteBuilder} interface with functionality + * specific to creating {@link PositionDeleteWriter} instances. + */ +public interface PositionDeleteWriteBuilder extends CommonWriteBuilder { + + /** + * Creates a position-based delete file writer configured with the current builder settings. + * + *

The returned {@link PositionDeleteWriter} produces files that identify records to be deleted + * by their file path and position, generating proper {@link DeleteFile} metadata on completion. + * The writer expects {@link PositionDelete} records as input. + * + * @param Only kept for backwards compatibility, the writer expects {@link PositionDelete} + * records as input, and the actual row data is not used. + * @return a fully configured {@link PositionDeleteWriter} instance + * @throws IOException if the writer cannot be created due to I/O errors + */ + PositionDeleteWriter build() throws IOException; +} diff --git a/core/src/main/java/org/apache/iceberg/formats/ReadBuilder.java b/core/src/main/java/org/apache/iceberg/formats/ReadBuilder.java new file mode 100644 index 000000000000..d023e2d028ab --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/formats/ReadBuilder.java @@ -0,0 +1,110 @@ +/* + * 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.formats; + +import java.util.Map; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.mapping.NameMapping; + +/** + * Builder interface for creating file readers across supported data file formats. The {@link + * FormatModel} implementations provides appropriate {@link ReadBuilder} instances + * + *

The {@link ReadBuilder} follows the builder pattern to configure and create {@link + * CloseableIterable} instances that read data from source files. Configuration options include + * schema projection, predicate filtering, record batching, and encryption settings. + * + *

This interface is directly exposed to users for parameterizing readers. + * + * @param the output data type produced by the reader + * @param the type of the schema for the output data type + */ +public interface ReadBuilder { + /** + * Restricts the read to the given range: [start, start + length). + * + * @param start the start position for this read + * @param length the length of the range this read should scan + */ + ReadBuilder split(long start, long length); + + /** Set the projection schema. */ + ReadBuilder project(Schema schema); + + /** + * Configures whether filtering should be case-sensitive. If the reader supports filtering, it + * must respect this setting. The default value is true. + * + * @param caseSensitive indicates if filtering is case-sensitive + */ + ReadBuilder caseSensitive(boolean caseSensitive); + + /** + * Pushes down the {@link Expression} filter for the reader to prevent reading unnecessary + * records. Some readers may not support filtering, or may only support filtering for certain + * expressions. In this case the reader might return unfiltered or partially filtered rows. It is + * the caller's responsibility to apply the filter again. + * + * @param filter the filter to set + */ + ReadBuilder filter(Expression filter); + + /** + * Set a reader configuration property which affects the reader behavior. Reader builders should + * ignore configuration keys not known for them. + * + * @param key a reader config property name + * @param value config value + * @return this for method chaining + */ + ReadBuilder set(String key, String value); + + /** + * Sets multiple reader configuration properties that affect the reader behavior. Reader builders + * should ignore configuration keys not known for them. + * + * @param properties reader config properties to set + * @return this for method chaining + */ + default ReadBuilder setAll(Map properties) { + properties.forEach(this::set); + return this; + } + + /** Enables reusing the containers returned by the reader. Decreases pressure on GC. */ + ReadBuilder reuseContainers(); + + /** Sets the batch size for vectorized readers. */ + ReadBuilder recordsPerBatch(int rowsPerBatch); + + /** + * Contains the values in the result objects which are coming from metadata and not coming from + * the data files themselves. The keys of the map are the column ids, the values are the constant + * values to be used in the result. + */ + ReadBuilder idToConstant(Map idToConstant); + + /** Sets a mapping from external schema names to Iceberg type IDs. */ + ReadBuilder withNameMapping(NameMapping nameMapping); + + /** Builds the reader. */ + CloseableIterable build(); +} diff --git a/core/src/main/java/org/apache/iceberg/formats/WriteBuilder.java b/core/src/main/java/org/apache/iceberg/formats/WriteBuilder.java new file mode 100644 index 000000000000..f1fee495e3da --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/formats/WriteBuilder.java @@ -0,0 +1,120 @@ +/* + * 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.formats; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Map; +import org.apache.iceberg.FileContent; +import org.apache.iceberg.MetricsConfig; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.FileAppender; + +/** + * Builder interface for creating file writers across supported data file formats. The {@link + * FormatModel} implementations provide the appropriate {@link WriteBuilder} instances. + * + *

The {@link WriteBuilder} follows the builder pattern to configure and create {@link + * FileAppender} instances that write data to the target output files. + * + *

This interface is directly exposed to users for parameterizing when only an appender is + * required. + * + * @param the output data type produced by the reader + * @param the type of the schema for the output data type + */ +public interface WriteBuilder { + /** Set the file schema. */ + WriteBuilder schema(Schema schema); + + /** + * Sets the input schema accepted by the writer. If not provided derived from the {@link + * #schema(Schema)}. + */ + WriteBuilder inputSchema(S schema); + + /** + * Set a writer configuration property which affects the writer behavior. Writer builders should + * ignore configuration keys not known for them. + * + * @param property a writer config property name + * @param value config value + * @return this for method chaining + */ + WriteBuilder set(String property, String value); + + /** + * Sets multiple writer configuration properties that affect the writer behavior. Writer builders + * should ignore configuration keys not known for them. + * + * @param properties writer config properties to set + * @return this for method chaining + */ + default WriteBuilder setAll(Map properties) { + properties.forEach(this::set); + return this; + } + + /** + * Set a file metadata property in the created file. + * + * @param property a file metadata property name + * @param value config value + * @return this for method chaining + */ + WriteBuilder meta(String property, String value); + + /** + * Sets multiple file metadata properties in the created file. + * + * @param properties file metadata properties to set + * @return this for method chaining + */ + default WriteBuilder meta(Map properties) { + properties.forEach(this::meta); + return this; + } + + /** + * Based on the target file content the generated {@link FileAppender} needs different + * configuration. + */ + WriteBuilder content(FileContent content); + + /** Sets the metrics configuration used for collecting column metrics for the created file. */ + WriteBuilder metricsConfig(MetricsConfig metricsConfig); + + /** Overwrite the file if it already exists. By default, overwrite is disabled. */ + WriteBuilder overwrite(); + + /** + * Sets the encryption key used for writing the file. If the writer does not support encryption, + * then an exception should be thrown. + */ + WriteBuilder withFileEncryptionKey(ByteBuffer encryptionKey); + + /** + * Sets the additional authentication data (AAD) prefix used for writing the file. If the writer + * does not support encryption, then an exception should be thrown. + */ + WriteBuilder withAADPrefix(ByteBuffer aadPrefix); + + /** Finalizes the configuration and builds the {@link FileAppender}. */ + FileAppender build() throws IOException; +} diff --git a/core/src/test/java/org/apache/iceberg/formats/TestFormatModelRegistry.java b/core/src/test/java/org/apache/iceberg/formats/TestFormatModelRegistry.java new file mode 100644 index 000000000000..24e168d3131b --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/formats/TestFormatModelRegistry.java @@ -0,0 +1,125 @@ +/* + * 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.formats; + +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.AssertionsForInterfaceTypes.assertThat; + +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.util.Pair; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +class TestFormatModelRegistry { + + @BeforeEach + void clearRegistry() { + FormatModelRegistry.models().clear(); + } + + @Test + void testSuccessfulRegister() { + FormatModel model = new DummyParquetFormatModel(Object.class, Object.class); + FormatModelRegistry.register(model); + assertThat(FormatModelRegistry.models()) + .containsEntry(Pair.of(FileFormat.PARQUET, Object.class), model); + } + + /** Tests that registering the same class with the same configuration updates the registration. */ + @Test + void testRegistrationForDifferentType() { + FormatModel model1 = new DummyParquetFormatModel(Object.class, Object.class); + FormatModel model2 = new DummyParquetFormatModel(Long.class, Object.class); + FormatModelRegistry.register(model1); + assertThat(FormatModelRegistry.models().get(Pair.of(FileFormat.PARQUET, model1.type()))) + .isSameAs(model1); + + // Registering a new model with the different format will succeed + FormatModelRegistry.register(model2); + assertThat(FormatModelRegistry.models().get(Pair.of(FileFormat.PARQUET, model1.type()))) + .isSameAs(model1); + assertThat(FormatModelRegistry.models().get(Pair.of(FileFormat.PARQUET, model2.type()))) + .isSameAs(model2); + } + + /** + * Tests that registering different classes, or different schema type for the same file format and + * type is failing. + */ + @Test + void testFailingReRegistrations() { + FormatModel model = new DummyParquetFormatModel(Object.class, Object.class); + FormatModelRegistry.register(model); + assertThat(FormatModelRegistry.models()) + .containsEntry(Pair.of(FileFormat.PARQUET, Object.class), model); + + // Registering a new model with different schema type should fail + assertThatThrownBy( + () -> + FormatModelRegistry.register( + new DummyParquetFormatModel(Object.class, String.class))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot register class"); + + // Registering a new model with null schema type should fail + assertThatThrownBy( + () -> FormatModelRegistry.register(new DummyParquetFormatModel(Object.class, null))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot register class"); + } + + private static class DummyParquetFormatModel implements FormatModel { + private final Class type; + private final Class schemaType; + + private DummyParquetFormatModel(Class type, Class schemaType) { + this.type = type; + this.schemaType = schemaType; + } + + @Override + public FileFormat format() { + return FileFormat.PARQUET; + } + + @Override + @SuppressWarnings("unchecked") + public Class type() { + return (Class) type; + } + + @Override + @SuppressWarnings("unchecked") + public Class schemaType() { + return (Class) schemaType; + } + + @Override + public WriteBuilder writeBuilder(EncryptedOutputFile outputFile) { + return null; + } + + @Override + public ReadBuilder readBuilder(InputFile inputFile) { + return null; + } + } +} diff --git a/data/src/main/java/org/apache/iceberg/data/BaseDeleteLoader.java b/data/src/main/java/org/apache/iceberg/data/BaseDeleteLoader.java index 99f5c742d37c..8dbb9dd44b8e 100644 --- a/data/src/main/java/org/apache/iceberg/data/BaseDeleteLoader.java +++ b/data/src/main/java/org/apache/iceberg/data/BaseDeleteLoader.java @@ -30,24 +30,18 @@ import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Schema; import org.apache.iceberg.StructLike; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.data.avro.PlannedDataReader; -import org.apache.iceberg.data.orc.GenericOrcReader; -import org.apache.iceberg.data.parquet.GenericParquetReaders; import org.apache.iceberg.deletes.Deletes; import org.apache.iceberg.deletes.PositionDeleteIndex; import org.apache.iceberg.deletes.PositionDeleteIndexUtil; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.formats.FormatModelRegistry; +import org.apache.iceberg.formats.ReadBuilder; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.DeleteSchemaUtil; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.RangeReadable; import org.apache.iceberg.io.SeekableInputStream; -import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.orc.OrcRowReader; -import org.apache.iceberg.parquet.Parquet; -import org.apache.iceberg.parquet.ParquetValueReader; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; @@ -58,8 +52,6 @@ import org.apache.iceberg.util.StructLikeSet; import org.apache.iceberg.util.Tasks; import org.apache.iceberg.util.ThreadPools; -import org.apache.orc.TypeDescription; -import org.apache.parquet.schema.MessageType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -229,44 +221,9 @@ private CloseableIterable openDeletes( LOG.trace("Opening delete file {}", deleteFile.location()); InputFile inputFile = loadInputFile.apply(deleteFile); - switch (format) { - case AVRO: - return Avro.read(inputFile) - .project(projection) - .reuseContainers() - .createResolvingReader(PlannedDataReader::create) - .build(); - - case PARQUET: - return Parquet.read(inputFile) - .project(projection) - .filter(filter) - .reuseContainers() - .createReaderFunc(newParquetReaderFunc(projection)) - .build(); - - case ORC: - // reusing containers is automatic for ORC, no need to call 'reuseContainers' - return ORC.read(inputFile) - .project(projection) - .filter(filter) - .createReaderFunc(newOrcReaderFunc(projection)) - .build(); - - default: - throw new UnsupportedOperationException( - String.format( - "Cannot read deletes, %s is not a supported file format: %s", - format.name(), inputFile.location())); - } - } - - private Function> newParquetReaderFunc(Schema projection) { - return fileSchema -> GenericParquetReaders.buildReader(projection, fileSchema); - } - - private Function> newOrcReaderFunc(Schema projection) { - return fileSchema -> GenericOrcReader.buildReader(projection, fileSchema); + ReadBuilder builder = + FormatModelRegistry.readBuilder(format, Record.class, inputFile); + return builder.project(projection).reuseContainers().filter(filter).build(); } private Iterable execute(Iterable objects, Function func) { diff --git a/data/src/main/java/org/apache/iceberg/data/BaseFileWriterFactory.java b/data/src/main/java/org/apache/iceberg/data/BaseFileWriterFactory.java index 55f3b5701e0b..486ea99f7aa6 100644 --- a/data/src/main/java/org/apache/iceberg/data/BaseFileWriterFactory.java +++ b/data/src/main/java/org/apache/iceberg/data/BaseFileWriterFactory.java @@ -40,7 +40,13 @@ import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -/** A base writer factory to be extended by query engine integrations. */ +/** + * A base writer factory to be extended by query engine integrations. + * + * @deprecated deprecated as of version 1.11.0 and will be removed in 1.12.0. Use {@link + * RegistryBasedFileWriterFactory} + */ +@Deprecated public abstract class BaseFileWriterFactory implements FileWriterFactory, Serializable { private final Table table; private final FileFormat dataFileFormat; @@ -75,13 +81,6 @@ protected BaseFileWriterFactory( this.positionDeleteRowSchema = null; } - /** - * @deprecated This constructor is deprecated as of version 1.11.0 and will be removed in 1.12.0. - * Position deletes that include row data are no longer supported. Use {@link - * #BaseFileWriterFactory(Table, FileFormat, Schema, SortOrder, FileFormat, int[], Schema, - * SortOrder, Map)} instead. - */ - @Deprecated protected BaseFileWriterFactory( Table table, FileFormat dataFileFormat, diff --git a/data/src/main/java/org/apache/iceberg/data/GenericFileWriterFactory.java b/data/src/main/java/org/apache/iceberg/data/GenericFileWriterFactory.java index e6872cc6e136..8d9b43f89685 100644 --- a/data/src/main/java/org/apache/iceberg/data/GenericFileWriterFactory.java +++ b/data/src/main/java/org/apache/iceberg/data/GenericFileWriterFactory.java @@ -22,21 +22,36 @@ import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT; import static org.apache.iceberg.TableProperties.DELETE_DEFAULT_FILE_FORMAT; +import java.io.IOException; +import java.io.UncheckedIOException; import java.util.Map; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetricsConfig; +import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.avro.Avro; import org.apache.iceberg.data.avro.DataWriter; import org.apache.iceberg.data.orc.GenericOrcWriter; import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.deletes.PositionDeleteWriter; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.formats.FormatModelRegistry; import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -public class GenericFileWriterFactory extends BaseFileWriterFactory { +public class GenericFileWriterFactory extends RegistryBasedFileWriterFactory { + private static final Logger LOG = LoggerFactory.getLogger(GenericFileWriterFactory.class); + + private Table table; + private FileFormat format; + private Schema positionDeleteRowSchema; GenericFileWriterFactory( Table table, @@ -50,13 +65,16 @@ public class GenericFileWriterFactory extends BaseFileWriterFactory { super( table, dataFileFormat, + Record.class, dataSchema, dataSortOrder, deleteFileFormat, equalityFieldIds, equalityDeleteRowSchema, equalityDeleteSortOrder, - ImmutableMap.of()); + ImmutableMap.of(), + null, + null); } /** @@ -80,14 +98,19 @@ public class GenericFileWriterFactory extends BaseFileWriterFactory { super( table, dataFileFormat, + Record.class, dataSchema, dataSortOrder, deleteFileFormat, equalityFieldIds, equalityDeleteRowSchema, equalityDeleteSortOrder, - positionDeleteRowSchema, - writerProperties); + writerProperties, + null, + null); + this.table = table; + this.format = dataFileFormat; + this.positionDeleteRowSchema = positionDeleteRowSchema; } /** @@ -107,62 +130,166 @@ public class GenericFileWriterFactory extends BaseFileWriterFactory { super( table, dataFileFormat, + Record.class, dataSchema, dataSortOrder, deleteFileFormat, equalityFieldIds, equalityDeleteRowSchema, equalityDeleteSortOrder, - positionDeleteRowSchema); + ImmutableMap.of(), + dataSchema, + equalityDeleteRowSchema); + this.table = table; + this.format = dataFileFormat; + this.positionDeleteRowSchema = positionDeleteRowSchema; } static Builder builderFor(Table table) { return new Builder(table); } - @Override + /** + * @deprecated Since 1.10.0, will be removed in 1.11.0. It won't be called starting 1.10.0 as the + * configuration is done by the {@link FormatModelRegistry}. + */ + @Deprecated protected void configureDataWrite(Avro.DataWriteBuilder builder) { - builder.createWriterFunc(DataWriter::create); + throwUnsupportedOperationException(); } - @Override + /** + * @deprecated Since 1.10.0, will be removed in 1.11.0. It won't be called starting 1.10.0 as the + * configuration is done by the {@link FormatModelRegistry}. + */ + @Deprecated protected void configureEqualityDelete(Avro.DeleteWriteBuilder builder) { - builder.createWriterFunc(DataWriter::create); + throwUnsupportedOperationException(); } - @Override + /** + * @deprecated Since 1.10.0, will be removed in 1.11.0. It won't be called starting 1.10.0 as the + * configuration is done by the {@link FormatModelRegistry}. + */ + @Deprecated protected void configurePositionDelete(Avro.DeleteWriteBuilder builder) { - builder.createWriterFunc(DataWriter::create); + throwUnsupportedOperationException(); } - @Override + /** + * @deprecated Since 1.10.0, will be removed in 1.11.0. It won't be called starting 1.10.0 as the + * configuration is done by the {@link FormatModelRegistry}. + */ + @Deprecated protected void configureDataWrite(Parquet.DataWriteBuilder builder) { - builder.createWriterFunc(GenericParquetWriter::create); + throwUnsupportedOperationException(); } - @Override + /** + * @deprecated Since 1.10.0, will be removed in 1.11.0. It won't be called starting 1.10.0 as the + * configuration is done by the {@link FormatModelRegistry}. + */ + @Deprecated protected void configureEqualityDelete(Parquet.DeleteWriteBuilder builder) { - builder.createWriterFunc(GenericParquetWriter::create); + throwUnsupportedOperationException(); } - @Override + /** + * @deprecated Since 1.10.0, will be removed in 1.11.0. It won't be called starting 1.10.0 as the + * configuration is done by the {@link FormatModelRegistry}. + */ + @Deprecated protected void configurePositionDelete(Parquet.DeleteWriteBuilder builder) { - builder.createWriterFunc(GenericParquetWriter::create); + throwUnsupportedOperationException(); } - @Override + /** + * @deprecated Since 1.10.0, will be removed in 1.11.0. It won't be called starting 1.10.0 as the + * configuration is done by the {@link FormatModelRegistry}. + */ + @Deprecated protected void configureDataWrite(ORC.DataWriteBuilder builder) { - builder.createWriterFunc(GenericOrcWriter::buildWriter); + throwUnsupportedOperationException(); } - @Override + /** + * @deprecated Since 1.10.0, will be removed in 1.11.0. It won't be called starting 1.10.0 as the + * configuration is done by the {@link FormatModelRegistry}. + */ + @Deprecated protected void configureEqualityDelete(ORC.DeleteWriteBuilder builder) { - builder.createWriterFunc(GenericOrcWriter::buildWriter); + throwUnsupportedOperationException(); } - @Override + /** + * @deprecated Since 1.10.0, will be removed in 1.11.0. It won't be called starting 1.10.0 as the + * configuration is done by the {@link FormatModelRegistry}. + */ + @Deprecated protected void configurePositionDelete(ORC.DeleteWriteBuilder builder) { - builder.createWriterFunc(GenericOrcWriter::buildWriter); + throwUnsupportedOperationException(); + } + + private void throwUnsupportedOperationException() { + throw new UnsupportedOperationException( + "Method is deprecated and should not be called. " + + "Configuration is already done by the registry."); + } + + @Override + public PositionDeleteWriter newPositionDeleteWriter( + EncryptedOutputFile file, PartitionSpec spec, StructLike partition) { + if (positionDeleteRowSchema == null) { + return super.newPositionDeleteWriter(file, spec, partition); + } else { + LOG.info( + "Deprecated feature used. Position delete row schema is used to create the position delete writer."); + MetricsConfig metricsConfig = + table != null + ? MetricsConfig.forPositionDelete(table) + : MetricsConfig.fromProperties(ImmutableMap.of()); + + try { + switch (format) { + case AVRO: + return Avro.writeDeletes(file) + .createWriterFunc(DataWriter::create) + .withPartition(partition) + .overwrite() + .rowSchema(positionDeleteRowSchema) + .withSpec(spec) + .withKeyMetadata(file.keyMetadata()) + .buildPositionWriter(); + + case ORC: + return ORC.writeDeletes(file) + .createWriterFunc(GenericOrcWriter::buildWriter) + .withPartition(partition) + .overwrite() + .rowSchema(positionDeleteRowSchema) + .withSpec(spec) + .withKeyMetadata(file.keyMetadata()) + .buildPositionWriter(); + + case PARQUET: + return Parquet.writeDeletes(file) + .createWriterFunc(GenericParquetWriter::create) + .withPartition(partition) + .overwrite() + .metricsConfig(metricsConfig) + .rowSchema(positionDeleteRowSchema) + .withSpec(spec) + .withKeyMetadata(file.keyMetadata()) + .buildPositionWriter(); + + default: + throw new UnsupportedOperationException( + "Cannot write pos-deletes for unsupported file format: " + format); + } + } catch (IOException e) { + throw new UncheckedIOException("Failed to create new position delete writer", e); + } + } } public static class Builder { diff --git a/data/src/main/java/org/apache/iceberg/data/GenericFormatModels.java b/data/src/main/java/org/apache/iceberg/data/GenericFormatModels.java new file mode 100644 index 000000000000..540b98c483e9 --- /dev/null +++ b/data/src/main/java/org/apache/iceberg/data/GenericFormatModels.java @@ -0,0 +1,99 @@ +/* + * 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.data; + +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.AvroFormatModel; +import org.apache.iceberg.data.avro.DataWriter; +import org.apache.iceberg.data.avro.PlannedDataReader; +import org.apache.iceberg.data.orc.GenericOrcReader; +import org.apache.iceberg.data.orc.GenericOrcWriter; +import org.apache.iceberg.data.parquet.GenericParquetReaders; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.formats.FormatModelRegistry; +import org.apache.iceberg.orc.ORCFormatModel; +import org.apache.iceberg.parquet.ParquetFormatModel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class GenericFormatModels { + private static final Logger LOG = LoggerFactory.getLogger(GenericFormatModels.class); + + public static void register() { + // ORC, Parquet are optional dependencies. If they are not present, we should just log and + // ignore NoClassDefFoundErrors + registerAvro(); + registerParquet(); + registerOrc(); + } + + private static void registerParquet() { + logAngIgnoreNoClassDefFoundError( + () -> + FormatModelRegistry.register( + new ParquetFormatModel<>( + Record.class, + Schema.class, + GenericParquetReaders::buildReader, + (schema, messageType, inputType) -> + GenericParquetWriter.create(schema, messageType)))); + logAngIgnoreNoClassDefFoundError( + () -> FormatModelRegistry.register(new ParquetFormatModel<>(PositionDelete.class))); + } + + private static void registerAvro() { + logAngIgnoreNoClassDefFoundError( + () -> + FormatModelRegistry.register( + new AvroFormatModel<>( + Record.class, + Schema.class, + PlannedDataReader::create, + (schema, inputSchema) -> DataWriter.create(schema)))); + logAngIgnoreNoClassDefFoundError( + () -> FormatModelRegistry.register(new AvroFormatModel<>(PositionDelete.class))); + } + + private static void registerOrc() { + logAngIgnoreNoClassDefFoundError( + () -> + FormatModelRegistry.register( + new ORCFormatModel<>( + Record.class, + Schema.class, + GenericOrcReader::buildReader, + (schema, typeDescription, unused) -> + GenericOrcWriter.buildWriter(schema, typeDescription)))); + logAngIgnoreNoClassDefFoundError( + () -> FormatModelRegistry.register(new ORCFormatModel<>(PositionDelete.class))); + } + + private GenericFormatModels() {} + + @SuppressWarnings("CatchBlockLogException") + private static void logAngIgnoreNoClassDefFoundError(Runnable runnable) { + try { + runnable.run(); + } catch (NoClassDefFoundError e) { + // Log the exception and ignore it + LOG.info("Exception occurred when trying to register format models: {}", e.getMessage()); + } + } +} diff --git a/data/src/main/java/org/apache/iceberg/data/GenericReader.java b/data/src/main/java/org/apache/iceberg/data/GenericReader.java index 9a1455f80fb0..f18f5785105f 100644 --- a/data/src/main/java/org/apache/iceberg/data/GenericReader.java +++ b/data/src/main/java/org/apache/iceberg/data/GenericReader.java @@ -22,26 +22,19 @@ import java.util.Map; import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Schema; import org.apache.iceberg.TableScan; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.data.avro.PlannedDataReader; -import org.apache.iceberg.data.orc.GenericOrcReader; -import org.apache.iceberg.data.parquet.GenericParquetReaders; import org.apache.iceberg.expressions.Evaluator; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.formats.FormatModelRegistry; +import org.apache.iceberg.formats.ReadBuilder; import org.apache.iceberg.io.CloseableGroup; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.CloseableIterator; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.InputFile; -import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.PartitionUtil; class GenericReader implements Serializable { @@ -96,58 +89,19 @@ private CloseableIterable openFile(FileScanTask task, Schema fileProject Map partition = PartitionUtil.constantsMap(task, IdentityPartitionConverters::convertConstant); - switch (task.file().format()) { - case AVRO: - Avro.ReadBuilder avro = - Avro.read(input) - .project(fileProjection) - .createResolvingReader(schema -> PlannedDataReader.create(schema, partition)) - .split(task.start(), task.length()); - - if (reuseContainers) { - avro.reuseContainers(); - } - - return avro.build(); - - case PARQUET: - Parquet.ReadBuilder parquet = - Parquet.read(input) - .project(fileProjection) - .createReaderFunc( - fileSchema -> - GenericParquetReaders.buildReader(fileProjection, fileSchema, partition)) - .split(task.start(), task.length()) - .caseSensitive(caseSensitive) - .filter(task.residual()); - - if (reuseContainers) { - parquet.reuseContainers(); - } - - return parquet.build(); - - case ORC: - Schema projectionWithoutConstantAndMetadataFields = - TypeUtil.selectNot( - fileProjection, Sets.union(partition.keySet(), MetadataColumns.metadataFieldIds())); - ORC.ReadBuilder orc = - ORC.read(input) - .project(projectionWithoutConstantAndMetadataFields) - .createReaderFunc( - fileSchema -> - GenericOrcReader.buildReader(fileProjection, fileSchema, partition)) - .split(task.start(), task.length()) - .caseSensitive(caseSensitive) - .filter(task.residual()); - - return orc.build(); - - default: - throw new UnsupportedOperationException( - String.format( - "Cannot read %s file: %s", task.file().format().name(), task.file().location())); + ReadBuilder builder = + FormatModelRegistry.readBuilder(task.file().format(), Record.class, input); + if (reuseContainers) { + builder = builder.reuseContainers(); } + + return builder + .project(fileProjection) + .idToConstant(partition) + .split(task.start(), task.length()) + .caseSensitive(caseSensitive) + .filter(task.residual()) + .build(); } private class CombinedTaskIterable extends CloseableGroup implements CloseableIterable { diff --git a/data/src/main/java/org/apache/iceberg/data/RegistryBasedFileWriterFactory.java b/data/src/main/java/org/apache/iceberg/data/RegistryBasedFileWriterFactory.java new file mode 100644 index 000000000000..57ceb2f4c565 --- /dev/null +++ b/data/src/main/java/org/apache/iceberg/data/RegistryBasedFileWriterFactory.java @@ -0,0 +1,184 @@ +/* + * 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.data; + +import java.io.IOException; +import java.io.Serializable; +import java.io.UncheckedIOException; +import java.util.Map; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetricsConfig; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.deletes.EqualityDeleteWriter; +import org.apache.iceberg.deletes.PositionDeleteWriter; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.encryption.EncryptionKeyMetadata; +import org.apache.iceberg.formats.DataWriteBuilder; +import org.apache.iceberg.formats.EqualityDeleteWriteBuilder; +import org.apache.iceberg.formats.FormatModelRegistry; +import org.apache.iceberg.formats.PositionDeleteWriteBuilder; +import org.apache.iceberg.io.DataWriter; +import org.apache.iceberg.io.FileWriterFactory; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; + +/** + * A base writer factory to be extended by query engine integrations. + * + * @param row type + */ +public abstract class RegistryBasedFileWriterFactory + implements FileWriterFactory, Serializable { + private final Table table; + private final FileFormat dataFileFormat; + private final Class inputType; + private final Schema dataSchema; + private final SortOrder dataSortOrder; + private final FileFormat deleteFileFormat; + private final int[] equalityFieldIds; + private final Schema equalityDeleteRowSchema; + private final SortOrder equalityDeleteSortOrder; + private final Map writerProperties; + private final S inputSchema; + private final S equalityDeleteInputSchema; + + protected RegistryBasedFileWriterFactory( + Table table, + FileFormat dataFileFormat, + Class inputType, + Schema dataSchema, + SortOrder dataSortOrder, + FileFormat deleteFileFormat, + int[] equalityFieldIds, + Schema equalityDeleteRowSchema, + SortOrder equalityDeleteSortOrder, + Map writerProperties, + S inputSchema, + S equalityDeleteInputSchema) { + this.table = table; + this.dataFileFormat = dataFileFormat; + this.inputType = inputType; + this.dataSchema = dataSchema; + this.dataSortOrder = dataSortOrder; + this.deleteFileFormat = deleteFileFormat; + this.equalityFieldIds = equalityFieldIds; + this.equalityDeleteRowSchema = equalityDeleteRowSchema; + this.equalityDeleteSortOrder = equalityDeleteSortOrder; + this.writerProperties = writerProperties != null ? writerProperties : ImmutableMap.of(); + this.inputSchema = inputSchema; + this.equalityDeleteInputSchema = equalityDeleteInputSchema; + } + + protected S inputSchema() { + return inputSchema; + } + + protected S equalityDeleteInputSchema() { + return equalityDeleteInputSchema; + } + + @Override + public DataWriter newDataWriter( + EncryptedOutputFile file, PartitionSpec spec, StructLike partition) { + Preconditions.checkNotNull(dataSchema, "Data schema must not be null"); + EncryptionKeyMetadata keyMetadata = file.keyMetadata(); + Map properties = table != null ? table.properties() : ImmutableMap.of(); + MetricsConfig metricsConfig = + table != null ? MetricsConfig.forTable(table) : MetricsConfig.getDefault(); + + try { + DataWriteBuilder builder = + FormatModelRegistry.dataWriteBuilder(dataFileFormat, inputType, file); + return builder + .schema(dataSchema) + .inputSchema(inputSchema()) + .setAll(properties) + .setAll(writerProperties) + .metricsConfig(metricsConfig) + .spec(spec) + .partition(partition) + .keyMetadata(keyMetadata) + .sortOrder(dataSortOrder) + .overwrite() + .build(); + } catch (IOException e) { + throw new UncheckedIOException("Failed to create new data writer", e); + } + } + + @Override + public EqualityDeleteWriter newEqualityDeleteWriter( + EncryptedOutputFile file, PartitionSpec spec, StructLike partition) { + Preconditions.checkNotNull(equalityDeleteRowSchema, "Equality delete schema must not be null"); + + EncryptionKeyMetadata keyMetadata = file.keyMetadata(); + Map properties = table != null ? table.properties() : ImmutableMap.of(); + MetricsConfig metricsConfig = + table != null ? MetricsConfig.forTable(table) : MetricsConfig.getDefault(); + + try { + EqualityDeleteWriteBuilder builder = + FormatModelRegistry.equalityDeleteWriteBuilder(deleteFileFormat, inputType, file); + return builder + .setAll(properties) + .setAll(writerProperties) + .metricsConfig(metricsConfig) + .rowSchema(equalityDeleteRowSchema) + .inputSchema(equalityDeleteInputSchema()) + .equalityFieldIds(equalityFieldIds) + .spec(spec) + .partition(partition) + .keyMetadata(keyMetadata) + .sortOrder(equalityDeleteSortOrder) + .overwrite() + .build(); + } catch (IOException e) { + throw new UncheckedIOException("Failed to create new equality delete writer", e); + } + } + + @Override + public PositionDeleteWriter newPositionDeleteWriter( + EncryptedOutputFile file, PartitionSpec spec, StructLike partition) { + EncryptionKeyMetadata keyMetadata = file.keyMetadata(); + Map properties = table != null ? table.properties() : ImmutableMap.of(); + MetricsConfig metricsConfig = + table != null ? MetricsConfig.forPositionDelete(table) : MetricsConfig.forPositionDelete(); + + try { + PositionDeleteWriteBuilder builder = + FormatModelRegistry.positionDeleteWriteBuilder(deleteFileFormat, file); + return builder + .setAll(properties) + .setAll(writerProperties) + .metricsConfig(metricsConfig) + .spec(spec) + .partition(partition) + .keyMetadata(keyMetadata) + .overwrite() + .build(); + } catch (IOException e) { + throw new UncheckedIOException("Failed to create new position delete writer", e); + } + } +} diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkFormatModels.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkFormatModels.java new file mode 100644 index 000000000000..ee307b7a7c71 --- /dev/null +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkFormatModels.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.data; + +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.avro.AvroFormatModel; +import org.apache.iceberg.formats.FormatModelRegistry; +import org.apache.iceberg.orc.ORCFormatModel; +import org.apache.iceberg.parquet.ParquetFormatModel; + +public class FlinkFormatModels { + public static void register() { + FormatModelRegistry.register( + new ParquetFormatModel<>( + RowData.class, + RowType.class, + FlinkParquetReaders::buildReader, + (unused, messageType, rowType) -> + FlinkParquetWriters.buildWriter(rowType, messageType))); + + FormatModelRegistry.register( + new AvroFormatModel<>( + RowData.class, + RowType.class, + FlinkPlannedAvroReader::create, + (unused, rowType) -> new FlinkAvroWriter(rowType))); + + FormatModelRegistry.register( + new ORCFormatModel<>( + RowData.class, + RowType.class, + FlinkOrcReader::new, + (schema, unused, rowType) -> FlinkOrcWriter.buildWriter(rowType, schema))); + } + + private FlinkFormatModels() {} +} diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java index 1440fde3248c..a76bac515b3d 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.data; import java.util.List; +import org.apache.flink.annotation.Internal; import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.MapType; @@ -29,9 +30,10 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +@Internal abstract class FlinkSchemaVisitor { - static T visit(RowType flinkType, Schema schema, FlinkSchemaVisitor visitor) { + public static T visit(RowType flinkType, Schema schema, FlinkSchemaVisitor visitor) { return visit(flinkType, schema.asStruct(), visitor); } @@ -94,24 +96,29 @@ private static T visitRecord( List fieldTypes = Lists.newArrayListWithExpectedSize(fieldSize); List nestedFields = struct.fields(); - for (int i = 0; i < fieldSize; i++) { - Types.NestedField iField = nestedFields.get(i); - int fieldIndex = rowType.getFieldIndex(iField.name()); - Preconditions.checkArgument( - fieldIndex >= 0, "NestedField: %s is not found in flink RowType: %s", iField, rowType); + visitor.beforeStruct(struct.asStructType()); - LogicalType fieldFlinkType = rowType.getTypeAt(fieldIndex); + try { + for (int i = 0; i < fieldSize; i++) { + Types.NestedField iField = nestedFields.get(i); + int fieldIndex = rowType.getFieldIndex(iField.name()); + Preconditions.checkArgument( + fieldIndex >= 0, "NestedField: %s is not found in flink RowType: %s", iField, rowType); - fieldTypes.add(fieldFlinkType); + LogicalType fieldFlinkType = rowType.getTypeAt(fieldIndex); + fieldTypes.add(fieldFlinkType); - visitor.beforeField(iField); - try { - if (iField.type() != Types.UnknownType.get()) { - results.add(visit(fieldFlinkType, iField.type(), visitor)); + visitor.beforeField(iField); + try { + if (iField.type() != Types.UnknownType.get()) { + results.add(visit(fieldFlinkType, iField.type(), visitor)); + } + } finally { + visitor.afterField(iField); } - } finally { - visitor.afterField(iField); } + } finally { + visitor.afterStruct(struct.asStructType()); } return visitor.record(struct, results, fieldTypes); @@ -137,6 +144,10 @@ public void beforeField(Types.NestedField field) {} public void afterField(Types.NestedField field) {} + public void beforeStruct(Types.StructType type) {} + + public void afterStruct(Types.StructType type) {} + public void beforeListElement(Types.NestedField elementField) { beforeField(elementField); } diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java index b3ada41737bc..d5247941d863 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java @@ -25,28 +25,19 @@ import java.io.Serializable; import java.util.Map; import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.data.BaseFileWriterFactory; +import org.apache.iceberg.data.RegistryBasedFileWriterFactory; import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.data.FlinkAvroWriter; -import org.apache.iceberg.flink.data.FlinkOrcWriter; -import org.apache.iceberg.flink.data.FlinkParquetWriters; -import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -public class FlinkFileWriterFactory extends BaseFileWriterFactory implements Serializable { - private RowType dataFlinkType; - private RowType equalityDeleteFlinkType; - - private FlinkFileWriterFactory( +public class FlinkFileWriterFactory extends RegistryBasedFileWriterFactory + implements Serializable { + FlinkFileWriterFactory( Table table, FileFormat dataFileFormat, Schema dataSchema, @@ -62,85 +53,30 @@ private FlinkFileWriterFactory( super( table, dataFileFormat, + RowData.class, dataSchema, dataSortOrder, deleteFileFormat, equalityFieldIds, equalityDeleteRowSchema, equalityDeleteSortOrder, - writeProperties); - - this.dataFlinkType = dataFlinkType; - this.equalityDeleteFlinkType = equalityDeleteFlinkType; - } - - static Builder builderFor(Table table) { - return new Builder(table); - } - - @Override - protected void configureDataWrite(Avro.DataWriteBuilder builder) { - builder.createWriterFunc(ignore -> new FlinkAvroWriter(dataFlinkType())); - } - - @Override - protected void configureEqualityDelete(Avro.DeleteWriteBuilder builder) { - builder.createWriterFunc(ignored -> new FlinkAvroWriter(equalityDeleteFlinkType())); - } - - @Override - protected void configurePositionDelete(Avro.DeleteWriteBuilder builder) {} - - @Override - protected void configureDataWrite(Parquet.DataWriteBuilder builder) { - builder.createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(dataFlinkType(), msgType)); - } - - @Override - protected void configureEqualityDelete(Parquet.DeleteWriteBuilder builder) { - builder.createWriterFunc( - msgType -> FlinkParquetWriters.buildWriter(equalityDeleteFlinkType(), msgType)); - } - - @Override - protected void configurePositionDelete(Parquet.DeleteWriteBuilder builder) { - builder.transformPaths(path -> StringData.fromString(path.toString())); - } - - @Override - protected void configureDataWrite(ORC.DataWriteBuilder builder) { - builder.createWriterFunc( - (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(dataFlinkType(), iSchema)); - } - - @Override - protected void configureEqualityDelete(ORC.DeleteWriteBuilder builder) { - builder.createWriterFunc( - (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(equalityDeleteFlinkType(), iSchema)); + writeProperties, + dataFlinkType == null ? FlinkSchemaUtil.convert(dataSchema) : dataFlinkType, + equalityDeleteInputSchema(equalityDeleteFlinkType, equalityDeleteRowSchema)); } - @Override - protected void configurePositionDelete(ORC.DeleteWriteBuilder builder) { - builder.transformPaths(path -> StringData.fromString(path.toString())); - } - - private RowType dataFlinkType() { - if (dataFlinkType == null) { - Preconditions.checkNotNull(dataSchema(), "Data schema must not be null"); - this.dataFlinkType = FlinkSchemaUtil.convert(dataSchema()); + private static RowType equalityDeleteInputSchema(RowType rowType, Schema rowSchema) { + if (rowType != null) { + return rowType; + } else if (rowSchema != null) { + return FlinkSchemaUtil.convert(rowSchema); + } else { + return null; } - - return dataFlinkType; } - private RowType equalityDeleteFlinkType() { - if (equalityDeleteFlinkType == null) { - Preconditions.checkNotNull( - equalityDeleteRowSchema(), "Equality delete schema must not be null"); - this.equalityDeleteFlinkType = FlinkSchemaUtil.convert(equalityDeleteRowSchema()); - } - - return equalityDeleteFlinkType; + static Builder builderFor(Table table) { + return new Builder(table); } public static class Builder { diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java index b8fb1ba32edf..586cdc415993 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java @@ -24,10 +24,8 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Schema; import org.apache.iceberg.StructLike; -import org.apache.iceberg.avro.Avro; import org.apache.iceberg.data.DeleteFilter; import org.apache.iceberg.encryption.InputFilesDecryptor; import org.apache.iceberg.expressions.Expression; @@ -35,19 +33,14 @@ import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.FlinkSourceFilter; import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.data.FlinkOrcReader; -import org.apache.iceberg.flink.data.FlinkParquetReaders; -import org.apache.iceberg.flink.data.FlinkPlannedAvroReader; import org.apache.iceberg.flink.data.RowDataProjection; import org.apache.iceberg.flink.data.RowDataUtil; +import org.apache.iceberg.formats.FormatModelRegistry; +import org.apache.iceberg.formats.ReadBuilder; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.CloseableIterator; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.mapping.NameMappingParser; -import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.parquet.Parquet; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.PartitionUtil; @Internal @@ -73,8 +66,7 @@ public RowDataFileScanTaskReader( if (filters != null && !filters.isEmpty()) { Expression combinedExpression = filters.stream().reduce(Expressions.alwaysTrue(), Expressions::and); - this.rowFilter = - new FlinkSourceFilter(this.projectedSchema, combinedExpression, this.caseSensitive); + this.rowFilter = new FlinkSourceFilter(projectedSchema, combinedExpression, caseSensitive); } else { this.rowFilter = null; } @@ -112,23 +104,23 @@ private CloseableIterable newIterable( if (task.isDataTask()) { throw new UnsupportedOperationException("Cannot read data task."); } else { - switch (task.file().format()) { - case PARQUET: - iter = newParquetIterable(task, schema, idToConstant, inputFilesDecryptor); - break; - - case AVRO: - iter = newAvroIterable(task, schema, idToConstant, inputFilesDecryptor); - break; - - case ORC: - iter = newOrcIterable(task, schema, idToConstant, inputFilesDecryptor); - break; - - default: - throw new UnsupportedOperationException( - "Cannot read unknown format: " + task.file().format()); + ReadBuilder builder = + FormatModelRegistry.readBuilder( + task.file().format(), RowData.class, inputFilesDecryptor.getInputFile(task)); + + if (nameMapping != null) { + builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); } + + iter = + builder + .project(schema) + .idToConstant(idToConstant) + .split(task.start(), task.length()) + .caseSensitive(caseSensitive) + .filter(task.residual()) + .reuseContainers() + .build(); } if (rowFilter != null) { @@ -137,72 +129,6 @@ private CloseableIterable newIterable( return iter; } - private CloseableIterable newAvroIterable( - FileScanTask task, - Schema schema, - Map idToConstant, - InputFilesDecryptor inputFilesDecryptor) { - Avro.ReadBuilder builder = - Avro.read(inputFilesDecryptor.getInputFile(task)) - .reuseContainers() - .project(schema) - .split(task.start(), task.length()) - .createReaderFunc(readSchema -> FlinkPlannedAvroReader.create(schema, idToConstant)); - - if (nameMapping != null) { - builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); - } - - return builder.build(); - } - - private CloseableIterable newParquetIterable( - FileScanTask task, - Schema schema, - Map idToConstant, - InputFilesDecryptor inputFilesDecryptor) { - Parquet.ReadBuilder builder = - Parquet.read(inputFilesDecryptor.getInputFile(task)) - .split(task.start(), task.length()) - .project(schema) - .createReaderFunc( - fileSchema -> FlinkParquetReaders.buildReader(schema, fileSchema, idToConstant)) - .filter(task.residual()) - .caseSensitive(caseSensitive) - .reuseContainers(); - - if (nameMapping != null) { - builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); - } - - return builder.build(); - } - - private CloseableIterable newOrcIterable( - FileScanTask task, - Schema schema, - Map idToConstant, - InputFilesDecryptor inputFilesDecryptor) { - Schema readSchemaWithoutConstantAndMetadataFields = - TypeUtil.selectNot( - schema, Sets.union(idToConstant.keySet(), MetadataColumns.metadataFieldIds())); - - ORC.ReadBuilder builder = - ORC.read(inputFilesDecryptor.getInputFile(task)) - .project(readSchemaWithoutConstantAndMetadataFields) - .split(task.start(), task.length()) - .createReaderFunc( - readOrcSchema -> new FlinkOrcReader(schema, readOrcSchema, idToConstant)) - .filter(task.residual()) - .caseSensitive(caseSensitive); - - if (nameMapping != null) { - builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); - } - - return builder.build(); - } - private static class FlinkDeleteFilter extends DeleteFilter { private final RowType requiredRowType; private final RowDataWrapper asStructLike; diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java index da5b5f6c28f0..3d032fac64e9 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java @@ -35,7 +35,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.common.DynFields; -import org.apache.iceberg.data.BaseFileWriterFactory; +import org.apache.iceberg.data.RegistryBasedFileWriterFactory; import org.apache.iceberg.flink.FlinkWriteConf; import org.apache.iceberg.flink.FlinkWriteOptions; import org.apache.iceberg.flink.SimpleDataUtil; @@ -252,7 +252,7 @@ private static Map appenderProperties( .build(writerField.get()); DynFields.BoundField> propsField = DynFields.builder() - .hiddenImpl(BaseFileWriterFactory.class, "writerProperties") + .hiddenImpl(RegistryBasedFileWriterFactory.class, "writerProperties") .build(writerFactoryField.get()); return propsField.get(); } diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java index 7f4f7758e519..9f508bbe717d 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java @@ -34,6 +34,8 @@ import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; @@ -319,11 +321,32 @@ public void testTableWithTargetFileSize() throws Exception { public void testPromotedFlinkDataType() throws Exception { Schema iSchema = new Schema( + Types.NestedField.required( + 4, "array", Types.ListType.ofOptional(5, Types.IntegerType.get())), + Types.NestedField.required( + 6, + "map", + Types.MapType.ofOptional(7, 8, Types.IntegerType.get(), Types.IntegerType.get())), + Types.NestedField.required( + 9, + "struct", + Types.StructType.of( + Types.NestedField.optional(10, "struct_1", Types.IntegerType.get()), + Types.NestedField.optional(11, "struct_2", Types.IntegerType.get()))), Types.NestedField.required(1, "tinyint", Types.IntegerType.get()), Types.NestedField.required(2, "smallint", Types.IntegerType.get()), Types.NestedField.optional(3, "int", Types.IntegerType.get())); ResolvedSchema flinkSchema = ResolvedSchema.of( + Column.physical("array", DataTypes.ARRAY(DataTypes.TINYINT()).notNull()), + Column.physical( + "map", DataTypes.MAP(DataTypes.TINYINT(), DataTypes.TINYINT()).notNull()), + Column.physical( + "struct", + DataTypes.ROW( + DataTypes.FIELD("struct_1", DataTypes.TINYINT()), + DataTypes.FIELD("struct_2", DataTypes.TINYINT())) + .notNull()), Column.physical("tinyint", DataTypes.TINYINT().notNull()), Column.physical("smallint", DataTypes.SMALLINT().notNull()), Column.physical("int", DataTypes.INT().nullable())); @@ -347,16 +370,74 @@ public void testPromotedFlinkDataType() throws Exception { List rows = Lists.newArrayList( - GenericRowData.of((byte) 0x01, (short) -32768, 101), - GenericRowData.of((byte) 0x02, (short) 0, 102), - GenericRowData.of((byte) 0x03, (short) 32767, 103)); + GenericRowData.of( + new GenericArrayData(new byte[] {(byte) 0x04, (byte) 0x05}), + new GenericMapData(ImmutableMap.of((byte) 0x06, (byte) 0x07)), + GenericRowData.of((byte) 0x08, (byte) 0x09), + (byte) 0x01, + (short) -32768, + 101), + GenericRowData.of( + new GenericArrayData(new byte[] {(byte) 0x0a, (byte) 0x0b}), + new GenericMapData(ImmutableMap.of((byte) 0x0c, (byte) 0x0d)), + GenericRowData.of((byte) 0x0e, (byte) 0x0f), + (byte) 0x02, + (short) 0, + 102), + GenericRowData.of( + new GenericArrayData(new byte[] {(byte) 0x10, (byte) 0x11}), + new GenericMapData(ImmutableMap.of((byte) 0x12, (byte) 0x13)), + GenericRowData.of((byte) 0x14, (byte) 0x15), + (byte) 0x03, + (short) 32767, + 103)); Record record = GenericRecord.create(iSchema); + Record struct = GenericRecord.create(iSchema.findField("struct").type().asStructType()); List expected = Lists.newArrayList( - record.copy(ImmutableMap.of("tinyint", 1, "smallint", -32768, "int", 101)), - record.copy(ImmutableMap.of("tinyint", 2, "smallint", 0, "int", 102)), - record.copy(ImmutableMap.of("tinyint", 3, "smallint", 32767, "int", 103))); + record.copy( + ImmutableMap.of( + "array", + Lists.newArrayList(4, 5), + "map", + ImmutableMap.of(6, 7), + "struct", + struct.copy(ImmutableMap.of("struct_1", 8, "struct_2", 9)), + "tinyint", + 1, + "smallint", + -32768, + "int", + 101)), + record.copy( + ImmutableMap.of( + "array", + Lists.newArrayList(10, 11), + "map", + ImmutableMap.of(12, 13), + "struct", + struct.copy(ImmutableMap.of("struct_1", 14, "struct_2", 15)), + "tinyint", + 2, + "smallint", + 0, + "int", + 102)), + record.copy( + ImmutableMap.of( + "array", + Lists.newArrayList(16, 17), + "map", + ImmutableMap.of(18, 19), + "struct", + struct.copy(ImmutableMap.of("struct_1", 20, "struct_2", 21)), + "tinyint", + 3, + "smallint", + 32767, + "int", + 103))); try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter(icebergTable, flinkSchema)) { diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java index d17848225f69..4dbf9f3129a3 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java @@ -33,7 +33,7 @@ import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.common.DynFields; -import org.apache.iceberg.data.BaseFileWriterFactory; +import org.apache.iceberg.data.RegistryBasedFileWriterFactory; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.sink.TestFlinkIcebergSinkBase; import org.apache.iceberg.io.BaseTaskWriter; @@ -280,7 +280,7 @@ private Map properties(DynamicWriter dynamicWriter) { .build(writerField.get().values().iterator().next()); DynFields.BoundField> propsField = DynFields.builder() - .hiddenImpl(BaseFileWriterFactory.class, "writerProperties") + .hiddenImpl(RegistryBasedFileWriterFactory.class, "writerProperties") .build(writerFactoryField.get()); return propsField.get(); } diff --git a/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java b/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java index 58966c666d5d..175109f38252 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java +++ b/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java @@ -21,13 +21,9 @@ import java.io.IOException; import java.io.Serializable; import java.io.UncheckedIOException; -import java.util.Collections; import java.util.Iterator; import java.util.List; -import java.util.Map; -import java.util.Set; import java.util.concurrent.ExecutorService; -import java.util.function.BiFunction; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.InputFormat; import org.apache.hadoop.mapreduce.InputSplit; @@ -39,8 +35,6 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DataTableScan; import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.MetadataColumns; -import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.SchemaParser; import org.apache.iceberg.SerializableTable; @@ -49,19 +43,17 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.TableScan; -import org.apache.iceberg.avro.Avro; import org.apache.iceberg.data.DeleteFilter; import org.apache.iceberg.data.GenericDeleteFilter; -import org.apache.iceberg.data.IdentityPartitionConverters; import org.apache.iceberg.data.InternalRecordWrapper; -import org.apache.iceberg.data.avro.PlannedDataReader; -import org.apache.iceberg.data.orc.GenericOrcReader; -import org.apache.iceberg.data.parquet.GenericParquetReaders; +import org.apache.iceberg.data.Record; import org.apache.iceberg.encryption.EncryptedFiles; import org.apache.iceberg.encryption.EncryptionManager; import org.apache.iceberg.expressions.Evaluator; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.formats.FormatModelRegistry; +import org.apache.iceberg.formats.ReadBuilder; import org.apache.iceberg.hadoop.HadoopConfigurable; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.CloseableIterator; @@ -70,13 +62,7 @@ import org.apache.iceberg.mapping.NameMappingParser; import org.apache.iceberg.mr.Catalogs; import org.apache.iceberg.mr.InputFormatConfig; -import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.util.PartitionUtil; import org.apache.iceberg.util.SerializationUtil; import org.apache.iceberg.util.ThreadPools; @@ -326,23 +312,27 @@ private CloseableIterable openTask(FileScanTask currentTask, Schema readSchem encryptionManager.decrypt( EncryptedFiles.encryptedInput(io.newInputFile(file.location()), file.keyMetadata())); - CloseableIterable iterable; - switch (file.format()) { - case AVRO: - iterable = newAvroIterable(inputFile, currentTask, readSchema); - break; - case ORC: - iterable = newOrcIterable(inputFile, currentTask, readSchema); - break; - case PARQUET: - iterable = newParquetIterable(inputFile, currentTask, readSchema); - break; - default: - throw new UnsupportedOperationException( - String.format("Cannot read %s file: %s", file.format().name(), file.location())); + ReadBuilder readBuilder = + FormatModelRegistry.readBuilder(file.format(), Record.class, inputFile); + + if (reuseContainers) { + readBuilder = readBuilder.reuseContainers(); } - return iterable; + if (nameMapping != null) { + readBuilder = readBuilder.withNameMapping(NameMappingParser.fromJson(nameMapping)); + } + + return applyResidualFiltering( + (CloseableIterable) + readBuilder + .project(readSchema) + .split(currentTask.start(), currentTask.length()) + .caseSensitive(caseSensitive) + .filter(currentTask.residual()) + .build(), + currentTask.residual(), + readSchema); } @SuppressWarnings("unchecked") @@ -369,86 +359,6 @@ private CloseableIterable applyResidualFiltering( } } - private CloseableIterable newAvroIterable( - InputFile inputFile, FileScanTask task, Schema readSchema) { - Avro.ReadBuilder avroReadBuilder = - Avro.read(inputFile).project(readSchema).split(task.start(), task.length()); - if (reuseContainers) { - avroReadBuilder.reuseContainers(); - } - if (nameMapping != null) { - avroReadBuilder.withNameMapping(NameMappingParser.fromJson(nameMapping)); - } - - avroReadBuilder.createResolvingReader( - schema -> - PlannedDataReader.create( - schema, constantsMap(task, IdentityPartitionConverters::convertConstant))); - return applyResidualFiltering(avroReadBuilder.build(), task.residual(), readSchema); - } - - private CloseableIterable newParquetIterable( - InputFile inputFile, FileScanTask task, Schema readSchema) { - Parquet.ReadBuilder parquetReadBuilder = - Parquet.read(inputFile) - .project(readSchema) - .filter(task.residual()) - .caseSensitive(caseSensitive) - .split(task.start(), task.length()); - if (reuseContainers) { - parquetReadBuilder.reuseContainers(); - } - if (nameMapping != null) { - parquetReadBuilder.withNameMapping(NameMappingParser.fromJson(nameMapping)); - } - parquetReadBuilder.createReaderFunc( - fileSchema -> - GenericParquetReaders.buildReader( - readSchema, - fileSchema, - constantsMap(task, IdentityPartitionConverters::convertConstant))); - CloseableIterable parquetIterator = parquetReadBuilder.build(); - return applyResidualFiltering(parquetIterator, task.residual(), readSchema); - } - - private CloseableIterable newOrcIterable( - InputFile inputFile, FileScanTask task, Schema readSchema) { - Map idToConstant = - constantsMap(task, IdentityPartitionConverters::convertConstant); - Schema readSchemaWithoutConstantAndMetadataFields = - TypeUtil.selectNot( - readSchema, Sets.union(idToConstant.keySet(), MetadataColumns.metadataFieldIds())); - - // ORC does not support reuse containers yet - ORC.ReadBuilder orcReadBuilder = - ORC.read(inputFile) - .project(readSchemaWithoutConstantAndMetadataFields) - .filter(task.residual()) - .caseSensitive(caseSensitive) - .split(task.start(), task.length()); - orcReadBuilder.createReaderFunc( - fileSchema -> GenericOrcReader.buildReader(readSchema, fileSchema, idToConstant)); - - if (nameMapping != null) { - orcReadBuilder.withNameMapping(NameMappingParser.fromJson(nameMapping)); - } - CloseableIterable orcIterator = orcReadBuilder.build(); - return applyResidualFiltering(orcIterator, task.residual(), readSchema); - } - - private Map constantsMap( - FileScanTask task, BiFunction converter) { - PartitionSpec spec = task.spec(); - Set idColumns = spec.identitySourceIds(); - Schema partitionSchema = TypeUtil.select(expectedSchema, idColumns); - boolean projectsIdentityPartitionColumns = !partitionSchema.columns().isEmpty(); - if (projectsIdentityPartitionColumns) { - return PartitionUtil.constantsMap(task, converter); - } else { - return Collections.emptyMap(); - } - } - private static Schema readSchema( Configuration conf, Schema tableSchema, boolean caseSensitive) { Schema readSchema = InputFormatConfig.readSchema(conf); diff --git a/orc/src/main/java/org/apache/iceberg/orc/ORC.java b/orc/src/main/java/org/apache/iceberg/orc/ORC.java index 451c670fcd54..6eb53db28de6 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/ORC.java +++ b/orc/src/main/java/org/apache/iceberg/orc/ORC.java @@ -45,6 +45,7 @@ import java.util.Locale; import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.function.BiFunction; import java.util.function.Function; import java.util.stream.Collectors; @@ -52,6 +53,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.MetricsConfig; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -79,7 +81,10 @@ import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.ArrayUtil; import org.apache.iceberg.util.PropertyUtil; import org.apache.orc.CompressionKind; @@ -180,14 +185,13 @@ public WriteBuilder metricsConfig(MetricsConfig newMetricsConfig) { } // supposed to always be a private method used strictly by data and delete write builders - private WriteBuilder createContextFunc( - Function, Context> newCreateContextFunc) { + WriteBuilder createContextFunc(Function, Context> newCreateContextFunc) { this.createContextFunc = newCreateContextFunc; return this; } public FileAppender build() { - Preconditions.checkNotNull(schema, "Schema is required"); + // Preconditions.checkNotNull(schema, "Schema is required"); for (Map.Entry entry : config.entrySet()) { this.conf.set(entry.getKey(), entry.getValue()); @@ -219,7 +223,7 @@ public FileAppender build() { metricsConfig); } - private static class Context { + static class Context { private final long stripeSize; private final long blockSize; private final int vectorizedRowBatchSize; @@ -699,6 +703,7 @@ public static class ReadBuilder { private Function> readerFunc; private Function> batchedReaderFunc; private int recordsPerBatch = VectorizedRowBatch.DEFAULT_SIZE; + private Set constantFieldIds = ImmutableSet.of(); private ReadBuilder(InputFile file) { Preconditions.checkNotNull(file, "Input file cannot be null"); @@ -775,12 +780,20 @@ public ReadBuilder withNameMapping(NameMapping newNameMapping) { return this; } + ReadBuilder constantFieldIds(Set newConstantFieldIds) { + this.constantFieldIds = newConstantFieldIds; + return this; + } + public CloseableIterable build() { Preconditions.checkNotNull(schema, "Schema is required"); return new OrcIterable<>( file, conf, - schema, + // This is a behavioral change. Previously there were an error if metadata columns were + // present in the schema, now they are removed and the correct reader is created + TypeUtil.selectNot( + schema, Sets.union(constantFieldIds, MetadataColumns.metadataFieldIds())), nameMapping, start, length, diff --git a/orc/src/main/java/org/apache/iceberg/orc/ORCFormatModel.java b/orc/src/main/java/org/apache/iceberg/orc/ORCFormatModel.java new file mode 100644 index 000000000000..0bc719c10923 --- /dev/null +++ b/orc/src/main/java/org/apache/iceberg/orc/ORCFormatModel.java @@ -0,0 +1,316 @@ +/* + * 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.orc; + +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.function.Function; +import org.apache.iceberg.FileContent; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetricsConfig; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.orc.GenericOrcWriter; +import org.apache.iceberg.data.orc.GenericOrcWriters; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.formats.FormatModel; +import org.apache.iceberg.formats.ReadBuilder; +import org.apache.iceberg.formats.WriteBuilder; +import org.apache.iceberg.io.DeleteSchemaUtil; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.orc.TypeDescription; + +public class ORCFormatModel implements FormatModel { + private final Class type; + private final Class schemaType; + private final ReaderFunction readerFunction; + private final BatchReaderFunction batchReaderFunction; + private final WriterFunction writerFunction; + + private ORCFormatModel( + Class type, + Class schemaType, + ReaderFunction readerFunction, + BatchReaderFunction batchReaderFunction, + WriterFunction writerFunction) { + this.type = type; + this.schemaType = schemaType; + this.readerFunction = readerFunction; + this.batchReaderFunction = batchReaderFunction; + this.writerFunction = writerFunction; + } + + public ORCFormatModel( + Class type, + Class schemaType, + ReaderFunction readerFunction, + WriterFunction writerFunction) { + this(type, schemaType, readerFunction, null, writerFunction); + } + + public ORCFormatModel( + Class type, Class schemaType, BatchReaderFunction batchReaderFunction) { + this(type, schemaType, null, batchReaderFunction, null); + } + + public ORCFormatModel(Class type) { + this(type, null, null, null, null); + } + + @Override + public FileFormat format() { + return FileFormat.ORC; + } + + @Override + public Class type() { + return type; + } + + @Override + public Class schemaType() { + return schemaType; + } + + @Override + public WriteBuilder writeBuilder(EncryptedOutputFile outputFile) { + return new WriteBuilderWrapper<>(outputFile, writerFunction); + } + + @Override + public ReadBuilder readBuilder(InputFile inputFile) { + return new ReadBuilderWrapper<>(inputFile, readerFunction, batchReaderFunction); + } + + @FunctionalInterface + public interface ReaderFunction { + OrcRowReader read(Schema schema, TypeDescription messageType, Map idToConstant); + } + + @FunctionalInterface + public interface BatchReaderFunction { + OrcBatchReader read( + Schema schema, TypeDescription messageType, Map idToConstant); + } + + @FunctionalInterface + public interface WriterFunction { + OrcRowWriter write(Schema schema, TypeDescription messageType, E nativeSchema); + } + + private static class ReadBuilderWrapper implements ReadBuilder { + private final ORC.ReadBuilder internal; + private final ReaderFunction readerFunction; + private final BatchReaderFunction batchReaderFunction; + private boolean reuseContainers = false; + private Schema icebergSchema; + private Map idToConstant = ImmutableMap.of(); + + private ReadBuilderWrapper( + InputFile inputFile, + ReaderFunction readerFunction, + BatchReaderFunction batchReaderFunction) { + this.internal = ORC.read(inputFile); + this.readerFunction = readerFunction; + this.batchReaderFunction = batchReaderFunction; + } + + @Override + public ReadBuilder split(long newStart, long newLength) { + internal.split(newStart, newLength); + return this; + } + + @Override + public ReadBuilder project(Schema schema) { + this.icebergSchema = schema; + internal.project(schema); + return this; + } + + @Override + public ReadBuilder caseSensitive(boolean caseSensitive) { + internal.caseSensitive(caseSensitive); + return this; + } + + @Override + public ReadBuilder filter(Expression filter) { + internal.filter(filter); + return this; + } + + @Override + public ReadBuilder set(String key, String value) { + internal.config(key, value); + return this; + } + + @Override + public ReadBuilder reuseContainers() { + this.reuseContainers = true; + return this; + } + + @Override + public ReadBuilder recordsPerBatch(int numRowsPerBatch) { + internal.recordsPerBatch(numRowsPerBatch); + return this; + } + + @Override + public ReadBuilder idToConstant(Map newIdToConstant) { + internal.constantFieldIds(newIdToConstant.keySet()); + this.idToConstant = newIdToConstant; + return this; + } + + @Override + public ReadBuilder withNameMapping(NameMapping nameMapping) { + internal.withNameMapping(nameMapping); + return this; + } + + @Override + public org.apache.iceberg.io.CloseableIterable build() { + Preconditions.checkNotNull(reuseContainers, "Reuse containers is required for ORC read"); + if (readerFunction != null) { + return internal + .createReaderFunc( + typeDescription -> + readerFunction.read(icebergSchema, typeDescription, idToConstant)) + .build(); + } else if (batchReaderFunction != null) { + return internal + .createBatchedReaderFunc( + typeDescription -> + batchReaderFunction.read(icebergSchema, typeDescription, idToConstant)) + .build(); + } else { + throw new IllegalStateException("Either readerFunction or batchReaderFunction must be set"); + } + } + } + + private static class WriteBuilderWrapper implements WriteBuilder { + private final ORC.WriteBuilder internal; + private final WriterFunction writerFunction; + private S inputSchema; + private FileContent content; + + private WriteBuilderWrapper(EncryptedOutputFile outputFile, WriterFunction writerFunction) { + this.internal = ORC.write(outputFile); + this.writerFunction = writerFunction; + } + + @Override + public WriteBuilder schema(Schema schema) { + internal.schema(schema); + return this; + } + + @Override + public WriteBuilder inputSchema(S schema) { + this.inputSchema = schema; + return this; + } + + @Override + public WriteBuilder set(String property, String value) { + internal.set(property, value); + return this; + } + + @Override + public WriteBuilder setAll(Map properties) { + internal.setAll(properties); + return this; + } + + @Override + public WriteBuilder meta(String property, String value) { + internal.metadata(property, value); + return this; + } + + @Override + public WriteBuilder content(FileContent newContent) { + this.content = newContent; + return this; + } + + @Override + public WriteBuilder metricsConfig(MetricsConfig metricsConfig) { + internal.metricsConfig(metricsConfig); + return this; + } + + @Override + public WriteBuilder overwrite() { + internal.overwrite(); + return this; + } + + @Override + public WriteBuilder withFileEncryptionKey(ByteBuffer encryptionKey) { + // ORC doesn't support file encryption + throw new UnsupportedOperationException("ORC does not support file encryption keys"); + } + + @Override + public WriteBuilder withAADPrefix(ByteBuffer aadPrefix) { + // ORC doesn't support file encryption + throw new UnsupportedOperationException("ORC does not support AAD prefix"); + } + + @Override + public org.apache.iceberg.io.FileAppender build() { + switch (content) { + case DATA: + internal.createContextFunc(ORC.WriteBuilder.Context::dataContext); + internal.createWriterFunc( + (icebergSchema, typeDescription) -> + writerFunction.write(icebergSchema, typeDescription, inputSchema)); + break; + case EQUALITY_DELETES: + internal.createContextFunc(ORC.WriteBuilder.Context::deleteContext); + internal.createWriterFunc( + (icebergSchema, typeDescription) -> + writerFunction.write(icebergSchema, typeDescription, inputSchema)); + break; + case POSITION_DELETES: + internal.createContextFunc(ORC.WriteBuilder.Context::deleteContext); + internal.createWriterFunc( + (icebergSchema, typeDescription) -> + GenericOrcWriters.positionDelete( + GenericOrcWriter.buildWriter(icebergSchema, typeDescription), + Function.identity())); + internal.schema(DeleteSchemaUtil.pathPosSchema()); + break; + default: + throw new IllegalArgumentException("Unknown file content: " + content); + } + + return internal.build(); + } + } +} diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java index 2b2e460ee994..b67b5db69c9c 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -302,8 +302,7 @@ WriteBuilder withWriterVersion(WriterVersion version) { } // supposed to always be a private method used strictly by data and delete write builders - private WriteBuilder createContextFunc( - Function, Context> newCreateContextFunc) { + WriteBuilder createContextFunc(Function, Context> newCreateContextFunc) { this.createContextFunc = newCreateContextFunc; return this; } @@ -498,7 +497,7 @@ public FileAppender build() throws IOException { } } - private static class Context { + static class Context { private final int rowGroupSize; private final int pageSize; private final int pageRowLimit; @@ -1175,7 +1174,7 @@ public static class ReadBuilder implements InternalData.ReadBuilder { private Schema schema = null; private Expression filter = null; private ReadSupport readSupport = null; - private Function> batchedReaderFunc = null; + private BatchReaderFunction batchedReaderFunc = null; private ReaderFunction readerFunction = null; private boolean filterRecords = true; private boolean caseSensitive = true; @@ -1241,6 +1240,50 @@ public ReaderFunction withSchema(Schema expectedSchema) { } } + public interface BatchReaderFunction { + Function> apply(); + + default BatchReaderFunction withSchema(Schema schema) { + return this; + } + } + + private static class UnaryBatchReaderFunction implements BatchReaderFunction { + private final Function> readerFunc; + + UnaryBatchReaderFunction(Function> readerFunc) { + this.readerFunc = readerFunc; + } + + @Override + public Function> apply() { + return readerFunc; + } + } + + private static class BinaryBatchReaderFunction implements BatchReaderFunction { + private final BiFunction> readerFuncWithSchema; + private Schema schema; + + BinaryBatchReaderFunction( + BiFunction> readerFuncWithSchema) { + this.readerFuncWithSchema = readerFuncWithSchema; + } + + @Override + public Function> apply() { + Preconditions.checkArgument( + schema != null, "Schema must be set for 2-argument reader function"); + return messageType -> readerFuncWithSchema.apply(schema, messageType); + } + + @Override + public BinaryBatchReaderFunction withSchema(Schema expectedSchema) { + this.schema = expectedSchema; + return this; + } + } + private ReadBuilder(InputFile file) { this.file = file; } @@ -1315,14 +1358,27 @@ public ReadBuilder createReaderFunc( return this; } - public ReadBuilder createBatchedReaderFunc(Function> func) { + public ReadBuilder createBatchedReaderFunc( + Function> newReaderFunction) { + Preconditions.checkArgument( + this.batchedReaderFunc == null, + "Cannot set batched reader function: batched reader function already set"); + Preconditions.checkArgument( + this.readerFunction == null, + "Cannot set batched reader function: ReaderFunction already set"); + this.batchedReaderFunc = new UnaryBatchReaderFunction(newReaderFunction); + return this; + } + + public ReadBuilder createBatchedReaderFunc( + BiFunction> newReaderFunction) { Preconditions.checkArgument( this.batchedReaderFunc == null, "Cannot set batched reader function: batched reader function already set"); Preconditions.checkArgument( this.readerFunction == null, "Cannot set batched reader function: ReaderFunction already set"); - this.batchedReaderFunc = func; + this.batchedReaderFunc = new BinaryBatchReaderFunction(newReaderFunction); return this; } @@ -1442,11 +1498,13 @@ public CloseableIterable build() { } if (batchedReaderFunc != null) { + Function> readBuilder = + batchedReaderFunc.withSchema(schema).apply(); return new VectorizedParquetReader<>( file, schema, options, - batchedReaderFunc, + readBuilder, mapping, filter, reuseContainers, diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetFormatModel.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetFormatModel.java new file mode 100644 index 000000000000..5e90ee3d088d --- /dev/null +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetFormatModel.java @@ -0,0 +1,329 @@ +/* + * 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.parquet; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.function.Function; +import org.apache.iceberg.FileContent; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetricsConfig; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.formats.FormatModel; +import org.apache.iceberg.formats.ReadBuilder; +import org.apache.iceberg.formats.WriteBuilder; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.DeleteSchemaUtil; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.parquet.column.ParquetProperties; +import org.apache.parquet.schema.MessageType; + +public class ParquetFormatModel implements FormatModel { + public static final String WRITER_VERSION_KEY = "parquet.writer.version"; + + private final Class type; + private final Class schemaType; + private final ReaderFunction readerFunction; + private final BatchReaderFunction batchReaderFunction; + private final WriterFunction writerFunction; + + private ParquetFormatModel( + Class type, + Class schemaType, + ReaderFunction readerFunction, + BatchReaderFunction batchReaderFunction, + WriterFunction writerFunction) { + this.type = type; + this.schemaType = schemaType; + this.readerFunction = readerFunction; + this.batchReaderFunction = batchReaderFunction; + this.writerFunction = writerFunction; + } + + public ParquetFormatModel(Class type) { + this(type, null, null, null); + } + + public ParquetFormatModel( + Class type, + Class schemaType, + ReaderFunction readerFunction, + WriterFunction writerFunction) { + this(type, schemaType, readerFunction, null, writerFunction); + } + + public ParquetFormatModel( + Class type, Class schemaType, BatchReaderFunction batchReaderFunction) { + this(type, schemaType, null, batchReaderFunction, null); + } + + @Override + public FileFormat format() { + return FileFormat.PARQUET; + } + + @Override + public Class type() { + return type; + } + + @Override + public Class schemaType() { + return schemaType; + } + + @Override + public WriteBuilder writeBuilder(EncryptedOutputFile outputFile) { + return new WriteBuilderWrapper<>(outputFile, writerFunction); + } + + @Override + public ReadBuilder readBuilder(InputFile inputFile) { + return new ReadBuilderWrapper<>(inputFile, readerFunction, batchReaderFunction); + } + + @FunctionalInterface + public interface ReaderFunction { + ParquetValueReader read( + Schema schema, MessageType messageType, Map idToConstant); + } + + @FunctionalInterface + public interface BatchReaderFunction { + VectorizedReader read(Schema schema, MessageType messageType, Map idToConstant); + } + + @FunctionalInterface + public interface WriterFunction { + ParquetValueWriter write(Schema icebergSchema, MessageType messageType, S engineSchema); + } + + private static class WriteBuilderWrapper implements WriteBuilder { + private final Parquet.WriteBuilder internal; + private final WriterFunction writerFunction; + private S inputSchema; + private FileContent content; + + private WriteBuilderWrapper(EncryptedOutputFile outputFile, WriterFunction writerFunction) { + this.internal = Parquet.write(outputFile); + this.writerFunction = writerFunction; + } + + @Override + public WriteBuilder schema(Schema schema) { + internal.schema(schema); + return this; + } + + @Override + public WriteBuilder inputSchema(S schema) { + this.inputSchema = schema; + return this; + } + + @Override + public WriteBuilder set(String property, String value) { + if (WRITER_VERSION_KEY.equals(property)) { + internal.writerVersion(ParquetProperties.WriterVersion.valueOf(value)); + } + + internal.set(property, value); + return this; + } + + @Override + public WriteBuilder setAll(Map properties) { + internal.setAll(properties); + return this; + } + + @Override + public WriteBuilder meta(String property, String value) { + internal.meta(property, value); + return this; + } + + @Override + public WriteBuilder meta(Map properties) { + internal.meta(properties); + return this; + } + + @Override + public WriteBuilder content(FileContent newContent) { + this.content = newContent; + return this; + } + + @Override + public WriteBuilder metricsConfig(MetricsConfig metricsConfig) { + internal.metricsConfig(metricsConfig); + return this; + } + + @Override + public WriteBuilder overwrite() { + internal.overwrite(); + return this; + } + + @Override + public WriteBuilder withFileEncryptionKey(ByteBuffer encryptionKey) { + internal.withFileEncryptionKey(encryptionKey); + return this; + } + + @Override + public WriteBuilder withAADPrefix(ByteBuffer aadPrefix) { + internal.withAADPrefix(aadPrefix); + return this; + } + + @Override + public FileAppender build() throws IOException { + switch (content) { + case DATA: + internal.createContextFunc(Parquet.WriteBuilder.Context::dataContext); + internal.createWriterFunc( + (icebergSchema, messageType) -> + writerFunction.write(icebergSchema, messageType, inputSchema)); + break; + case EQUALITY_DELETES: + internal.createContextFunc(Parquet.WriteBuilder.Context::deleteContext); + internal.createWriterFunc( + (icebergSchema, messageType) -> + writerFunction.write(icebergSchema, messageType, inputSchema)); + break; + case POSITION_DELETES: + internal.createContextFunc(Parquet.WriteBuilder.Context::deleteContext); + internal.createWriterFunc( + (icebergSchema, messageType) -> + new ParquetValueWriters.PositionDeleteStructWriter( + (ParquetValueWriters.StructWriter) + GenericParquetWriter.create(icebergSchema, messageType), + Function.identity())); + internal.schema(DeleteSchemaUtil.pathPosSchema()); + break; + default: + throw new IllegalArgumentException("Unknown file content: " + content); + } + + return internal.build(); + } + } + + private static class ReadBuilderWrapper implements ReadBuilder { + private final Parquet.ReadBuilder internal; + private final ReaderFunction readerFunction; + private final BatchReaderFunction batchReaderFunction; + private final Map config = Maps.newHashMap(); + private Map idToConstant = ImmutableMap.of(); + + private ReadBuilderWrapper( + InputFile inputFile, + ReaderFunction readerFunction, + BatchReaderFunction batchReaderFunction) { + this.internal = Parquet.read(inputFile); + this.readerFunction = readerFunction; + this.batchReaderFunction = batchReaderFunction; + } + + @Override + public ReadBuilder split(long newStart, long newLength) { + internal.split(newStart, newLength); + return this; + } + + @Override + public ReadBuilder project(Schema schema) { + internal.project(schema); + return this; + } + + @Override + public ReadBuilder caseSensitive(boolean caseSensitive) { + internal.caseSensitive(caseSensitive); + return this; + } + + @Override + public ReadBuilder filter(Expression filter) { + internal.filter(filter); + return this; + } + + @Override + public ReadBuilder set(String key, String value) { + this.config.put(key, value); + internal.set(key, value); + return this; + } + + @Override + public ReadBuilder reuseContainers() { + internal.reuseContainers(); + return this; + } + + @Override + public ReadBuilder recordsPerBatch(int numRowsPerBatch) { + internal.recordsPerBatch(numRowsPerBatch); + return this; + } + + @Override + public ReadBuilder idToConstant(Map newIdToConstant) { + this.idToConstant = newIdToConstant; + return this; + } + + @Override + public ReadBuilder withNameMapping(NameMapping nameMapping) { + internal.withNameMapping(nameMapping); + return this; + } + + @Override + public CloseableIterable build() { + if (readerFunction != null) { + return internal + .createReaderFunc( + (icebergSchema, messageType) -> + readerFunction.read(icebergSchema, messageType, idToConstant)) + .build(); + } else if (batchReaderFunction != null) { + return internal + .createBatchedReaderFunc( + (icebergSchema, messageType) -> + batchReaderFunction.read(icebergSchema, messageType, idToConstant)) + .build(); + } else { + throw new IllegalStateException("Either readerFunction or batchReaderFunction must be set"); + } + } + } +} diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java b/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java index fc10a57ec0e0..6d4e64603771 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java @@ -46,7 +46,7 @@ public class VectorizedParquetReader extends CloseableGroup implements Closea private final Function> batchReaderFunc; private final Expression filter; private final boolean reuseContainers; - private final boolean caseSensitive; + private final boolean filterCaseSensitive; private final int batchSize; private final NameMapping nameMapping; @@ -58,7 +58,7 @@ public VectorizedParquetReader( NameMapping nameMapping, Expression filter, boolean reuseContainers, - boolean caseSensitive, + boolean filterCaseSensitive, int maxRecordsPerBatch) { this.input = input; this.expectedSchema = expectedSchema; @@ -67,7 +67,7 @@ public VectorizedParquetReader( // replace alwaysTrue with null to avoid extra work evaluating a trivial filter this.filter = filter == Expressions.alwaysTrue() ? null : filter; this.reuseContainers = reuseContainers; - this.caseSensitive = caseSensitive; + this.filterCaseSensitive = filterCaseSensitive; this.batchSize = maxRecordsPerBatch; this.nameMapping = nameMapping; } @@ -86,7 +86,7 @@ private ReadConf init() { batchReaderFunc, nameMapping, reuseContainers, - caseSensitive, + filterCaseSensitive, batchSize); this.conf = readConf.copy(); return readConf; diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java index 58850ec7c9f4..3132929fb330 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java @@ -218,7 +218,7 @@ public void testTwoLevelList() throws IOException { writer.close(); GenericData.Record recordRead = - Iterables.getOnlyElement( + Iterables.getOnlyElement( Parquet.read(Files.localInput(testFile)).project(schema).callInit().build()); assertThat(recordRead.get("arraybytes")).isEqualTo(expectedByteList); diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java index d6a13bcd515d..220244f9197f 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java @@ -52,13 +52,14 @@ import org.apache.iceberg.avro.Avro; import org.apache.iceberg.data.Record; import org.apache.iceberg.data.avro.DataWriter; -import org.apache.iceberg.data.avro.PlannedDataReader; -import org.apache.iceberg.data.orc.GenericOrcReader; import org.apache.iceberg.data.orc.GenericOrcWriter; -import org.apache.iceberg.data.parquet.GenericParquetReaders; import org.apache.iceberg.data.parquet.GenericParquetWriter; import org.apache.iceberg.deletes.PositionDeleteWriter; +import org.apache.iceberg.encryption.EncryptedFiles; import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.formats.FormatModelRegistry; +import org.apache.iceberg.formats.PositionDeleteWriteBuilder; +import org.apache.iceberg.formats.ReadBuilder; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.DeleteSchemaUtil; import org.apache.iceberg.io.FileIO; @@ -681,6 +682,13 @@ public CloseableIterable reader( return positionDeletesReader(inputFile, format, spec); } + @Override + public PositionDeleteWriter writer( + OutputFile outputFile, FileFormat format, PartitionSpec spec, StructLike partition) + throws IOException { + return positionDeletesWriter(outputFile, format, spec, partition, null); + } + @Override public PositionDeleteWriter writer( OutputFile outputFile, @@ -720,31 +728,8 @@ private ForeachFunction rewritePositionDelete( private static CloseableIterable positionDeletesReader( InputFile inputFile, FileFormat format, PartitionSpec spec) { Schema deleteSchema = DeleteSchemaUtil.posDeleteReadSchema(spec.schema()); - switch (format) { - case AVRO: - return Avro.read(inputFile) - .project(deleteSchema) - .reuseContainers() - .createReaderFunc(fileSchema -> PlannedDataReader.create(deleteSchema)) - .build(); - - case PARQUET: - return Parquet.read(inputFile) - .project(deleteSchema) - .reuseContainers() - .createReaderFunc( - fileSchema -> GenericParquetReaders.buildReader(deleteSchema, fileSchema)) - .build(); - - case ORC: - return ORC.read(inputFile) - .project(deleteSchema) - .createReaderFunc(fileSchema -> GenericOrcReader.buildReader(deleteSchema, fileSchema)) - .build(); - - default: - throw new UnsupportedOperationException("Unsupported file format: " + format); - } + ReadBuilder builder = FormatModelRegistry.readBuilder(format, Record.class, inputFile); + return builder.project(deleteSchema).reuseContainers().build(); } private static PositionDeleteWriter positionDeletesWriter( @@ -754,30 +739,37 @@ private static PositionDeleteWriter positionDeletesWriter( StructLike partition, Schema rowSchema) throws IOException { - switch (format) { - case AVRO: - return Avro.writeDeletes(outputFile) - .createWriterFunc(DataWriter::create) - .withPartition(partition) - .rowSchema(rowSchema) - .withSpec(spec) - .buildPositionWriter(); - case PARQUET: - return Parquet.writeDeletes(outputFile) - .createWriterFunc(GenericParquetWriter::create) - .withPartition(partition) - .rowSchema(rowSchema) - .withSpec(spec) - .buildPositionWriter(); - case ORC: - return ORC.writeDeletes(outputFile) - .createWriterFunc(GenericOrcWriter::buildWriter) - .withPartition(partition) - .rowSchema(rowSchema) - .withSpec(spec) - .buildPositionWriter(); - default: - throw new UnsupportedOperationException("Unsupported file format: " + format); + if (rowSchema == null) { + PositionDeleteWriteBuilder builder = + FormatModelRegistry.positionDeleteWriteBuilder( + format, EncryptedFiles.plainAsEncryptedOutput(outputFile)); + return builder.partition(partition).spec(spec).build(); + } else { + switch (format) { + case AVRO: + return Avro.writeDeletes(outputFile) + .createWriterFunc(DataWriter::create) + .withPartition(partition) + .rowSchema(rowSchema) + .withSpec(spec) + .buildPositionWriter(); + case PARQUET: + return Parquet.writeDeletes(outputFile) + .createWriterFunc(GenericParquetWriter::create) + .withPartition(partition) + .rowSchema(rowSchema) + .withSpec(spec) + .buildPositionWriter(); + case ORC: + return ORC.writeDeletes(outputFile) + .createWriterFunc(GenericOrcWriter::buildWriter) + .withPartition(partition) + .rowSchema(rowSchema) + .withSpec(spec) + .buildPositionWriter(); + default: + throw new UnsupportedOperationException("Unsupported file format: " + format); + } } } diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java index 8e25e81a05b2..aa1b496cf302 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java @@ -30,6 +30,8 @@ import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.spark.SparkUtil; import org.apache.parquet.schema.MessageType; +import org.apache.spark.sql.vectorized.ColumnVector; +import org.apache.spark.sql.vectorized.ColumnarBatch; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -75,9 +77,9 @@ public static ColumnarBatchReader buildReader( return buildReader(expectedSchema, fileSchema, idToConstant, ArrowAllocation.rootAllocator()); } - public static CometColumnarBatchReader buildCometReader( + public static VectorizedReader buildCometReader( Schema expectedSchema, MessageType fileSchema, Map idToConstant) { - return (CometColumnarBatchReader) + return (VectorizedReader) TypeWithSchemaVisitor.visit( expectedSchema.asStruct(), fileSchema, @@ -88,6 +90,13 @@ public static CometColumnarBatchReader buildCometReader( readers -> new CometColumnarBatchReader(readers, expectedSchema))); } + /** A subclass of ColumnarBatch to identify Comet readers. */ + public static class CometColumnarBatch extends ColumnarBatch { + public CometColumnarBatch(ColumnVector[] columns) { + super(columns); + } + } + // enables unsafe memory access to avoid costly checks to see if index is within bounds // as long as it is not configured explicitly (see BoundsChecking in Arrow) private static void enableUnsafeMemoryAccess() { diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java index ff30f29aeae6..110ce355a223 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import java.util.Map; -import java.util.Set; import javax.annotation.Nonnull; import org.apache.iceberg.FileFormat; import org.apache.iceberg.MetadataColumns; @@ -29,21 +28,18 @@ import org.apache.iceberg.Table; import org.apache.iceberg.data.DeleteFilter; import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.formats.FormatModelRegistry; +import org.apache.iceberg.formats.ReadBuilder; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.InputFile; -import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.OrcBatchReadConf; import org.apache.iceberg.spark.ParquetBatchReadConf; import org.apache.iceberg.spark.ParquetReaderType; import org.apache.iceberg.spark.data.vectorized.ColumnVectorWithFilter; import org.apache.iceberg.spark.data.vectorized.ColumnarBatchUtil; import org.apache.iceberg.spark.data.vectorized.UpdatableDeletedColumnVector; -import org.apache.iceberg.spark.data.vectorized.VectorizedSparkOrcReaders; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; -import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.Pair; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.vectorized.ColumnVector; @@ -68,6 +64,7 @@ abstract class BaseBatchReader extends BaseReader newBatchIterable( InputFile inputFile, FileFormat format, @@ -76,79 +73,38 @@ protected CloseableIterable newBatchIterable( Expression residual, Map idToConstant, @Nonnull SparkDeleteFilter deleteFilter) { - CloseableIterable iterable; - switch (format) { - case PARQUET: - iterable = - newParquetIterable( - inputFile, start, length, residual, idToConstant, deleteFilter.requiredSchema()); - break; - case ORC: - iterable = newOrcIterable(inputFile, start, length, residual, idToConstant); - break; - default: - throw new UnsupportedOperationException( - "Format: " + format + " not supported for batched reads"); + ReadBuilder readBuilder; + if (parquetConf != null) { + readBuilder = + parquetConf.readerType() == ParquetReaderType.COMET + ? FormatModelRegistry.readBuilder( + format, VectorizedSparkParquetReaders.CometColumnarBatch.class, inputFile) + : FormatModelRegistry.readBuilder(format, ColumnarBatch.class, inputFile); + + readBuilder = readBuilder.recordsPerBatch(parquetConf.batchSize()); + } else { + readBuilder = FormatModelRegistry.readBuilder(format, ColumnarBatch.class, inputFile); + if (orcConf != null) { + readBuilder = readBuilder.recordsPerBatch(orcConf.batchSize()); + } } - return CloseableIterable.transform(iterable, new BatchDeleteFilter(deleteFilter)::filterBatch); - } + CloseableIterable iterable = + readBuilder + .project(deleteFilter.requiredSchema()) + .idToConstant(idToConstant) + .split(start, length) + .caseSensitive(caseSensitive()) + .filter(residual) + // Spark eagerly consumes the batches. So the underlying memory allocated could be + // reused without worrying about subsequent reads clobbering over each other. This + // improves read performance as every batch read doesn't have to pay the cost of + // allocating memory. + .reuseContainers() + .withNameMapping(nameMapping()) + .build(); - private CloseableIterable newParquetIterable( - InputFile inputFile, - long start, - long length, - Expression residual, - Map idToConstant, - Schema requiredSchema) { - return Parquet.read(inputFile) - .project(requiredSchema) - .split(start, length) - .createBatchedReaderFunc( - fileSchema -> { - if (parquetConf.readerType() == ParquetReaderType.COMET) { - return VectorizedSparkParquetReaders.buildCometReader( - requiredSchema, fileSchema, idToConstant); - } else { - return VectorizedSparkParquetReaders.buildReader( - requiredSchema, fileSchema, idToConstant); - } - }) - .recordsPerBatch(parquetConf.batchSize()) - .filter(residual) - .caseSensitive(caseSensitive()) - // Spark eagerly consumes the batches. So the underlying memory allocated could be reused - // without worrying about subsequent reads clobbering over each other. This improves - // read performance as every batch read doesn't have to pay the cost of allocating memory. - .reuseContainers() - .withNameMapping(nameMapping()) - .build(); - } - - private CloseableIterable newOrcIterable( - InputFile inputFile, - long start, - long length, - Expression residual, - Map idToConstant) { - Set constantFieldIds = idToConstant.keySet(); - Set metadataFieldIds = MetadataColumns.metadataFieldIds(); - Sets.SetView constantAndMetadataFieldIds = - Sets.union(constantFieldIds, metadataFieldIds); - Schema schemaWithoutConstantAndMetadataFields = - TypeUtil.selectNot(expectedSchema(), constantAndMetadataFieldIds); - - return ORC.read(inputFile) - .project(schemaWithoutConstantAndMetadataFields) - .split(start, length) - .createBatchedReaderFunc( - fileSchema -> - VectorizedSparkOrcReaders.buildReader(expectedSchema(), fileSchema, idToConstant)) - .recordsPerBatch(orcConf.batchSize()) - .filter(residual) - .caseSensitive(caseSensitive()) - .withNameMapping(nameMapping()) - .build(); + return CloseableIterable.transform(iterable, new BatchDeleteFilter(deleteFilter)::filterBatch); } @VisibleForTesting diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseRowReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseRowReader.java index c12931e786b1..53d44e760afe 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseRowReader.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseRowReader.java @@ -20,22 +20,15 @@ import java.util.Map; import org.apache.iceberg.FileFormat; -import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.ScanTask; import org.apache.iceberg.ScanTaskGroup; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; -import org.apache.iceberg.avro.Avro; import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.formats.FormatModelRegistry; +import org.apache.iceberg.formats.ReadBuilder; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.InputFile; -import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.parquet.Parquet; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.spark.data.SparkOrcReader; -import org.apache.iceberg.spark.data.SparkParquetReaders; -import org.apache.iceberg.spark.data.SparkPlannedAvroReader; -import org.apache.iceberg.types.TypeUtil; import org.apache.spark.sql.catalyst.InternalRow; abstract class BaseRowReader extends BaseReader { @@ -58,69 +51,15 @@ protected CloseableIterable newIterable( Expression residual, Schema projection, Map idToConstant) { - switch (format) { - case PARQUET: - return newParquetIterable(file, start, length, residual, projection, idToConstant); - - case AVRO: - return newAvroIterable(file, start, length, projection, idToConstant); - - case ORC: - return newOrcIterable(file, start, length, residual, projection, idToConstant); - - default: - throw new UnsupportedOperationException("Cannot read unknown format: " + format); - } - } - - private CloseableIterable newAvroIterable( - InputFile file, long start, long length, Schema projection, Map idToConstant) { - return Avro.read(file) - .reuseContainers() + ReadBuilder reader = + FormatModelRegistry.readBuilder(format, InternalRow.class, file); + return reader .project(projection) - .split(start, length) - .createResolvingReader(schema -> SparkPlannedAvroReader.create(schema, idToConstant)) - .withNameMapping(nameMapping()) - .build(); - } - - private CloseableIterable newParquetIterable( - InputFile file, - long start, - long length, - Expression residual, - Schema readSchema, - Map idToConstant) { - return Parquet.read(file) + .idToConstant(idToConstant) .reuseContainers() .split(start, length) - .project(readSchema) - .createReaderFunc( - fileSchema -> SparkParquetReaders.buildReader(readSchema, fileSchema, idToConstant)) - .filter(residual) .caseSensitive(caseSensitive()) - .withNameMapping(nameMapping()) - .build(); - } - - private CloseableIterable newOrcIterable( - InputFile file, - long start, - long length, - Expression residual, - Schema readSchema, - Map idToConstant) { - Schema readSchemaWithoutConstantAndMetadataFields = - TypeUtil.selectNot( - readSchema, Sets.union(idToConstant.keySet(), MetadataColumns.metadataFieldIds())); - - return ORC.read(file) - .project(readSchemaWithoutConstantAndMetadataFields) - .split(start, length) - .createReaderFunc( - readOrcSchema -> new SparkOrcReader(readSchema, readOrcSchema, idToConstant)) .filter(residual) - .caseSensitive(caseSensitive()) .withNameMapping(nameMapping()) .build(); } diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkFileWriterFactory.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkFileWriterFactory.java index a93db17e4a0f..1f8cee27a818 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkFileWriterFactory.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkFileWriterFactory.java @@ -23,13 +23,20 @@ import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT; import static org.apache.iceberg.TableProperties.DELETE_DEFAULT_FILE_FORMAT; +import java.io.IOException; +import java.io.UncheckedIOException; import java.util.Map; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetricsConfig; +import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.data.BaseFileWriterFactory; +import org.apache.iceberg.data.RegistryBasedFileWriterFactory; +import org.apache.iceberg.deletes.PositionDeleteWriter; +import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.io.DeleteSchemaUtil; import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; @@ -40,14 +47,18 @@ import org.apache.iceberg.spark.data.SparkOrcWriter; import org.apache.iceberg.spark.data.SparkParquetWriters; import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; import org.apache.spark.unsafe.types.UTF8String; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -class SparkFileWriterFactory extends BaseFileWriterFactory { - private StructType dataSparkType; - private StructType equalityDeleteSparkType; +class SparkFileWriterFactory extends RegistryBasedFileWriterFactory { + private static final Logger LOG = LoggerFactory.getLogger(SparkFileWriterFactory.class); private StructType positionDeleteSparkType; + private boolean useDeprecatedPositionDeleteWriter = false; + private final Schema positionDeleteRowSchema; + private final Table table; + private final FileFormat format; private final Map writeProperties; /** @@ -75,18 +86,26 @@ class SparkFileWriterFactory extends BaseFileWriterFactory { super( table, dataFileFormat, + InternalRow.class, dataSchema, dataSortOrder, deleteFileFormat, equalityFieldIds, equalityDeleteRowSchema, equalityDeleteSortOrder, - positionDeleteRowSchema); + writeProperties, + calculateSparkType(dataSparkType, dataSchema), + calculateSparkType(equalityDeleteSparkType, equalityDeleteRowSchema)); - this.dataSparkType = dataSparkType; - this.equalityDeleteSparkType = equalityDeleteSparkType; - this.positionDeleteSparkType = positionDeleteSparkType; + this.table = table; + this.format = dataFileFormat; this.writeProperties = writeProperties != null ? writeProperties : ImmutableMap.of(); + this.positionDeleteRowSchema = positionDeleteRowSchema; + this.positionDeleteSparkType = positionDeleteSparkType; + this.useDeprecatedPositionDeleteWriter = + positionDeleteRowSchema != null + || (positionDeleteSparkType != null + && positionDeleteSparkType.getFieldIndex(DELETE_FILE_ROW_FIELD_NAME).isDefined()); } SparkFileWriterFactory( @@ -105,119 +124,106 @@ class SparkFileWriterFactory extends BaseFileWriterFactory { super( table, dataFileFormat, + InternalRow.class, dataSchema, dataSortOrder, deleteFileFormat, equalityFieldIds, equalityDeleteRowSchema, equalityDeleteSortOrder, - ImmutableMap.of()); + writeProperties, + calculateSparkType(dataSparkType, dataSchema), + calculateSparkType(equalityDeleteSparkType, equalityDeleteRowSchema)); - this.dataSparkType = dataSparkType; - this.equalityDeleteSparkType = equalityDeleteSparkType; - this.positionDeleteSparkType = null; + this.table = table; + this.format = dataFileFormat; this.writeProperties = writeProperties != null ? writeProperties : ImmutableMap.of(); + this.positionDeleteRowSchema = null; + this.useDeprecatedPositionDeleteWriter = false; } static Builder builderFor(Table table) { return new Builder(table); } - @Override - protected void configureDataWrite(Avro.DataWriteBuilder builder) { - builder.createWriterFunc(ignored -> new SparkAvroWriter(dataSparkType())); - builder.setAll(writeProperties); - } - - @Override - protected void configureEqualityDelete(Avro.DeleteWriteBuilder builder) { - builder.createWriterFunc(ignored -> new SparkAvroWriter(equalityDeleteSparkType())); - builder.setAll(writeProperties); - } - - @Override - protected void configurePositionDelete(Avro.DeleteWriteBuilder builder) { - boolean withRow = - positionDeleteSparkType().getFieldIndex(DELETE_FILE_ROW_FIELD_NAME).isDefined(); - if (withRow) { - // SparkAvroWriter accepts just the Spark type of the row ignoring the path and pos - StructField rowField = positionDeleteSparkType().apply(DELETE_FILE_ROW_FIELD_NAME); - StructType positionDeleteRowSparkType = (StructType) rowField.dataType(); - builder.createWriterFunc(ignored -> new SparkAvroWriter(positionDeleteRowSparkType)); - } - - builder.setAll(writeProperties); - } - - @Override - protected void configureDataWrite(Parquet.DataWriteBuilder builder) { - builder.createWriterFunc(msgType -> SparkParquetWriters.buildWriter(dataSparkType(), msgType)); - builder.setAll(writeProperties); - } - - @Override - protected void configureEqualityDelete(Parquet.DeleteWriteBuilder builder) { - builder.createWriterFunc( - msgType -> SparkParquetWriters.buildWriter(equalityDeleteSparkType(), msgType)); - builder.setAll(writeProperties); - } - - @Override - protected void configurePositionDelete(Parquet.DeleteWriteBuilder builder) { - builder.createWriterFunc( - msgType -> SparkParquetWriters.buildWriter(positionDeleteSparkType(), msgType)); - builder.transformPaths(path -> UTF8String.fromString(path.toString())); - builder.setAll(writeProperties); - } - - @Override - protected void configureDataWrite(ORC.DataWriteBuilder builder) { - builder.createWriterFunc(SparkOrcWriter::new); - builder.setAll(writeProperties); - } - - @Override - protected void configureEqualityDelete(ORC.DeleteWriteBuilder builder) { - builder.createWriterFunc(SparkOrcWriter::new); - builder.setAll(writeProperties); - } - - @Override - protected void configurePositionDelete(ORC.DeleteWriteBuilder builder) { - builder.createWriterFunc(SparkOrcWriter::new); - builder.transformPaths(path -> UTF8String.fromString(path.toString())); - builder.setAll(writeProperties); - } - - private StructType dataSparkType() { - if (dataSparkType == null) { - Preconditions.checkNotNull(dataSchema(), "Data schema must not be null"); - this.dataSparkType = SparkSchemaUtil.convert(dataSchema()); - } - - return dataSparkType; - } - - private StructType equalityDeleteSparkType() { - if (equalityDeleteSparkType == null) { - Preconditions.checkNotNull( - equalityDeleteRowSchema(), "Equality delete schema must not be null"); - this.equalityDeleteSparkType = SparkSchemaUtil.convert(equalityDeleteRowSchema()); - } - - return equalityDeleteSparkType; - } - private StructType positionDeleteSparkType() { if (positionDeleteSparkType == null) { // wrap the optional row schema into the position delete schema containing path and position - Schema positionDeleteSchema = DeleteSchemaUtil.posDeleteSchema(positionDeleteRowSchema()); + Schema positionDeleteSchema = DeleteSchemaUtil.posDeleteSchema(positionDeleteRowSchema); this.positionDeleteSparkType = SparkSchemaUtil.convert(positionDeleteSchema); } return positionDeleteSparkType; } + @Override + public PositionDeleteWriter newPositionDeleteWriter( + EncryptedOutputFile file, PartitionSpec spec, StructLike partition) { + if (!useDeprecatedPositionDeleteWriter) { + return super.newPositionDeleteWriter(file, spec, partition); + } else { + LOG.info( + "Deprecated feature used. Position delete row schema is used to create the position delete writer."); + MetricsConfig metricsConfig = + table != null + ? MetricsConfig.forPositionDelete(table) + : MetricsConfig.fromProperties(ImmutableMap.of()); + + try { + switch (format) { + case AVRO: + StructType positionDeleteRowSparkType = + (StructType) positionDeleteSparkType().apply(DELETE_FILE_ROW_FIELD_NAME).dataType(); + + return Avro.writeDeletes(file) + .createWriterFunc(ignored -> new SparkAvroWriter(positionDeleteRowSparkType)) + .withPartition(partition) + .overwrite() + .rowSchema(positionDeleteRowSchema) + .withSpec(spec) + .withKeyMetadata(file.keyMetadata()) + .setAll(writeProperties) + .metricsConfig(metricsConfig) + .buildPositionWriter(); + + case ORC: + return ORC.writeDeletes(file) + .createWriterFunc(SparkOrcWriter::new) + .transformPaths(path -> UTF8String.fromString(path.toString())) + .withPartition(partition) + .overwrite() + .rowSchema(positionDeleteRowSchema) + .withSpec(spec) + .withKeyMetadata(file.keyMetadata()) + .setAll(writeProperties) + .metricsConfig(metricsConfig) + .buildPositionWriter(); + + case PARQUET: + return Parquet.writeDeletes(file) + .createWriterFunc( + msgType -> SparkParquetWriters.buildWriter(positionDeleteSparkType(), msgType)) + .transformPaths(path -> UTF8String.fromString(path.toString())) + .withPartition(partition) + .overwrite() + .metricsConfig(metricsConfig) + .rowSchema(positionDeleteRowSchema) + .withSpec(spec) + .withKeyMetadata(file.keyMetadata()) + .setAll(writeProperties) + .metricsConfig(metricsConfig) + .buildPositionWriter(); + + default: + throw new UnsupportedOperationException( + "Cannot write pos-deletes for unsupported file format: " + format); + } + } catch (IOException e) { + throw new UncheckedIOException("Failed to create new position delete writer", e); + } + } + } + static class Builder { private final Table table; private FileFormat dataFileFormat; @@ -340,4 +346,14 @@ SparkFileWriterFactory build() { writeProperties); } } + + private static StructType calculateSparkType(StructType sparkType, Schema schema) { + if (sparkType != null) { + return sparkType; + } else if (schema != null) { + return SparkSchemaUtil.convert(schema); + } else { + return null; + } + } } diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java new file mode 100644 index 000000000000..63229a89d0ed --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import org.apache.iceberg.avro.AvroFormatModel; +import org.apache.iceberg.formats.FormatModelRegistry; +import org.apache.iceberg.orc.ORCFormatModel; +import org.apache.iceberg.parquet.ParquetFormatModel; +import org.apache.iceberg.spark.data.SparkAvroWriter; +import org.apache.iceberg.spark.data.SparkOrcReader; +import org.apache.iceberg.spark.data.SparkOrcWriter; +import org.apache.iceberg.spark.data.SparkParquetReaders; +import org.apache.iceberg.spark.data.SparkParquetWriters; +import org.apache.iceberg.spark.data.SparkPlannedAvroReader; +import org.apache.iceberg.spark.data.vectorized.VectorizedSparkOrcReaders; +import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.vectorized.ColumnarBatch; + +public class SparkFormatModels { + public static void register() { + FormatModelRegistry.register( + new AvroFormatModel<>( + InternalRow.class, + StructType.class, + SparkPlannedAvroReader::create, + (avroSchema, inputSchema) -> new SparkAvroWriter(inputSchema))); + + FormatModelRegistry.register( + new ParquetFormatModel<>( + InternalRow.class, + StructType.class, + SparkParquetReaders::buildReader, + (icebergSchema, messageType, inputType) -> + SparkParquetWriters.buildWriter(inputType, messageType))); + + FormatModelRegistry.register( + new ParquetFormatModel<>( + ColumnarBatch.class, StructType.class, VectorizedSparkParquetReaders::buildReader)); + + FormatModelRegistry.register( + new ParquetFormatModel<>( + VectorizedSparkParquetReaders.CometColumnarBatch.class, + StructType.class, + VectorizedSparkParquetReaders::buildCometReader)); + + FormatModelRegistry.register( + new ORCFormatModel<>( + InternalRow.class, + StructType.class, + SparkOrcReader::new, + (schema, typeDescription, unused) -> new SparkOrcWriter(schema, typeDescription))); + + FormatModelRegistry.register( + new ORCFormatModel<>( + ColumnarBatch.class, StructType.class, VectorizedSparkOrcReaders::buildReader)); + } + + private SparkFormatModels() {} +}