diff --git a/build.gradle b/build.gradle index 7df7c9f079b3..8a18558cdf37 100644 --- a/build.gradle +++ b/build.gradle @@ -311,6 +311,7 @@ project(':iceberg-flink') { testCompile project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') testCompile project(path: ':iceberg-api', configuration: 'testArtifacts') testCompile project(path: ':iceberg-data', configuration: 'testArtifacts') + testCompile project(path: ':iceberg-core', configuration: 'testArtifacts') // By default, hive-exec is a fat/uber jar and it exports a guava library // that's really old. We use the core classifier to be able to override our guava diff --git a/core/src/main/java/org/apache/iceberg/ContentFileWriter.java b/core/src/main/java/org/apache/iceberg/ContentFileWriter.java new file mode 100644 index 000000000000..9cc8a1cab37a --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/ContentFileWriter.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg; + +import java.io.Closeable; +import java.util.Iterator; + +public interface ContentFileWriter extends Closeable { + + void write(R record); + + default void writeAll(Iterator values) { + while (values.hasNext()) { + write(values.next()); + } + } + + default void writeAll(Iterable values) { + writeAll(values.iterator()); + } + + /** + * Returns the length of this file. + */ + long length(); + + /** + * Return a {@link ContentFile} which is either {@link DeleteFile} or {@link DataFile}. + */ + T toContentFile(); +} diff --git a/core/src/main/java/org/apache/iceberg/ContentFileWriterFactory.java b/core/src/main/java/org/apache/iceberg/ContentFileWriterFactory.java new file mode 100644 index 000000000000..f7528a0ebcf6 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/ContentFileWriterFactory.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg; + +import org.apache.iceberg.encryption.EncryptedOutputFile; + +/** + * Factory to create a new {@link ContentFileWriter} to write INSERT or DELETE records. + * + * @param Content file type, it's either {@link DataFile} or {@link DeleteFile}. + * @param data type of the rows to write. + */ +public interface ContentFileWriterFactory { + + /** + * Create a new {@link ContentFileWriter} + * + * @param partitionKey an partition key to indicate which partition the rows will be written. Null if it's + * unpartitioned. + * @param outputFile an OutputFile used to create an output stream. + * @param fileFormat File format. + * @return a newly created {@link ContentFileWriter} + */ + ContentFileWriter createWriter(PartitionKey partitionKey, EncryptedOutputFile outputFile, + FileFormat fileFormat); +} diff --git a/core/src/main/java/org/apache/iceberg/DataFileWriter.java b/core/src/main/java/org/apache/iceberg/DataFileWriter.java new file mode 100644 index 000000000000..4f2438482f54 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/DataFileWriter.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg; + +import java.io.IOException; +import java.nio.ByteBuffer; +import org.apache.iceberg.encryption.EncryptionKeyMetadata; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +public class DataFileWriter implements ContentFileWriter { + private final FileAppender appender; + private final FileFormat format; + private final String location; + private final PartitionKey partitionKey; + private final PartitionSpec spec; + private final ByteBuffer keyMetadata; + private DataFile dataFile = null; + + public DataFileWriter(FileAppender appender, FileFormat format, + String location, PartitionKey partitionKey, PartitionSpec spec, + EncryptionKeyMetadata keyMetadata) { + this.appender = appender; + this.format = format; + this.location = location; + this.partitionKey = partitionKey; // set null if unpartitioned. + this.spec = spec; + this.keyMetadata = keyMetadata != null ? keyMetadata.buffer() : null; + } + + @Override + public void write(T row) { + appender.add(row); + } + + @Override + public long length() { + return appender.length(); + } + + @Override + public DataFile toContentFile() { + Preconditions.checkState(dataFile != null, "Cannot create data file from unclosed writer"); + return dataFile; + } + + @Override + public void close() throws IOException { + if (dataFile == null) { + appender.close(); + this.dataFile = DataFiles.builder(spec) + .withEncryptionKeyMetadata(keyMetadata) + .withFormat(format) + .withPath(location) + .withFileSizeInBytes(appender.length()) + .withPartition(partitionKey) // set null if unpartitioned + .withMetrics(appender.metrics()) + .withSplitOffsets(appender.splitOffsets()) + .build(); + } + } +} 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 584c28070c9b..198b479fa053 100644 --- a/core/src/main/java/org/apache/iceberg/avro/Avro.java +++ b/core/src/main/java/org/apache/iceberg/avro/Avro.java @@ -38,11 +38,11 @@ import org.apache.avro.io.Encoder; import org.apache.avro.specific.SpecificData; import org.apache.iceberg.FileFormat; -import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.SchemaParser; import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; +import org.apache.iceberg.deletes.DeletesUtil; import org.apache.iceberg.deletes.EqualityDeleteWriter; import org.apache.iceberg.deletes.PositionDelete; import org.apache.iceberg.deletes.PositionDeleteWriter; @@ -53,7 +53,6 @@ import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types.NestedField; import org.apache.iceberg.util.ArrayUtil; import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION; @@ -301,21 +300,12 @@ public PositionDeleteWriter buildPositionWriter() throws IOException { meta("delete-type", "position"); if (rowSchema != null && createWriterFunc != null) { - // the appender uses the row schema wrapped with position fields - appenderBuilder.schema(new org.apache.iceberg.Schema( - MetadataColumns.DELETE_FILE_PATH, - MetadataColumns.DELETE_FILE_POS, - NestedField.optional( - MetadataColumns.DELETE_FILE_ROW_FIELD_ID, "row", rowSchema.asStruct(), - MetadataColumns.DELETE_FILE_ROW_DOC))); + appenderBuilder.schema(DeletesUtil.pathPosSchema(rowSchema)); appenderBuilder.createWriterFunc( avroSchema -> new PositionAndRowDatumWriter<>(createWriterFunc.apply(avroSchema))); - } else { - appenderBuilder.schema(new org.apache.iceberg.Schema( - MetadataColumns.DELETE_FILE_PATH, - MetadataColumns.DELETE_FILE_POS)); + appenderBuilder.schema(DeletesUtil.pathPosSchema()); appenderBuilder.createWriterFunc(ignored -> new PositionDatumWriter()); } diff --git a/core/src/main/java/org/apache/iceberg/deletes/DeletesUtil.java b/core/src/main/java/org/apache/iceberg/deletes/DeletesUtil.java new file mode 100644 index 000000000000..b1b69b5ff832 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/deletes/DeletesUtil.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.deletes; + +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Schema; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Types; + +public class DeletesUtil { + + private DeletesUtil() { + } + + public static Schema pathPosSchema() { + return new Schema( + MetadataColumns.DELETE_FILE_PATH, + MetadataColumns.DELETE_FILE_POS); + } + + public static Schema pathPosSchema(Schema rowSchema) { + Preconditions.checkNotNull(rowSchema, "Row schema should not be null when constructing the pos-delete schema."); + + // the appender uses the row schema wrapped with position fields + return new Schema( + MetadataColumns.DELETE_FILE_PATH, + MetadataColumns.DELETE_FILE_POS, + Types.NestedField.required( + MetadataColumns.DELETE_FILE_ROW_FIELD_ID, "row", rowSchema.asStruct(), + MetadataColumns.DELETE_FILE_ROW_DOC)); + } + + public static Schema posDeleteSchema(Schema rowSchema) { + return rowSchema == null ? pathPosSchema() : pathPosSchema(rowSchema); + } +} diff --git a/core/src/main/java/org/apache/iceberg/deletes/EqualityDeleteWriter.java b/core/src/main/java/org/apache/iceberg/deletes/EqualityDeleteWriter.java index 1838ef749b08..b339881c83c9 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/EqualityDeleteWriter.java +++ b/core/src/main/java/org/apache/iceberg/deletes/EqualityDeleteWriter.java @@ -19,9 +19,9 @@ package org.apache.iceberg.deletes; -import java.io.Closeable; import java.io.IOException; import java.nio.ByteBuffer; +import org.apache.iceberg.ContentFileWriter; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.FileMetadata; @@ -31,7 +31,7 @@ import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -public class EqualityDeleteWriter implements Closeable { +public class EqualityDeleteWriter implements ContentFileWriter { private final FileAppender appender; private final FileFormat format; private final String location; @@ -53,11 +53,13 @@ public EqualityDeleteWriter(FileAppender appender, FileFormat format, String this.equalityFieldIds = equalityFieldIds; } - public void deleteAll(Iterable rows) { - appender.addAll(rows); + @Override + public long length() { + return appender.length(); } - public void delete(T row) { + @Override + public void write(T row) { appender.add(row); } @@ -77,7 +79,8 @@ public void close() throws IOException { } } - public DeleteFile toDeleteFile() { + @Override + public DeleteFile toContentFile() { Preconditions.checkState(deleteFile != null, "Cannot create delete file from unclosed writer"); return deleteFile; } diff --git a/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteWriter.java b/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteWriter.java index 7bacdc07ba83..227d85f987c1 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteWriter.java +++ b/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteWriter.java @@ -19,10 +19,10 @@ package org.apache.iceberg.deletes; -import java.io.Closeable; import java.io.IOException; import java.nio.ByteBuffer; import java.util.Set; +import org.apache.iceberg.ContentFileWriter; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.FileMetadata; @@ -33,7 +33,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.util.CharSequenceSet; -public class PositionDeleteWriter implements Closeable { +public class PositionDeleteWriter implements ContentFileWriter> { private final FileAppender appender; private final FileFormat format; private final String location; @@ -85,7 +85,18 @@ public Set referencedDataFiles() { return pathSet; } - public DeleteFile toDeleteFile() { + @Override + public void write(PositionDelete record) { + delete(record.path(), record.pos(), record.row()); + } + + @Override + public long length() { + return appender.length(); + } + + @Override + public DeleteFile toContentFile() { Preconditions.checkState(deleteFile != null, "Cannot create delete file from unclosed writer"); return deleteFile; } diff --git a/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java b/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java new file mode 100644 index 000000000000..129c8dcf9371 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java @@ -0,0 +1,207 @@ +/* + * 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.io; + +import java.io.IOException; +import java.util.List; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.StructLikeMap; + +public abstract class BaseDeltaWriter implements DeltaWriter { + private final RollingContentFileWriter dataWriter; + private final RollingEqDeleteWriter eqDeleteWriter; + private final RollingPosDeleteWriter posDeleteWriter; + + private final PositionDelete positionDelete = new PositionDelete<>(); + private final StructLikeMap insertedRowMap; + + protected BaseDeltaWriter(RollingContentFileWriter dataWriter, + RollingPosDeleteWriter posDeleteWriter, + RollingEqDeleteWriter eqDeleteWriter, + Schema tableSchema, + List equalityFieldIds) { + + Preconditions.checkNotNull(dataWriter, "Data writer should always not be null."); + + if (posDeleteWriter == null) { + // Only accept INSERT records. + Preconditions.checkArgument(eqDeleteWriter == null, + "Could not accept equality deletes when position delete writer is null."); + } + + if (posDeleteWriter != null && eqDeleteWriter == null) { + // Only accept INSERT records and POS-DELETE records. + Preconditions.checkArgument(tableSchema == null, "Table schema is only required for equality delete writer."); + Preconditions.checkArgument(equalityFieldIds == null, + "Equality field id list is only required for equality delete writer."); + } + + if (eqDeleteWriter != null) { + // Accept INSERT records, POS-DELETE records and EQUALITY-DELETE records. + Preconditions.checkNotNull(posDeleteWriter, + "Position delete writer shouldn't be null when writing equality deletions."); + Preconditions.checkNotNull(tableSchema, "Iceberg table schema shouldn't be null"); + Preconditions.checkNotNull(equalityFieldIds, "Equality field ids shouldn't be null"); + + Schema deleteSchema = TypeUtil.select(tableSchema, Sets.newHashSet(equalityFieldIds)); + this.insertedRowMap = StructLikeMap.create(deleteSchema.asStruct()); + } else { + this.insertedRowMap = null; + } + + this.dataWriter = dataWriter; + this.eqDeleteWriter = eqDeleteWriter; + this.posDeleteWriter = posDeleteWriter; + } + + protected abstract StructLike asKey(T row); + + protected abstract StructLike asCopiedKey(T row); + + @Override + public void writeRow(T row) { + if (allowEqDelete()) { + RowOffset rowOffset = RowOffset.create(dataWriter.currentPath(), dataWriter.currentRows()); + + // Copy the key to avoid messing up the insertedRowMap. + StructLike key = asCopiedKey(row); + RowOffset previous = insertedRowMap.putIfAbsent(key, rowOffset); + ValidationException.check(previous == null, "Detected duplicate insert for %s", key); + } + + dataWriter.write(row); + } + + @Override + public void writeEqualityDelete(T equalityDelete) { + Preconditions.checkState(allowEqDelete(), "Could not accept equality deletion."); + + StructLike key = asKey(equalityDelete); + RowOffset existing = insertedRowMap.get(key); + + if (existing == null) { + // Delete the row which have been written by other completed delta writer. + eqDeleteWriter.write(equalityDelete); + } else { + // Delete the rows which was written in current delta writer. If the position delete row schema is null, then the + // writer won't write the records even if we provide the rows here. + posDeleteWriter.write(positionDelete.set(existing.path, existing.rowId, equalityDelete)); + // Remove the records from insertedRowMap because we've already deleted it by writing position delete file. + insertedRowMap.remove(key); + } + } + + @Override + public void writePosDelete(CharSequence path, long offset, T row) { + Preconditions.checkState(allowPosDelete(), "Could not accept position deletion."); + + posDeleteWriter.write(positionDelete.set(path, offset, row)); + } + + @Override + public void abort() { + if (dataWriter != null) { + dataWriter.abort(); + } + + if (eqDeleteWriter != null) { + eqDeleteWriter.abort(); + insertedRowMap.clear(); + } + + if (posDeleteWriter != null) { + posDeleteWriter.abort(); + } + } + + @Override + public WriterResult complete() { + WriterResult.Builder builder = WriterResult.builder(); + + if (dataWriter != null) { + builder.add(dataWriter.complete()); + } + + if (eqDeleteWriter != null) { + builder.add(eqDeleteWriter.complete()); + insertedRowMap.clear(); + } + + if (posDeleteWriter != null) { + builder.add(posDeleteWriter.complete()); + } + + return builder.build(); + } + + @Override + public void close() throws IOException { + if (dataWriter != null) { + dataWriter.close(); + } + + if (eqDeleteWriter != null) { + eqDeleteWriter.close(); + insertedRowMap.clear(); + } + + if (posDeleteWriter != null) { + posDeleteWriter.close(); + } + } + + private boolean allowEqDelete() { + return eqDeleteWriter != null && posDeleteWriter != null; + } + + private boolean allowPosDelete() { + return posDeleteWriter != null; + } + + private static class RowOffset { + private final CharSequence path; + private final long rowId; + + private RowOffset(CharSequence path, long rowId) { + this.path = path; + this.rowId = rowId; + } + + private static RowOffset create(CharSequence path, long pos) { + return new RowOffset(path, pos); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("path", path) + .add("row_id", rowId) + .toString(); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java b/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java deleted file mode 100644 index ca16318d8bdc..000000000000 --- a/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java +++ /dev/null @@ -1,146 +0,0 @@ -/* - * 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.io; - -import java.io.Closeable; -import java.io.IOException; -import java.util.List; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Metrics; -import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.encryption.EncryptedOutputFile; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.util.Tasks; - -public abstract class BaseTaskWriter implements TaskWriter { - private final List completedFiles = Lists.newArrayList(); - private final PartitionSpec spec; - private final FileFormat format; - private final FileAppenderFactory appenderFactory; - private final OutputFileFactory fileFactory; - private final FileIO io; - private final long targetFileSize; - - protected BaseTaskWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, FileIO io, long targetFileSize) { - this.spec = spec; - this.format = format; - this.appenderFactory = appenderFactory; - this.fileFactory = fileFactory; - this.io = io; - this.targetFileSize = targetFileSize; - } - - @Override - public void abort() throws IOException { - close(); - - // clean up files created by this writer - Tasks.foreach(completedFiles) - .throwFailureWhenFinished() - .noRetry() - .run(file -> io.deleteFile(file.path().toString())); - } - - @Override - public DataFile[] complete() throws IOException { - close(); - - return completedFiles.toArray(new DataFile[0]); - } - - protected class RollingFileWriter implements Closeable { - private static final int ROWS_DIVISOR = 1000; - private final PartitionKey partitionKey; - - private EncryptedOutputFile currentFile = null; - private FileAppender currentAppender = null; - private long currentRows = 0; - - public RollingFileWriter(PartitionKey partitionKey) { - this.partitionKey = partitionKey; - openCurrent(); - } - - public void add(T record) throws IOException { - this.currentAppender.add(record); - this.currentRows++; - - if (shouldRollToNewFile()) { - closeCurrent(); - openCurrent(); - } - } - - private void openCurrent() { - if (partitionKey == null) { - // unpartitioned - currentFile = fileFactory.newOutputFile(); - } else { - // partitioned - currentFile = fileFactory.newOutputFile(partitionKey); - } - currentAppender = appenderFactory.newAppender(currentFile.encryptingOutputFile(), format); - currentRows = 0; - } - - private boolean shouldRollToNewFile() { - // TODO: ORC file now not support target file size before closed - return !format.equals(FileFormat.ORC) && - currentRows % ROWS_DIVISOR == 0 && currentAppender.length() >= targetFileSize; - } - - private void closeCurrent() throws IOException { - if (currentAppender != null) { - currentAppender.close(); - // metrics are only valid after the appender is closed - Metrics metrics = currentAppender.metrics(); - long fileSizeInBytes = currentAppender.length(); - List splitOffsets = currentAppender.splitOffsets(); - this.currentAppender = null; - - if (metrics.recordCount() == 0L) { - io.deleteFile(currentFile.encryptingOutputFile()); - } else { - DataFile dataFile = DataFiles.builder(spec) - .withEncryptionKeyMetadata(currentFile.keyMetadata()) - .withPath(currentFile.encryptingOutputFile().location()) - .withFileSizeInBytes(fileSizeInBytes) - .withPartition(spec.fields().size() == 0 ? null : partitionKey) // set null if unpartitioned - .withMetrics(metrics) - .withSplitOffsets(splitOffsets) - .build(); - completedFiles.add(dataFile); - } - - this.currentFile = null; - this.currentRows = 0; - } - } - - @Override - public void close() throws IOException { - closeCurrent(); - } - } -} diff --git a/core/src/main/java/org/apache/iceberg/io/DataFileWriterFactory.java b/core/src/main/java/org/apache/iceberg/io/DataFileWriterFactory.java new file mode 100644 index 000000000000..8839eee52d10 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/io/DataFileWriterFactory.java @@ -0,0 +1,51 @@ +/* + * 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.io; + +import org.apache.iceberg.ContentFileWriter; +import org.apache.iceberg.ContentFileWriterFactory; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFileWriter; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.encryption.EncryptedOutputFile; + +/** + * TODO: This factory would be removed once we've replaced this with {@link DeltaWriterFactory} in the compute engines. + */ +public class DataFileWriterFactory implements ContentFileWriterFactory { + private final FileAppenderFactory appenderFactory; + private final PartitionSpec spec; + + public DataFileWriterFactory(FileAppenderFactory appenderFactory, PartitionSpec spec) { + this.appenderFactory = appenderFactory; + this.spec = spec; + } + + @Override + public ContentFileWriter createWriter(PartitionKey partitionKey, + EncryptedOutputFile outputFile, + FileFormat fileFormat) { + FileAppender appender = appenderFactory.newAppender(outputFile.encryptingOutputFile(), fileFormat); + return new DataFileWriter<>(appender, fileFormat, outputFile.encryptingOutputFile().location(), partitionKey, + spec, outputFile.keyMetadata()); + } +} diff --git a/core/src/main/java/org/apache/iceberg/io/DeltaWriter.java b/core/src/main/java/org/apache/iceberg/io/DeltaWriter.java new file mode 100644 index 000000000000..57df75c53c47 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/io/DeltaWriter.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.io; + +import java.io.Closeable; + +/** + * The write interface could accept INSERT, POS-DELETION, EQUALITY-DELETION. It usually write those operations + * in a given partition or bucket. + */ +public interface DeltaWriter extends Closeable { + + /** + * Write the insert record. + */ + void writeRow(T row); + + /** + * Write the equality delete record. + */ + void writeEqualityDelete(T equalityDelete); + + /** + * Write the deletion with file path and position into underlying system. + */ + default void writePosDelete(CharSequence path, long offset) { + writePosDelete(path, offset, null); + } + + /** + * Write the deletion with file path, position and original row into underlying system. + */ + void writePosDelete(CharSequence path, long offset, T row); + + /** + * Abort the writer to clean all generated files. + */ + void abort(); + + /** + * Close the writer and get all the completed data files and delete files. + */ + WriterResult complete(); +} diff --git a/core/src/main/java/org/apache/iceberg/io/DeltaWriterFactory.java b/core/src/main/java/org/apache/iceberg/io/DeltaWriterFactory.java new file mode 100644 index 000000000000..422ed368982f --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/io/DeltaWriterFactory.java @@ -0,0 +1,152 @@ +/* + * 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.io; + +import java.util.List; +import org.apache.iceberg.ContentFileWriterFactory; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.Schema; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; + +/** + * Factory to create {@link DeltaWriter}, which have few dependencies factories to create different kinds of writers. + */ +public interface DeltaWriterFactory { + + /** + * Create a factory to initialize the {@link DeltaWriter}. + */ + DeltaWriter createDeltaWriter(PartitionKey partitionKey, Context context); + + /** + * Create a factory to initialize the {@link FileAppender}. + */ + FileAppenderFactory createFileAppenderFactory(); + + /** + * Create a factory to initialize the {@link org.apache.iceberg.DataFileWriter}. + */ + ContentFileWriterFactory createDataFileWriterFactory(); + + /** + * Create a factory to initialize the {@link org.apache.iceberg.deletes.EqualityDeleteWriter}. + */ + ContentFileWriterFactory createEqualityDeleteWriterFactory(List equalityFieldIds, + Schema rowSchema); + + /** + * Create a factory to initialize the {@link org.apache.iceberg.deletes.PositionDeleteWriter}. + */ + ContentFileWriterFactory> createPosDeleteWriterFactory(Schema rowSchema); + + class Context { + private final boolean allowPosDelete; + private final boolean allowEqualityDelete; + private final List equalityFieldIds; + private final Schema posDeleteRowSchema; + private final Schema eqDeleteRowSchema; + + private Context(boolean allowPosDelete, boolean allowEqualityDelete, + List equalityFieldIds, Schema eqDeleteRowSchema, + Schema posDeleteRowSchema) { + this.allowPosDelete = allowPosDelete; + this.allowEqualityDelete = allowEqualityDelete; + this.equalityFieldIds = equalityFieldIds; + this.eqDeleteRowSchema = eqDeleteRowSchema; + this.posDeleteRowSchema = posDeleteRowSchema; + } + + public boolean allowPosDelete() { + return allowPosDelete; + } + + public boolean allowEqualityDelete() { + return allowEqualityDelete; + } + + public List equalityFieldIds() { + return equalityFieldIds; + } + + public Schema posDeleteRowSchema() { + return posDeleteRowSchema; + } + + public Schema eqDeleteRowSchema() { + return eqDeleteRowSchema; + } + + public static Builder builder() { + return new Builder(); + } + + public static class Builder { + private boolean allowPosDelete = false; + private boolean allowEqualityDelete = false; + private List equalityFieldIds = ImmutableList.of(); + private Schema eqDeleteRowSchema; + private Schema posDeleteRowSchema; + + public Builder allowPosDelete(boolean enable) { + this.allowPosDelete = enable; + return this; + } + + public Builder allowEqualityDelete(boolean enable) { + this.allowEqualityDelete = enable; + return this; + } + + public Builder equalityFieldIds(List newEqualityFieldIds) { + this.equalityFieldIds = ImmutableList.copyOf(newEqualityFieldIds); + return this; + } + + public Builder eqDeleteRowSchema(Schema newRowSchema) { + this.eqDeleteRowSchema = newRowSchema; + return this; + } + + public Builder posDeleteRowSchema(Schema newRowSchema) { + this.posDeleteRowSchema = newRowSchema; + return this; + } + + public Context build() { + if (allowEqualityDelete) { + Preconditions.checkNotNull(equalityFieldIds, "Equality field ids shouldn't be null for equality deletes"); + Preconditions.checkNotNull(eqDeleteRowSchema, "Row schema shouldn't be null for equality deletes"); + + for (Integer fieldId : equalityFieldIds) { + Preconditions.checkNotNull(eqDeleteRowSchema.findField(fieldId), + "Unknown field with id %s in provided equality row schema: %s", fieldId, eqDeleteRowSchema); + } + } + + return new Context(allowPosDelete, allowEqualityDelete, equalityFieldIds, eqDeleteRowSchema, + posDeleteRowSchema); + } + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java b/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java index 7b5fa90e4502..317de6559018 100644 --- a/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java @@ -21,25 +21,38 @@ import java.io.IOException; import java.util.Set; +import org.apache.iceberg.ContentFileWriterFactory; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.Tasks; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public abstract class PartitionedWriter extends BaseTaskWriter { +public abstract class PartitionedWriter implements TaskWriter { private static final Logger LOG = LoggerFactory.getLogger(PartitionedWriter.class); + private final FileFormat format; + private final OutputFileFactory fileFactory; + private final FileIO io; + private final long targetFileSize; + private final ContentFileWriterFactory writerFactory; + private final WriterResult.Builder resultBuilder; + private final Set completedPartitions = Sets.newHashSet(); private PartitionKey currentKey = null; - private RollingFileWriter currentWriter = null; + private RollingContentFileWriter currentWriter = null; - public PartitionedWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, FileIO io, long targetFileSize) { - super(spec, format, appenderFactory, fileFactory, io, targetFileSize); + public PartitionedWriter(FileFormat format, OutputFileFactory fileFactory, FileIO io, long targetFileSize, + ContentFileWriterFactory writerFactory) { + this.format = format; + this.fileFactory = fileFactory; + this.io = io; + this.targetFileSize = targetFileSize; + this.writerFactory = writerFactory; + this.resultBuilder = WriterResult.builder(); } /** @@ -58,7 +71,7 @@ public void write(T row) throws IOException { if (!key.equals(currentKey)) { if (currentKey != null) { // if the key is null, there was no previous current key and current writer. - currentWriter.close(); + resultBuilder.add(currentWriter.complete()); completedPartitions.add(currentKey); } @@ -70,10 +83,11 @@ public void write(T row) throws IOException { } currentKey = key.copy(); - currentWriter = new RollingFileWriter(currentKey); + currentWriter = new RollingContentFileWriter<>(currentKey, format, + fileFactory, io, targetFileSize, writerFactory); } - currentWriter.add(row); + currentWriter.write(row); } @Override @@ -82,4 +96,31 @@ public void close() throws IOException { currentWriter.close(); } } + + @Override + public void abort() throws IOException { + close(); + + if (currentWriter != null) { + currentWriter.abort(); + currentWriter = null; + } + + Tasks.foreach(resultBuilder.build().contentFiles()) + .throwFailureWhenFinished() + .noRetry() + .run(file -> io.deleteFile(file.path().toString())); + } + + @Override + public WriterResult complete() throws IOException { + close(); + + if (currentWriter != null) { + resultBuilder.add(currentWriter.complete()); + currentWriter = null; + } + + return resultBuilder.build(); + } } diff --git a/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java b/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java new file mode 100644 index 000000000000..09ef4ac3ab57 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java @@ -0,0 +1,162 @@ +/* + * 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.io; + +import java.io.Closeable; +import java.io.IOException; +import java.io.UncheckedIOException; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.ContentFileWriter; +import org.apache.iceberg.ContentFileWriterFactory; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.util.Tasks; + +public class RollingContentFileWriter implements Closeable { + + private static final int ROWS_DIVISOR = 1000; + private final PartitionKey partitionKey; + private final FileFormat format; + private final OutputFileFactory fileFactory; + private final FileIO io; + private final long targetFileSize; + private final ContentFileWriterFactory writerFactory; + + private final WriterResult.Builder resultBuilder; + + private EncryptedOutputFile currentFile = null; + private ContentFileWriter currentFileWriter = null; + private long currentRows = 0; + + public RollingContentFileWriter(PartitionKey partitionKey, FileFormat format, + OutputFileFactory fileFactory, FileIO io, long targetFileSize, + ContentFileWriterFactory writerFactory) { + this.partitionKey = partitionKey; + this.format = format; + this.fileFactory = fileFactory; + this.io = io; + this.targetFileSize = targetFileSize; + this.writerFactory = writerFactory; + + this.resultBuilder = WriterResult.builder(); + + openCurrent(); + } + + CharSequence currentPath() { + return currentFile.encryptingOutputFile().location(); + } + + long currentRows() { + return currentRows; + } + + public void write(T record) { + this.currentFileWriter.write(record); + this.currentRows++; + + if (shouldRollToNewFile()) { + closeCurrent(); + openCurrent(); + } + } + + public void abort() { + closeCurrent(); + + WriterResult result = resultBuilder.build(); + + Tasks.foreach(result.contentFiles()) + .throwFailureWhenFinished() + .noRetry() + .run(file -> io.deleteFile(file.path().toString())); + } + + public WriterResult complete() { + closeCurrent(); + + return resultBuilder.build(); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("partitionKey", partitionKey) + .add("format", format) + .toString(); + } + + private void openCurrent() { + if (partitionKey == null) { + // unpartitioned + currentFile = fileFactory.newOutputFile(); + } else { + // partitioned + currentFile = fileFactory.newOutputFile(partitionKey); + } + currentFileWriter = writerFactory.createWriter(partitionKey, currentFile, format); + currentRows = 0; + } + + private boolean shouldRollToNewFile() { + // TODO: ORC file now not support target file size before closed + return !format.equals(FileFormat.ORC) && + currentRows % ROWS_DIVISOR == 0 && currentFileWriter.length() >= targetFileSize; + } + + protected void beforeClose(ContentFileWriter writer) { + } + + private void closeCurrent() { + if (currentFileWriter != null) { + + // Call the hook before close the current writer. + beforeClose(currentFileWriter); + + try { + currentFileWriter.close(); + } catch (IOException e) { + throw new UncheckedIOException("Failed to close the current file writer: ", e); + } + + ContentFileT contentFile = currentFileWriter.toContentFile(); + this.currentFileWriter = null; + + if (currentRows == 0L) { + io.deleteFile(currentFile.encryptingOutputFile()); + } else if (contentFile instanceof ContentFile) { + resultBuilder.add((ContentFile) contentFile); + } else { + throw new RuntimeException(String.format( + "The newly generated content file must be DataFile or DeleteFile: %s", contentFile)); + } + + this.currentFile = null; + this.currentRows = 0; + } + } + + @Override + public void close() throws IOException { + closeCurrent(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/io/RollingEqDeleteWriter.java b/core/src/main/java/org/apache/iceberg/io/RollingEqDeleteWriter.java new file mode 100644 index 000000000000..0247dafea9c7 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/io/RollingEqDeleteWriter.java @@ -0,0 +1,33 @@ +/* + * 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.io; + +import org.apache.iceberg.ContentFileWriterFactory; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionKey; + +public class RollingEqDeleteWriter extends RollingContentFileWriter { + public RollingEqDeleteWriter(PartitionKey partitionKey, FileFormat format, + OutputFileFactory fileFactory, FileIO io, long targetFileSize, + ContentFileWriterFactory writerFactory) { + super(partitionKey, format, fileFactory, io, targetFileSize, writerFactory); + } +} diff --git a/core/src/main/java/org/apache/iceberg/io/RollingPosDeleteWriter.java b/core/src/main/java/org/apache/iceberg/io/RollingPosDeleteWriter.java new file mode 100644 index 000000000000..8c9244f58a95 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/io/RollingPosDeleteWriter.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.io; + +import java.util.Set; +import org.apache.iceberg.ContentFileWriter; +import org.apache.iceberg.ContentFileWriterFactory; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.deletes.PositionDeleteWriter; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; + +public class RollingPosDeleteWriter extends RollingContentFileWriter> { + private final Set referencedDataFiles; + + public RollingPosDeleteWriter(PartitionKey partitionKey, FileFormat format, + OutputFileFactory fileFactory, FileIO io, long targetFileSize, + ContentFileWriterFactory> writerFactory) { + super(partitionKey, format, fileFactory, io, targetFileSize, writerFactory); + + this.referencedDataFiles = Sets.newHashSet(); + } + + @Override + protected void beforeClose(ContentFileWriter> writer) { + PositionDeleteWriter positionDeleteWriter = (PositionDeleteWriter) writer; + referencedDataFiles.addAll(positionDeleteWriter.referencedDataFiles()); + } + + public Set referencedDataFiles() { + return referencedDataFiles; + } +} diff --git a/core/src/main/java/org/apache/iceberg/io/TaskWriter.java b/core/src/main/java/org/apache/iceberg/io/TaskWriter.java index 806e37de1bee..98fd75e96cd2 100644 --- a/core/src/main/java/org/apache/iceberg/io/TaskWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/TaskWriter.java @@ -21,7 +21,6 @@ import java.io.Closeable; import java.io.IOException; -import org.apache.iceberg.DataFile; /** * The writer interface could accept records and provide the generated data files. @@ -47,5 +46,5 @@ public interface TaskWriter extends Closeable { * * @return the completed data files of this task writer. */ - DataFile[] complete() throws IOException; + WriterResult complete() throws IOException; } diff --git a/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java b/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java index 37f3db49aef4..3f81678e48bd 100644 --- a/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java @@ -20,22 +20,32 @@ package org.apache.iceberg.io; import java.io.IOException; +import org.apache.iceberg.ContentFileWriterFactory; import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -public class UnpartitionedWriter extends BaseTaskWriter { +public class UnpartitionedWriter implements TaskWriter { - private final RollingFileWriter currentWriter; + private final RollingContentFileWriter currentWriter; - public UnpartitionedWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, FileIO io, long targetFileSize) { - super(spec, format, appenderFactory, fileFactory, io, targetFileSize); - currentWriter = new RollingFileWriter(null); + public UnpartitionedWriter(FileFormat format, OutputFileFactory fileFactory, FileIO io, long targetFileSize, + ContentFileWriterFactory writerFactory) { + currentWriter = new RollingContentFileWriter<>(null, format, fileFactory, io, + targetFileSize, writerFactory); } @Override public void write(T record) throws IOException { - currentWriter.add(record); + currentWriter.write(record); + } + + @Override + public void abort() { + currentWriter.abort(); + } + + @Override + public WriterResult complete() { + return currentWriter.complete(); } @Override diff --git a/core/src/main/java/org/apache/iceberg/io/WriterResult.java b/core/src/main/java/org/apache/iceberg/io/WriterResult.java new file mode 100644 index 000000000000..9beb2682c568 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/io/WriterResult.java @@ -0,0 +1,134 @@ +/* + * 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.io; + +import java.io.Serializable; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +public class WriterResult implements Serializable { + private static final DataFile[] EMPTY_DATA_FILES = new DataFile[0]; + private static final DeleteFile[] EMPTY_DELETE_FILES = new DeleteFile[0]; + + private DataFile[] dataFiles; + private DeleteFile[] deleteFiles; + + private WriterResult(DataFile[] dataFiles, DeleteFile[] deleteFiles) { + this.dataFiles = dataFiles; + this.deleteFiles = deleteFiles; + } + + private WriterResult(List dataFiles, List deleteFiles) { + this.dataFiles = dataFiles.toArray(new DataFile[0]); + this.deleteFiles = deleteFiles.toArray(new DeleteFile[0]); + } + + public DataFile[] dataFiles() { + return dataFiles; + } + + public DeleteFile[] deleteFiles() { + return deleteFiles; + } + + public Iterable> contentFiles() { + return () -> new Iterator>() { + private int currentIndex = 0; + + @Override + public boolean hasNext() { + return currentIndex < dataFiles.length + deleteFiles.length; + } + + @Override + public ContentFile next() { + ContentFile contentFile; + if (currentIndex < dataFiles.length) { + contentFile = dataFiles[currentIndex]; + } else if (currentIndex < dataFiles.length + deleteFiles.length) { + contentFile = deleteFiles[currentIndex - dataFiles.length]; + } else { + throw new NoSuchElementException(); + } + currentIndex += 1; + return contentFile; + } + }; + } + + public static WriterResult create(DataFile dataFile) { + return new WriterResult(new DataFile[] {dataFile}, EMPTY_DELETE_FILES); + } + + public static WriterResult create(DeleteFile deleteFile) { + return new WriterResult(EMPTY_DATA_FILES, new DeleteFile[] {deleteFile}); + } + + public static Builder builder() { + return new Builder(); + } + + public static class Builder { + private final List dataFiles; + private final List deleteFiles; + + private Builder() { + this.dataFiles = Lists.newArrayList(); + this.deleteFiles = Lists.newArrayList(); + } + + public void add(WriterResult result) { + addAll(result.contentFiles()); + } + + public void addAll(Iterable> iterable) { + for (ContentFile contentFile : iterable) { + add(contentFile); + } + } + + public void add(ContentFile contentFile) { + Preconditions.checkNotNull(contentFile, "Content file shouldn't be null."); + switch (contentFile.content()) { + case DATA: + this.dataFiles.add((DataFile) contentFile); + break; + + case EQUALITY_DELETES: + case POSITION_DELETES: + this.deleteFiles.add((DeleteFile) contentFile); + break; + + default: + throw new UnsupportedOperationException("Unknown content file: " + contentFile.content()); + } + } + + public WriterResult build() { + return new WriterResult(dataFiles, deleteFiles); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/util/StructLikeMap.java b/core/src/main/java/org/apache/iceberg/util/StructLikeMap.java new file mode 100644 index 000000000000..5b045e66d0f6 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/util/StructLikeMap.java @@ -0,0 +1,128 @@ +/* + * 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.util; + +import java.util.AbstractMap; +import java.util.Collection; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; + +public class StructLikeMap extends AbstractMap implements Map { + + public static StructLikeMap create(Types.StructType type) { + return new StructLikeMap<>(type); + } + + private final Types.StructType type; + private final Map wrapperMap; + private final ThreadLocal wrappers; + + private StructLikeMap(Types.StructType type) { + this.type = type; + this.wrapperMap = Maps.newHashMap(); + this.wrappers = ThreadLocal.withInitial(() -> StructLikeWrapper.forType(type)); + } + + @Override + public int size() { + return wrapperMap.size(); + } + + @Override + public boolean isEmpty() { + return wrapperMap.isEmpty(); + } + + @Override + public boolean containsKey(Object key) { + if (key instanceof StructLike) { + StructLikeWrapper wrapper = wrappers.get(); + boolean result = wrapperMap.containsKey(wrapper.set((StructLike) key)); + wrapper.set(null); + return result; + } + return false; + } + + @Override + public boolean containsValue(Object value) { + throw new UnsupportedOperationException(); + } + + @Override + public T get(Object key) { + if (key instanceof StructLike) { + StructLikeWrapper wrapper = wrappers.get(); + T value = wrapperMap.get(wrapper.set((StructLike) key)); + wrapper.set(null); + return value; + } + return null; + } + + @Override + public T put(StructLike key, T value) { + return wrapperMap.put(StructLikeWrapper.forType(type).set(key), value); + } + + @Override + public T remove(Object key) { + if (key instanceof StructLike) { + StructLikeWrapper wrapper = wrappers.get(); + T value = wrapperMap.remove(wrapper.set((StructLike) key)); + wrapper.set(null); // don't hold a reference to the value. + return value; + } + return null; + } + + @Override + public void putAll(Map keyValues) { + if (keyValues != null && !keyValues.isEmpty()) { + for (Map.Entry pair : keyValues.entrySet()) { + wrapperMap.put(StructLikeWrapper.forType(type).set(pair.getKey()), pair.getValue()); + } + } + } + + @Override + public void clear() { + wrapperMap.clear(); + } + + @Override + public Set keySet() { + return wrapperMap.keySet().stream().map(StructLikeWrapper::get).collect(Collectors.toSet()); + } + + @Override + public Collection values() { + return wrapperMap.values(); + } + + @Override + public Set> entrySet() { + throw new UnsupportedOperationException(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/util/StructLikeSet.java b/core/src/main/java/org/apache/iceberg/util/StructLikeSet.java index 047eac0de841..63a461d5deb3 100644 --- a/core/src/main/java/org/apache/iceberg/util/StructLikeSet.java +++ b/core/src/main/java/org/apache/iceberg/util/StructLikeSet.java @@ -19,6 +19,7 @@ package org.apache.iceberg.util; +import java.util.AbstractCollection; import java.util.Collection; import java.util.Iterator; import java.util.Objects; @@ -29,7 +30,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; -public class StructLikeSet implements Set { +public class StructLikeSet extends AbstractCollection implements Set { public static StructLikeSet create(Types.StructType type) { return new StructLikeSet(type); } @@ -104,7 +105,7 @@ public boolean add(StructLike struct) { @Override public boolean remove(Object obj) { - if (obj instanceof CharSequence) { + if (obj instanceof StructLike) { StructLikeWrapper wrapper = wrappers.get(); boolean result = wrapperSet.remove(wrapper.set((StructLike) obj)); wrapper.set(null); // don't hold a reference to the value diff --git a/core/src/test/java/org/apache/iceberg/TableTestBase.java b/core/src/test/java/org/apache/iceberg/TableTestBase.java index 6009db38b858..16992f22907f 100644 --- a/core/src/test/java/org/apache/iceberg/TableTestBase.java +++ b/core/src/test/java/org/apache/iceberg/TableTestBase.java @@ -50,7 +50,7 @@ public class TableTestBase { ); // Partition spec used to create tables - static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA) + public static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA) .bucket("data", 16) .build(); diff --git a/core/src/test/java/org/apache/iceberg/avro/TestAvroDeleteWriters.java b/core/src/test/java/org/apache/iceberg/avro/TestAvroDeleteWriters.java index dc6113922119..d6184ee05317 100644 --- a/core/src/test/java/org/apache/iceberg/avro/TestAvroDeleteWriters.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestAvroDeleteWriters.java @@ -85,10 +85,10 @@ public void testEqualityDeleteWriter() throws IOException { .buildEqualityWriter(); try (EqualityDeleteWriter writer = deleteWriter) { - writer.deleteAll(records); + writer.writeAll(records); } - DeleteFile metadata = deleteWriter.toDeleteFile(); + DeleteFile metadata = deleteWriter.toContentFile(); Assert.assertEquals("Format should be Avro", FileFormat.AVRO, metadata.format()); Assert.assertEquals("Should be equality deletes", FileContent.EQUALITY_DELETES, metadata.content()); Assert.assertEquals("Record count should be correct", records.size(), metadata.recordCount()); @@ -138,7 +138,7 @@ public void testPositionDeleteWriter() throws IOException { } } - DeleteFile metadata = deleteWriter.toDeleteFile(); + DeleteFile metadata = deleteWriter.toContentFile(); Assert.assertEquals("Format should be Avro", FileFormat.AVRO, metadata.format()); Assert.assertEquals("Should be position deletes", FileContent.POSITION_DELETES, metadata.content()); Assert.assertEquals("Record count should be correct", records.size(), metadata.recordCount()); @@ -185,7 +185,7 @@ public void testPositionDeleteWriterWithEmptyRow() throws IOException { } } - DeleteFile metadata = deleteWriter.toDeleteFile(); + DeleteFile metadata = deleteWriter.toContentFile(); Assert.assertEquals("Format should be Avro", FileFormat.AVRO, metadata.format()); Assert.assertEquals("Should be position deletes", FileContent.POSITION_DELETES, metadata.content()); Assert.assertEquals("Record count should be correct", records.size(), metadata.recordCount()); diff --git a/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java b/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java new file mode 100644 index 000000000000..323920d77ad2 --- /dev/null +++ b/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java @@ -0,0 +1,243 @@ +/* + * 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.UncheckedIOException; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.ContentFileWriterFactory; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFileWriter; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetricsConfig; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.data.avro.DataWriter; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.io.BaseDeltaWriter; +import org.apache.iceberg.io.DeltaWriter; +import org.apache.iceberg.io.DeltaWriterFactory; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.io.RollingContentFileWriter; +import org.apache.iceberg.io.RollingEqDeleteWriter; +import org.apache.iceberg.io.RollingPosDeleteWriter; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +public class GenericDeltaWriterFactory implements DeltaWriterFactory { + + private final Schema schema; + private final PartitionSpec spec; + private final FileFormat format; + private final OutputFileFactory fileFactory; + private final FileIO io; + private final long targetFileSize; + private final Map tableProperties; + private final FileAppenderFactory appenderFactory; + + public GenericDeltaWriterFactory(Schema schema, + PartitionSpec spec, + FileFormat format, + OutputFileFactory fileFactory, + FileIO io, + long targetFileSize, + Map tableProperties) { + this.schema = schema; + this.spec = spec; + this.format = format; + this.fileFactory = fileFactory; + this.io = io; + this.targetFileSize = targetFileSize; + this.tableProperties = tableProperties; + this.appenderFactory = createFileAppenderFactory(); + } + + @Override + public DeltaWriter createDeltaWriter(PartitionKey partitionKey, Context ctxt) { + RollingContentFileWriter dataWriter = new RollingContentFileWriter<>(partitionKey, + format, fileFactory, io, targetFileSize, createDataFileWriterFactory()); + + if (!ctxt.allowPosDelete() && !ctxt.allowEqualityDelete()) { + // Only accept INSERT records. + return new GenericDeltaWriter(dataWriter); + } + + RollingPosDeleteWriter posDeleteWriter = new RollingPosDeleteWriter<>(partitionKey, + format, fileFactory, io, targetFileSize, createPosDeleteWriterFactory(ctxt.posDeleteRowSchema())); + + if (ctxt.allowPosDelete() && !ctxt.allowEqualityDelete()) { + // Only accept INSERT records and POS-DELETE records. + return new GenericDeltaWriter(dataWriter, posDeleteWriter); + } + + // Accept INSERT records, POS-DELETE records and EQUALITY-DELETE records. + Preconditions.checkState(ctxt.allowEqualityDelete(), "Should always allow equality-delete here."); + Preconditions.checkState(ctxt.equalityFieldIds() != null && !ctxt.equalityFieldIds().isEmpty(), + "Equality field id list shouldn't be null or emtpy."); + + RollingEqDeleteWriter eqDeleteWriter = new RollingEqDeleteWriter<>(partitionKey, + format, fileFactory, io, targetFileSize, + createEqualityDeleteWriterFactory(ctxt.equalityFieldIds(), ctxt.eqDeleteRowSchema())); + + return new GenericDeltaWriter(dataWriter, posDeleteWriter, eqDeleteWriter, schema, ctxt.equalityFieldIds()); + } + + @Override + public FileAppenderFactory createFileAppenderFactory() { + return new GenericAppenderFactory(schema); + } + + @Override + public ContentFileWriterFactory createDataFileWriterFactory() { + return (partitionKey, outputFile, fileFormat) -> { + FileAppender appender = appenderFactory.newAppender(outputFile.encryptingOutputFile(), fileFormat); + + return new DataFileWriter<>(appender, + fileFormat, + outputFile.encryptingOutputFile().location(), + partitionKey, + spec, outputFile.keyMetadata()); + }; + } + + @Override + public ContentFileWriterFactory createEqualityDeleteWriterFactory( + List equalityFieldIds, Schema rowSchema) { + return (partitionKey, outputFile, fileFormat) -> { + + MetricsConfig metricsConfig = MetricsConfig.fromProperties(tableProperties); + try { + switch (fileFormat) { + case AVRO: + return Avro.writeDeletes(outputFile.encryptingOutputFile()) + .createWriterFunc(DataWriter::create) + .withPartition(partitionKey) + .overwrite() + .setAll(tableProperties) + .rowSchema(rowSchema) + .withSpec(spec) + .withKeyMetadata(outputFile.keyMetadata()) + .equalityFieldIds(equalityFieldIds) + .buildEqualityWriter(); + + case PARQUET: + return Parquet.writeDeletes(outputFile.encryptingOutputFile()) + .createWriterFunc(GenericParquetWriter::buildWriter) + .withPartition(partitionKey) + .overwrite() + .setAll(tableProperties) + .metricsConfig(metricsConfig) + .rowSchema(rowSchema) + .withSpec(spec) + .withKeyMetadata(outputFile.keyMetadata()) + .equalityFieldIds(equalityFieldIds) + .buildEqualityWriter(); + + case ORC: + throw new UnsupportedOperationException("Orc file format does not support writing equality delete."); + + default: + throw new UnsupportedOperationException("Cannot write unknown file format: " + fileFormat); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + }; + } + + @Override + public ContentFileWriterFactory> createPosDeleteWriterFactory(Schema rowSchema) { + return (partitionKey, outputFile, fileFormat) -> { + MetricsConfig metricsConfig = MetricsConfig.fromProperties(tableProperties); + try { + switch (fileFormat) { + case AVRO: + return Avro.writeDeletes(outputFile.encryptingOutputFile()) + .createWriterFunc(DataWriter::create) + .withPartition(partitionKey) + .overwrite() + .setAll(tableProperties) + .rowSchema(rowSchema) // it's a nullable field. + .withSpec(spec) + .withKeyMetadata(outputFile.keyMetadata()) + .buildPositionWriter(); + + case PARQUET: + return Parquet.writeDeletes(outputFile.encryptingOutputFile()) + .createWriterFunc(GenericParquetWriter::buildWriter) + .withPartition(partitionKey) + .overwrite() + .setAll(tableProperties) + .metricsConfig(metricsConfig) + .rowSchema(rowSchema) // it's a nullable field. + .withSpec(spec) + .withKeyMetadata(outputFile.keyMetadata()) + .buildPositionWriter(); + + case ORC: + throw new UnsupportedOperationException("Orc file format does not support writing positional delete."); + + default: + throw new UnsupportedOperationException("Cannot write unknown file format: " + fileFormat); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + }; + } + + private static class GenericDeltaWriter extends BaseDeltaWriter { + + GenericDeltaWriter(RollingContentFileWriter dataWriter) { + this(dataWriter, null); + } + + GenericDeltaWriter(RollingContentFileWriter dataWriter, + RollingPosDeleteWriter posDeleteWriter) { + this(dataWriter, posDeleteWriter, null, null, null); + } + + GenericDeltaWriter(RollingContentFileWriter dataWriter, + RollingPosDeleteWriter posDeleteWriter, + RollingEqDeleteWriter eqDeleteWriter, Schema tableSchema, + List equalityFieldIds) { + super(dataWriter, posDeleteWriter, eqDeleteWriter, tableSchema, equalityFieldIds); + } + + @Override + protected StructLike asKey(Record row) { + return row; + } + + @Override + protected StructLike asCopiedKey(Record row) { + return row.copy(); + } + } +} diff --git a/data/src/test/java/org/apache/iceberg/data/FileHelpers.java b/data/src/test/java/org/apache/iceberg/data/FileHelpers.java index 64488e78e65b..43d3ca474ccf 100644 --- a/data/src/test/java/org/apache/iceberg/data/FileHelpers.java +++ b/data/src/test/java/org/apache/iceberg/data/FileHelpers.java @@ -64,7 +64,7 @@ public static Pair> writeDeleteFile(Table table, O } } - return Pair.of(writer.toDeleteFile(), writer.referencedDataFiles()); + return Pair.of(writer.toContentFile(), writer.referencedDataFiles()); } public static DeleteFile writeDeleteFile(Table table, OutputFile out, List deletes, Schema deleteRowSchema) @@ -84,10 +84,10 @@ public static DeleteFile writeDeleteFile(Table table, OutputFile out, StructLike .buildEqualityWriter(); try (Closeable toClose = writer) { - writer.deleteAll(deletes); + writer.writeAll(deletes); } - return writer.toDeleteFile(); + return writer.toContentFile(); } public static DataFile writeDataFile(Table table, OutputFile out, List rows) throws IOException { diff --git a/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java b/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java new file mode 100644 index 000000000000..00553be09d0b --- /dev/null +++ b/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java @@ -0,0 +1,351 @@ +/* + * 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.File; +import java.io.IOException; +import java.util.List; +import java.util.Locale; +import java.util.Set; +import org.apache.iceberg.AssertHelpers; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestTables; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.io.DeltaWriter; +import org.apache.iceberg.io.DeltaWriterFactory; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.io.WriterResult; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.StructLikeSet; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestGenericDeltaWriter extends TableTestBase { + + private static final int FORMAT_VERSION_V2 = 2; + private static final String TABLE_NAME = "delta_table"; + + @Rule + public final TemporaryFolder tempFolder = new TemporaryFolder(); + + @Parameterized.Parameters(name = "format = {0}, partitioned = {1}") + public static Object[][] parameters() { + return new Object[][] { + {"avro", false}, + {"parquet", false}, + }; + } + + private final FileFormat format; + private final boolean partitioned; + private Table table; + + public TestGenericDeltaWriter(String format, boolean partitioned) { + super(FORMAT_VERSION_V2); + this.format = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH)); + this.partitioned = partitioned; + } + + @Before + public void before() throws IOException { + File tableDir = tempFolder.newFolder(); + Assert.assertTrue(tableDir.delete()); + + if (partitioned) { + this.table = TestTables.create(tableDir, TABLE_NAME, SCHEMA, SPEC, formatVersion); + } else { + this.table = TestTables.create(tableDir, TABLE_NAME, SCHEMA, PartitionSpec.unpartitioned(), formatVersion); + } + } + + @Test + public void testWritePureInsert() { + DeltaWriterFactory writerFactory = createDeltaWriterFactory(); + + DeltaWriterFactory.Context ctxt = DeltaWriterFactory.Context.builder() + .allowPosDelete(false) + .allowEqualityDelete(false) + .build(); + + DeltaWriter deltaWriter = writerFactory.createDeltaWriter(null, ctxt); + + List expected = RandomGenericData.generate(SCHEMA, 100, 22112234L); + expected.forEach(deltaWriter::writeRow); + + WriterResult result = deltaWriter.complete(); + + Assert.assertEquals(result.dataFiles().length, 1); + Assert.assertEquals(result.deleteFiles().length, 0); + + commitTransaction(result); + + Assert.assertEquals(Sets.newHashSet(expected), Sets.newHashSet(IcebergGenerics.read(table).build())); + } + + @Test + public void testWriteEqualityDelete() { + DeltaWriterFactory writerFactory = createDeltaWriterFactory(); + + List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); + DeltaWriterFactory.Context ctxt = DeltaWriterFactory.Context.builder() + .allowEqualityDelete(true) + .equalityFieldIds(equalityFieldIds) + .eqDeleteRowSchema(table.schema()) + .build(); + + // TODO More unit tests to test the partitioned case. + DeltaWriter deltaWriter = writerFactory.createDeltaWriter(null, ctxt); + + GenericRecord record = GenericRecord.create(SCHEMA); + Record record1 = record.copy("id", 1, "data", "aaa"); + Record record2 = record.copy("id", 2, "data", "bbb"); + Record record3 = record.copy("id", 3, "data", "ccc"); + + deltaWriter.writeRow(record1); + deltaWriter.writeRow(record2); + + deltaWriter.writeEqualityDelete(record1); + deltaWriter.writeEqualityDelete(record2); + + deltaWriter.writeRow(record3); + + WriterResult result = deltaWriter.complete(); + Assert.assertEquals(result.dataFiles().length, 1); + Assert.assertEquals(result.deleteFiles().length, 1); + commitTransaction(result); + + assertTableRecords(Sets.newHashSet(record3)); + + deltaWriter = writerFactory.createDeltaWriter(null, ctxt); + deltaWriter.writeEqualityDelete(record3); + + result = deltaWriter.complete(); + Assert.assertEquals(result.dataFiles().length, 0); + Assert.assertEquals(result.deleteFiles().length, 1); + commitTransaction(result); + + assertTableRecords(ImmutableSet.of()); + } + + @Test + public void testEqualityDeleteSameRow() { + DeltaWriterFactory writerFactory = createDeltaWriterFactory(); + + List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); + DeltaWriterFactory.Context ctxt = DeltaWriterFactory.Context.builder() + .allowEqualityDelete(true) + .equalityFieldIds(equalityFieldIds) + .eqDeleteRowSchema(table.schema()) + .posDeleteRowSchema(table.schema()) + .build(); + + DeltaWriter deltaWriter1 = writerFactory.createDeltaWriter(null, ctxt); + + GenericRecord record = GenericRecord.create(SCHEMA); + Record record1 = record.copy("id", 1, "data", "aaa"); + + deltaWriter1.writeRow(record1); + deltaWriter1.writeEqualityDelete(record1); + deltaWriter1.writeRow(record1); + deltaWriter1.writeEqualityDelete(record1); + deltaWriter1.writeRow(record1); + + AssertHelpers.assertThrows("Encountered duplicated keys in the same transaction", + ValidationException.class, () -> deltaWriter1.writeRow(record1)); + + WriterResult result = deltaWriter1.complete(); + Assert.assertEquals(result.dataFiles().length, 1); + Assert.assertEquals(result.deleteFiles().length, 1); + commitTransaction(result); + + assertTableRecords(ImmutableSet.of(record1)); + + DeltaWriter deltaWriter2 = writerFactory.createDeltaWriter(null, ctxt); + deltaWriter2.writeRow(record1); + + result = deltaWriter2.complete(); + Assert.assertEquals(result.dataFiles().length, 1); + Assert.assertEquals(result.deleteFiles().length, 0); + commitTransaction(result); + + assertTableRecords(ImmutableSet.of(record1, record1)); + } + + @Test + public void testPositionDelete() { + DeltaWriterFactory writerFactory = createDeltaWriterFactory(); + DeltaWriterFactory.Context ctxt = DeltaWriterFactory.Context.builder() + .allowPosDelete(true) + .build(); + + DeltaWriter deltaWriter = writerFactory.createDeltaWriter(null, ctxt); + + GenericRecord record = GenericRecord.create(SCHEMA); + Record record1 = record.copy("id", 1, "data", "aaa"); + Record record2 = record.copy("id", 1, "data", "bbb"); + Record record3 = record.copy("id", 1, "data", "ccc"); + + // Write two records. + deltaWriter.writeRow(record1); + deltaWriter.writeRow(record2); + + WriterResult result = deltaWriter.complete(); + Assert.assertEquals(result.dataFiles().length, 1); + Assert.assertEquals(result.deleteFiles().length, 0); + commitTransaction(result); + + CharSequence dataFilePath = result.dataFiles()[0].path(); + + // Delete the second record. + deltaWriter = writerFactory.createDeltaWriter(null, ctxt); + deltaWriter.writePosDelete(dataFilePath, 1); + + result = deltaWriter.complete(); + Assert.assertEquals(result.dataFiles().length, 0); + Assert.assertEquals(result.deleteFiles().length, 1); + commitTransaction(result); + + assertTableRecords(ImmutableSet.of(record1)); + + // Delete the first record. + deltaWriter = writerFactory.createDeltaWriter(null, ctxt); + deltaWriter.writePosDelete(dataFilePath, 0); + deltaWriter.writeRow(record3); + + result = deltaWriter.complete(); + Assert.assertEquals(result.dataFiles().length, 1); + Assert.assertEquals(result.deleteFiles().length, 1); + commitTransaction(result); + + assertTableRecords(ImmutableSet.of(record3)); + } + + @Test + public void testUpsertSameRow() { + DeltaWriterFactory writerFactory = createDeltaWriterFactory(); + List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); + DeltaWriterFactory.Context ctxt = DeltaWriterFactory.Context.builder() + .allowEqualityDelete(true) + .equalityFieldIds(equalityFieldIds) + .eqDeleteRowSchema(table.schema().select("id")) + .posDeleteRowSchema(table.schema()) + .build(); + final DeltaWriter deltaWriter1 = writerFactory.createDeltaWriter(null, ctxt); + + GenericRecord record = GenericRecord.create(SCHEMA); + Record record1 = record.copy("id", 1, "data", "aaa"); + Record record2 = record.copy("id", 1, "data", "bbb"); + Record record3 = record.copy("id", 1, "data", "ccc"); + Record record4 = record.copy("id", 1, "data", "ddd"); + Record record5 = record.copy("id", 1, "data", "eee"); + Record record6 = record.copy("id", 1, "data", "fff"); + Record record7 = record.copy("id", 1, "data", "ggg"); + + deltaWriter1.writeRow(record1); + AssertHelpers.assertThrows("Detect duplicated keys", ValidationException.class, + () -> deltaWriter1.writeRow(record2)); + + // Commit the transaction. + WriterResult result = deltaWriter1.complete(); + Assert.assertEquals(result.dataFiles().length, 1); + Assert.assertEquals(result.deleteFiles().length, 0); + commitTransaction(result); + + assertTableRecords(ImmutableSet.of(record1)); + + final DeltaWriter deltaWriter2 = writerFactory.createDeltaWriter(null, ctxt); + + // UPSERT (1, "ccc") + deltaWriter2.writeEqualityDelete(record3); + deltaWriter2.writeRow(record3); + + // INSERT (1, "ddd") + // INSERT (1, "eee") + AssertHelpers.assertThrows("Detect duplicated keys", ValidationException.class, + () -> deltaWriter2.writeRow(record4)); + AssertHelpers.assertThrows("Detect duplicated keys", ValidationException.class, + () -> deltaWriter2.writeRow(record5)); + + // UPSERT (1, "fff") + deltaWriter2.writeEqualityDelete(record6); + deltaWriter2.writeRow(record6); + + // INSERT (1, "ggg") + AssertHelpers.assertThrows("Detect duplicated keys", ValidationException.class, + () -> deltaWriter2.writeRow(record7)); + + // Commit the transaction. + result = deltaWriter2.complete(); + Assert.assertEquals(1, result.dataFiles().length); + // One pos-delete file, and one equality-delete file. + Assert.assertEquals(2, result.deleteFiles().length); + commitTransaction(result); + + assertTableRecords(ImmutableSet.of(record6)); + } + + private void assertTableRecords(Set expectedRecords) { + StructLikeSet expectedSet = StructLikeSet.create(SCHEMA.asStruct()); + expectedSet.addAll(expectedRecords); + + StructLikeSet actualSet = StructLikeSet.create(SCHEMA.asStruct()); + Iterables.addAll(actualSet, IcebergGenerics.read(table).build()); + Assert.assertEquals(expectedSet, actualSet); + } + + private void commitTransaction(WriterResult result) { + RowDelta rowDelta = table.newRowDelta(); + + for (DataFile dataFile : result.dataFiles()) { + rowDelta.addRows(dataFile); + } + + for (DeleteFile deleteFile : result.deleteFiles()) { + rowDelta.addDeletes(deleteFile); + } + + rowDelta.commit(); + } + + private DeltaWriterFactory createDeltaWriterFactory() { + OutputFileFactory outputFileFactory = + new OutputFileFactory(table.spec(), format, table.locationProvider(), table.io(), + table.encryption(), 1, 1); + + return new GenericDeltaWriterFactory(table.schema(), table.spec(), format, outputFileFactory, table.io(), + 128 * 1024 * 1024L, table.properties() + ); + } +} diff --git a/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java b/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java index 401e9db65992..4ab6bfd923ec 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java +++ b/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java @@ -53,6 +53,15 @@ public RowDataWrapper(RowType rowType, Types.StructType struct) { } } + private RowDataWrapper(RowDataWrapper toCopy) { + this.types = toCopy.types; + this.getters = toCopy.getters; + } + + public RowDataWrapper copy() { + return new RowDataWrapper(this); + } + public RowDataWrapper wrap(RowData data) { this.rowData = data; return this; @@ -135,4 +144,9 @@ private static PositionalGetter buildGetter(LogicalType logicalType, Type typ return null; } } + + @Override + public String toString() { + return rowData == null ? null : rowData.toString(); + } } diff --git a/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestSerializer.java b/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java similarity index 50% rename from flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestSerializer.java rename to flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java index bec4e65d0cad..d58b856c7da0 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestSerializer.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java @@ -19,30 +19,42 @@ package org.apache.iceberg.flink.sink; -import java.io.IOException; -import org.apache.flink.core.io.SimpleVersionedSerializer; +import java.util.Iterator; +import java.util.List; import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.ManifestFiles; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.jetbrains.annotations.NotNull; -class FlinkManifestSerializer implements SimpleVersionedSerializer { - private static final int VERSION_NUM = 1; - static final FlinkManifestSerializer INSTANCE = new FlinkManifestSerializer(); +class DeltaManifests implements Iterable { - @Override - public int getVersion() { - return VERSION_NUM; + private final ManifestFile dataManifest; + private final ManifestFile deleteManifest; + + DeltaManifests(ManifestFile dataManifest, ManifestFile deleteManifest) { + this.dataManifest = dataManifest; + this.deleteManifest = deleteManifest; } - @Override - public byte[] serialize(ManifestFile manifestFile) throws IOException { - Preconditions.checkNotNull(manifestFile, "ManifestFile to be serialized should not be null"); + ManifestFile dataManifest() { + return dataManifest; + } - return ManifestFiles.encode(manifestFile); + ManifestFile deleteManifest() { + return deleteManifest; } + @NotNull @Override - public ManifestFile deserialize(int version, byte[] serialized) throws IOException { - return ManifestFiles.decode(serialized); + public Iterator iterator() { + List manifests = Lists.newArrayListWithCapacity(2); + if (dataManifest != null) { + manifests.add(dataManifest); + } + + if (deleteManifest != null) { + manifests.add(deleteManifest); + } + + return manifests.iterator(); } } diff --git a/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java b/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java new file mode 100644 index 000000000000..d99964b30796 --- /dev/null +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java @@ -0,0 +1,94 @@ +/* + * 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.sink; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class DeltaManifestsSerializer implements SimpleVersionedSerializer { + private static final int VERSION_NUM = 1; + private static final byte[] EMPTY_BINARY = new byte[0]; + + static final DeltaManifestsSerializer INSTANCE = new DeltaManifestsSerializer(); + + @Override + public int getVersion() { + return VERSION_NUM; + } + + @Override + public byte[] serialize(DeltaManifests deltaManifests) throws IOException { + Preconditions.checkNotNull(deltaManifests, "DeltaManifests to be serialized should not be null"); + + ByteArrayOutputStream binaryOut = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(binaryOut); + + byte[] dataManifestBinary = EMPTY_BINARY; + if (deltaManifests.dataManifest() != null) { + dataManifestBinary = ManifestFiles.encode(deltaManifests.dataManifest()); + } + + out.writeInt(dataManifestBinary.length); + out.write(dataManifestBinary); + + byte[] deleteManifestBinary = EMPTY_BINARY; + if (deltaManifests.deleteManifest() != null) { + deleteManifestBinary = ManifestFiles.encode(deltaManifests.deleteManifest()); + } + + out.writeInt(deleteManifestBinary.length); + out.write(deleteManifestBinary); + + return binaryOut.toByteArray(); + } + + @Override + public DeltaManifests deserialize(int version, byte[] serialized) throws IOException { + ByteArrayInputStream binaryIn = new ByteArrayInputStream(serialized); + DataInputStream in = new DataInputStream(binaryIn); + + ManifestFile dataManifest = null; + int dataManifestSize = in.readInt(); + if (dataManifestSize > 0) { + byte[] dataManifestBinary = new byte[dataManifestSize]; + Preconditions.checkState(in.read(dataManifestBinary) == dataManifestSize); + + dataManifest = ManifestFiles.decode(dataManifestBinary); + } + + ManifestFile deleteManifest = null; + int deleteManifestSize = in.readInt(); + if (deleteManifestSize > 0) { + byte[] deleteManifestBinary = new byte[deleteManifestSize]; + Preconditions.checkState(in.read(deleteManifestBinary) == deleteManifestSize); + + deleteManifest = ManifestFiles.decode(deleteManifestBinary); + } + + return new DeltaManifests(dataManifest, deleteManifest); + } +} diff --git a/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkDeltaWriterFactory.java b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkDeltaWriterFactory.java new file mode 100644 index 000000000000..5f7b382102af --- /dev/null +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkDeltaWriterFactory.java @@ -0,0 +1,280 @@ +/* + * 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.sink; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import org.apache.avro.io.DatumWriter; +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.ContentFileWriterFactory; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFileWriter; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetricsConfig; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.deletes.DeletesUtil; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.data.FlinkAvroWriter; +import org.apache.iceberg.flink.data.FlinkParquetWriters; +import org.apache.iceberg.io.BaseDeltaWriter; +import org.apache.iceberg.io.DeltaWriter; +import org.apache.iceberg.io.DeltaWriterFactory; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.io.RollingContentFileWriter; +import org.apache.iceberg.io.RollingEqDeleteWriter; +import org.apache.iceberg.io.RollingPosDeleteWriter; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +public class FlinkDeltaWriterFactory implements DeltaWriterFactory { + + private final Schema schema; + private final RowType flinkSchema; + private final PartitionSpec spec; + private final FileFormat format; + private final OutputFileFactory fileFactory; + private final FileIO io; + private final long targetFileSize; + private final Map tableProperties; + private final FileAppenderFactory appenderFactory; + + public FlinkDeltaWriterFactory(Schema schema, + RowType flinkSchema, + PartitionSpec spec, + FileFormat format, + OutputFileFactory fileFactory, + FileIO io, + long targetFileSize, + Map tableProperties) { + this.schema = schema; + this.flinkSchema = flinkSchema; + this.spec = spec; + this.format = format; + this.fileFactory = fileFactory; + this.io = io; + this.targetFileSize = targetFileSize; + this.tableProperties = tableProperties; + this.appenderFactory = createFileAppenderFactory(); + } + + @Override + public DeltaWriter createDeltaWriter(PartitionKey partitionKey, Context ctxt) { + RollingContentFileWriter dataWriter = new RollingContentFileWriter<>(partitionKey, + format, fileFactory, io, targetFileSize, createDataFileWriterFactory()); + + if (!ctxt.allowPosDelete() && !ctxt.allowEqualityDelete()) { + // Only accept INSERT records. + return new RowDataDeltaWriter(dataWriter); + } + + RollingPosDeleteWriter posDeleteWriter = new RollingPosDeleteWriter<>(partitionKey, + format, fileFactory, io, targetFileSize, createPosDeleteWriterFactory(ctxt.posDeleteRowSchema())); + + if (ctxt.allowPosDelete() && !ctxt.allowEqualityDelete()) { + // Only accept INSERT records and POS-DELETE records. + return new RowDataDeltaWriter(dataWriter, posDeleteWriter); + } + + // Accept INSERT records, POS-DELETE records and EQUALITY-DELETE records. + Preconditions.checkState(ctxt.allowEqualityDelete(), "Should always allow equality-delete here."); + Preconditions.checkState(ctxt.equalityFieldIds() != null && !ctxt.equalityFieldIds().isEmpty(), + "Equality field id list shouldn't be null or emtpy."); + + RollingEqDeleteWriter eqDeleteWriter = new RollingEqDeleteWriter<>(partitionKey, + format, fileFactory, io, targetFileSize, + createEqualityDeleteWriterFactory(ctxt.equalityFieldIds(), ctxt.eqDeleteRowSchema())); + + return new RowDataDeltaWriter(dataWriter, posDeleteWriter, eqDeleteWriter, schema, ctxt.equalityFieldIds()); + } + + @Override + public FileAppenderFactory createFileAppenderFactory() { + return new FlinkFileAppenderFactory(schema, flinkSchema, tableProperties); + } + + @Override + public ContentFileWriterFactory createDataFileWriterFactory() { + // TODO Move this to its default function ??? + return (partitionKey, outputFile, fileFormat) -> { + FileAppender appender = appenderFactory.newAppender(outputFile.encryptingOutputFile(), fileFormat); + + return new DataFileWriter<>(appender, + fileFormat, + outputFile.encryptingOutputFile().location(), + partitionKey, + spec, outputFile.keyMetadata()); + }; + } + + @Override + public ContentFileWriterFactory createEqualityDeleteWriterFactory(List equalityFieldIds, + Schema rowSchema) { + Preconditions.checkNotNull(rowSchema, "Row schema shouldn't be null for equality deletes."); + RowType flinkRowSchema = FlinkSchemaUtil.convert(rowSchema); + + return (partitionKey, outputFile, fileFormat) -> { + + MetricsConfig metricsConfig = MetricsConfig.fromProperties(tableProperties); + try { + switch (fileFormat) { + case AVRO: + return Avro.writeDeletes(outputFile.encryptingOutputFile()) + .createWriterFunc(ignore -> new FlinkAvroWriter(flinkRowSchema)) + .withPartition(partitionKey) + .overwrite() + .setAll(tableProperties) + .rowSchema(rowSchema) + .withSpec(spec) + .withKeyMetadata(outputFile.keyMetadata()) + .equalityFieldIds(equalityFieldIds) + .buildEqualityWriter(); + + case PARQUET: + return Parquet.writeDeletes(outputFile.encryptingOutputFile()) + .createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(flinkRowSchema, msgType)) + .withPartition(partitionKey) + .overwrite() + .setAll(tableProperties) + .metricsConfig(metricsConfig) + .rowSchema(rowSchema) + .withSpec(spec) + .withKeyMetadata(outputFile.keyMetadata()) + .equalityFieldIds(equalityFieldIds) + .buildEqualityWriter(); + + case ORC: + throw new UnsupportedOperationException("Orc file format does not support writing equality delete."); + + default: + throw new UnsupportedOperationException("Cannot write unknown file format: " + fileFormat); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + }; + } + + @Override + public ContentFileWriterFactory> createPosDeleteWriterFactory(Schema rowSchema) { + + return (partitionKey, outputFile, fileFormat) -> { + MetricsConfig metricsConfig = MetricsConfig.fromProperties(tableProperties); + try { + switch (fileFormat) { + case AVRO: + // Produce the positional delete schema that writer will use for the file. + Function> writeFunc = + rowSchema == null ? null : ignore -> new FlinkAvroWriter(FlinkSchemaUtil.convert(rowSchema)); + return Avro.writeDeletes(outputFile.encryptingOutputFile()) + .createWriterFunc(writeFunc) + .withPartition(partitionKey) + .overwrite() + .setAll(tableProperties) + .rowSchema(rowSchema) // it's a nullable field. + .withSpec(spec) + .withKeyMetadata(outputFile.keyMetadata()) + .buildPositionWriter(); + + case PARQUET: + RowType flinkParquetRowType = FlinkSchemaUtil.convert(DeletesUtil.posDeleteSchema(rowSchema)); + return Parquet.writeDeletes(outputFile.encryptingOutputFile()) + .createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(flinkParquetRowType, msgType)) + .withPartition(partitionKey) + .overwrite() + .setAll(tableProperties) + .metricsConfig(metricsConfig) + .rowSchema(rowSchema) // it's a nullable field. + .withSpec(spec) + .withKeyMetadata(outputFile.keyMetadata()) + .buildPositionWriter(RowDataPositionAccessor.INSTANCE); + + case ORC: + throw new UnsupportedOperationException("Orc file format does not support writing positional delete."); + + default: + throw new UnsupportedOperationException("Cannot write unknown file format: " + fileFormat); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + }; + } + + private class RowDataDeltaWriter extends BaseDeltaWriter { + private final RowDataWrapper rowDataWrapper; + + RowDataDeltaWriter(RollingContentFileWriter dataWriter) { + this(dataWriter, null); + } + + RowDataDeltaWriter(RollingContentFileWriter dataWriter, + RollingPosDeleteWriter posDeleteWriter) { + this(dataWriter, posDeleteWriter, null, null, null); + } + + RowDataDeltaWriter(RollingContentFileWriter dataWriter, + RollingPosDeleteWriter posDeleteWriter, + RollingEqDeleteWriter equalityDeleteWriter, Schema tableSchema, + List equalityFieldIds) { + super(dataWriter, posDeleteWriter, equalityDeleteWriter, tableSchema, equalityFieldIds); + + this.rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); + } + + @Override + protected StructLike asKey(RowData row) { + return rowDataWrapper.wrap(row); + } + + @Override + protected StructLike asCopiedKey(RowData row) { + return rowDataWrapper.copy().wrap(row); + } + } + + private static class RowDataPositionAccessor implements Parquet.PositionAccessor { + private static final RowDataPositionAccessor INSTANCE = new RowDataPositionAccessor(); + + @Override + public StringData accessPath(CharSequence path) { + return StringData.fromString(path.toString()); + } + + @Override + public Long accessPos(long pos) { + return pos; + } + } +} diff --git a/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileAppenderFactory.java b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileAppenderFactory.java new file mode 100644 index 000000000000..1c217b1c0665 --- /dev/null +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileAppenderFactory.java @@ -0,0 +1,90 @@ +/* + * 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.sink; + +import java.io.IOException; +import java.io.Serializable; +import java.io.UncheckedIOException; +import java.util.Map; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetricsConfig; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.Avro; +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.io.FileAppender; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.parquet.Parquet; + +public class FlinkFileAppenderFactory implements FileAppenderFactory, Serializable { + + private final Schema schema; + private final RowType flinkSchema; + private final Map props; + + public FlinkFileAppenderFactory(Schema schema, RowType flinkSchema, Map props) { + this.schema = schema; + this.flinkSchema = flinkSchema; + this.props = props; + } + + @Override + public FileAppender newAppender(OutputFile outputFile, FileFormat format) { + MetricsConfig metricsConfig = MetricsConfig.fromProperties(props); + try { + switch (format) { + case AVRO: + return Avro.write(outputFile) + .createWriterFunc(ignore -> new FlinkAvroWriter(flinkSchema)) + .setAll(props) + .schema(schema) + .overwrite() + .build(); + + case ORC: + return ORC.write(outputFile) + .createWriterFunc((iSchema, typDesc) -> FlinkOrcWriter.buildWriter(flinkSchema, iSchema)) + .setAll(props) + .schema(schema) + .overwrite() + .build(); + + case PARQUET: + return Parquet.write(outputFile) + .createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(flinkSchema, msgType)) + .setAll(props) + .metricsConfig(metricsConfig) + .schema(schema) + .overwrite() + .build(); + + default: + throw new UnsupportedOperationException("Cannot write unknown file format: " + format); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } +} diff --git a/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java index aa36c7344e67..bdd2a0a4ac33 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java @@ -21,7 +21,9 @@ import java.io.IOException; import java.util.List; +import java.util.function.Supplier; import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.ManifestFiles; @@ -32,10 +34,11 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.WriterResult; import org.apache.iceberg.relocated.com.google.common.collect.Lists; class FlinkManifestUtil { - private static final int ICEBERG_FORMAT_VERSION = 2; + private static final int FORMAT_V2 = 2; private static final Long DUMMY_SNAPSHOT_ID = 0L; private FlinkManifestUtil() { @@ -43,7 +46,7 @@ private FlinkManifestUtil() { static ManifestFile writeDataFiles(OutputFile outputFile, PartitionSpec spec, List dataFiles) throws IOException { - ManifestWriter writer = ManifestFiles.write(ICEBERG_FORMAT_VERSION, spec, outputFile, DUMMY_SNAPSHOT_ID); + ManifestWriter writer = ManifestFiles.write(FORMAT_V2, spec, outputFile, DUMMY_SNAPSHOT_ID); try (ManifestWriter closeableWriter = writer) { closeableWriter.addAll(dataFiles); @@ -63,4 +66,69 @@ static ManifestOutputFileFactory createOutputFileFactory(Table table, String fli TableOperations ops = ((HasTableOperations) table).operations(); return new ManifestOutputFileFactory(ops, table.io(), table.properties(), flinkJobId, subTaskId, attemptNumber); } + + static DeltaManifests writeCompletedFiles(WriterResult result, + Supplier outputFileSupplier, + PartitionSpec spec) throws IOException { + + ManifestFile dataManifest = null; + ManifestFile deleteManifest = null; + + // Write the completed data files into a newly created data manifest file. + if (result.dataFiles() != null && result.dataFiles().length > 0) { + OutputFile dataManifestFile = outputFileSupplier.get(); + + ManifestWriter writer = + ManifestFiles.write(FORMAT_V2, spec, dataManifestFile, DUMMY_SNAPSHOT_ID); + try (ManifestWriter closeableWriter = writer) { + for (DataFile dataFile : result.dataFiles()) { + closeableWriter.add(dataFile); + } + } + + dataManifest = writer.toManifestFile(); + } + + // Write the completed delete files into a newly created delete manifest file. + if (result.deleteFiles() != null && result.deleteFiles().length > 0) { + OutputFile deleteManifestFile = outputFileSupplier.get(); + + ManifestWriter writer = + ManifestFiles.writeDeleteManifest(FORMAT_V2, spec, deleteManifestFile, DUMMY_SNAPSHOT_ID); + try (ManifestWriter closeableWriter = writer) { + for (DeleteFile deleteFile : result.deleteFiles()) { + closeableWriter.add(deleteFile); + } + } + + deleteManifest = writer.toManifestFile(); + } + + return new DeltaManifests(dataManifest, deleteManifest); + } + + static WriterResult readCompletedFiles(DeltaManifests deltaManifests, FileIO io) throws IOException { + WriterResult.Builder builder = WriterResult.builder(); + + // Read the completed data files from persisted data manifest file. + if (deltaManifests.dataManifest() != null) { + try (CloseableIterable dataFiles = ManifestFiles.read(deltaManifests.dataManifest(), io)) { + for (DataFile dataFile : dataFiles) { + builder.add(dataFile); + } + } + } + + // Read the completed delete files from persisted delete manifests file. + if (deltaManifests.deleteManifest() != null) { + try (CloseableIterable deleteFiles = ManifestFiles + .readDeleteManifest(deltaManifests.deleteManifest(), io, null)) { + for (DeleteFile deleteFile : deleteFiles) { + builder.add(deleteFile); + } + } + } + + return builder.build(); + } } diff --git a/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java index 7f1d9ca021fa..e927b73c839e 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.io.UncheckedIOException; +import java.util.List; import java.util.Locale; import java.util.Map; import org.apache.flink.api.common.functions.MapFunction; @@ -42,7 +43,9 @@ import org.apache.iceberg.Table; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.WriterResult; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.PropertyUtil; @@ -112,6 +115,7 @@ public static class Builder { private TableSchema tableSchema; private boolean overwrite = false; private Integer writeParallelism = null; + private List equalityFieldNames = null; private Builder() { } @@ -168,6 +172,11 @@ public Builder writeParallelism(int newWriteParallelism) { return this; } + public Builder equalityFieldNames(List columnNames) { + this.equalityFieldNames = columnNames; + return this; + } + @SuppressWarnings("unchecked") public DataStreamSink build() { Preconditions.checkArgument(rowDataInput != null, @@ -183,13 +192,22 @@ public DataStreamSink build() { } } - IcebergStreamWriter streamWriter = createStreamWriter(table, tableSchema); + List equalityFieldIds = Lists.newArrayList(); + if (equalityFieldNames != null && !equalityFieldNames.isEmpty()) { + for (String fieldName : equalityFieldNames) { + org.apache.iceberg.types.Types.NestedField field = table.schema().findField(fieldName); + Preconditions.checkNotNull(field, "Field with name '%s' does not exist", fieldName); + equalityFieldIds.add(field.fieldId()); + } + } + + IcebergStreamWriter streamWriter = createStreamWriter(table, tableSchema, equalityFieldIds); IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(tableLoader, overwrite); this.writeParallelism = writeParallelism == null ? rowDataInput.getParallelism() : writeParallelism; DataStream returnStream = rowDataInput - .transform(ICEBERG_STREAM_WRITER_NAME, TypeInformation.of(DataFile.class), streamWriter) + .transform(ICEBERG_STREAM_WRITER_NAME, TypeInformation.of(WriterResult.class), streamWriter) .setParallelism(writeParallelism) .transform(ICEBERG_FILES_COMMITTER_NAME, Types.VOID, filesCommitter) .setParallelism(1) @@ -201,7 +219,8 @@ public DataStreamSink build() { } } - static IcebergStreamWriter createStreamWriter(Table table, TableSchema requestedSchema) { + static IcebergStreamWriter createStreamWriter(Table table, TableSchema requestedSchema, + List equalityFieldIds) { Preconditions.checkArgument(table != null, "Iceberg table should't be null"); RowType flinkSchema; @@ -224,7 +243,8 @@ static IcebergStreamWriter createStreamWriter(Table table, TableSchema FileFormat fileFormat = getFileFormat(props); TaskWriterFactory taskWriterFactory = new RowDataTaskWriterFactory(table.schema(), flinkSchema, - table.spec(), table.locationProvider(), table.io(), table.encryption(), targetFileSize, fileFormat, props); + table.spec(), table.locationProvider(), table.io(), table.encryption(), targetFileSize, fileFormat, props, + equalityFieldIds); return new IcebergStreamWriter<>(table.name(), taskWriterFactory); } diff --git a/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java index ad890613a073..c53e1d6f515d 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java @@ -38,18 +38,20 @@ import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.table.runtime.typeutils.SortedMapTypeInfo; -import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.ReplacePartitions; +import org.apache.iceberg.RowDelta; import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotUpdate; import org.apache.iceberg.Table; import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.WriterResult; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.base.Strings; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Comparators; @@ -58,7 +60,7 @@ import org.slf4j.LoggerFactory; class IcebergFilesCommitter extends AbstractStreamOperator - implements OneInputStreamOperator, BoundedOneInput { + implements OneInputStreamOperator, BoundedOneInput { private static final long serialVersionUID = 1L; private static final long INITIAL_CHECKPOINT_ID = -1L; @@ -85,9 +87,9 @@ class IcebergFilesCommitter extends AbstractStreamOperator // iceberg table when the next checkpoint happen. private final NavigableMap dataFilesPerCheckpoint = Maps.newTreeMap(); - // The data files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the - // 'dataFilesPerCheckpoint'. - private final List dataFilesOfCurrentCheckpoint = Lists.newArrayList(); + // The completed files cache for current checkpoint. Once the snapshot barrier received, it will be flushed to the + // 'writerResultsOfCurrentCkpt'. + private List writerResultsOfCurrentCkpt = Lists.newArrayList(); // It will have an unique identifier for one job. private transient String flinkJobId; @@ -165,7 +167,7 @@ public void snapshotState(StateSnapshotContext context) throws Exception { jobIdState.add(flinkJobId); // Clear the local buffer for current checkpoint. - dataFilesOfCurrentCheckpoint.clear(); + writerResultsOfCurrentCkpt.clear(); } @Override @@ -184,36 +186,37 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { } } - private void commitUpToCheckpoint(NavigableMap manifestsMap, + private void commitUpToCheckpoint(NavigableMap deltaManifestsMap, String newFlinkJobId, long checkpointId) throws IOException { - NavigableMap pendingManifestMap = manifestsMap.headMap(checkpointId, true); + NavigableMap pendingMap = deltaManifestsMap.headMap(checkpointId, true); - List manifestFiles = Lists.newArrayList(); - List pendingDataFiles = Lists.newArrayList(); - for (byte[] manifestData : pendingManifestMap.values()) { - if (Arrays.equals(EMPTY_MANIFEST_DATA, manifestData)) { + List deltaManifestsList = Lists.newArrayList(); + WriterResult.Builder resultBuilder = WriterResult.builder(); + for (byte[] serializedData : pendingMap.values()) { + if (Arrays.equals(EMPTY_MANIFEST_DATA, serializedData)) { // Skip the empty flink manifest. continue; } - ManifestFile manifestFile = - SimpleVersionedSerialization.readVersionAndDeSerialize(FlinkManifestSerializer.INSTANCE, manifestData); + DeltaManifests deltaManifests = + SimpleVersionedSerialization.readVersionAndDeSerialize(DeltaManifestsSerializer.INSTANCE, serializedData); - manifestFiles.add(manifestFile); - pendingDataFiles.addAll(FlinkManifestUtil.readDataFiles(manifestFile, table.io())); + deltaManifestsList.add(deltaManifests); + resultBuilder.add(FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io())); } + WriterResult result = resultBuilder.build(); if (replacePartitions) { - replacePartitions(pendingDataFiles, newFlinkJobId, checkpointId); + commitOverwriteTxn(result, newFlinkJobId, checkpointId); } else { - append(pendingDataFiles, newFlinkJobId, checkpointId); + commitRowDeltaTxn(result, newFlinkJobId, checkpointId); } - pendingManifestMap.clear(); + pendingMap.clear(); // Delete the committed manifests and clear the committed data files from dataFilesPerCheckpoint. - for (ManifestFile manifestFile : manifestFiles) { + for (ManifestFile manifestFile : Iterables.concat(deltaManifestsList)) { try { table.io().deleteFile(manifestFile.path()); } catch (Exception e) { @@ -222,6 +225,7 @@ private void commitUpToCheckpoint(NavigableMap manifestsMap, .add("flinkJobId", newFlinkJobId) .add("checkpointId", checkpointId) .add("manifestPath", manifestFile.path()) + .add("content", manifestFile.content()) .toString(); LOG.warn("The iceberg transaction has been committed, but we failed to clean the temporary flink manifests: {}", details, e); @@ -229,11 +233,14 @@ private void commitUpToCheckpoint(NavigableMap manifestsMap, } } - private void replacePartitions(List dataFiles, String newFlinkJobId, long checkpointId) { + private void commitOverwriteTxn(WriterResult result, String newFlinkJobId, long checkpointId) { + Preconditions.checkState(result.deleteFiles() == null || result.deleteFiles().length == 0, + "Could not overwrite partitions with delete files."); + ReplacePartitions dynamicOverwrite = table.newReplacePartitions(); int numFiles = 0; - for (DataFile file : dataFiles) { + for (DataFile file : result.dataFiles()) { numFiles += 1; dynamicOverwrite.addFile(file); } @@ -241,16 +248,26 @@ private void replacePartitions(List dataFiles, String newFlinkJobId, l commitOperation(dynamicOverwrite, numFiles, "dynamic partition overwrite", newFlinkJobId, checkpointId); } - private void append(List dataFiles, String newFlinkJobId, long checkpointId) { - AppendFiles appendFiles = table.newAppend(); + private void commitRowDeltaTxn(WriterResult result, String newFlinkJobId, long checkpointId) { + RowDelta rowDelta = table.newRowDelta(); int numFiles = 0; - for (DataFile file : dataFiles) { - numFiles += 1; - appendFiles.appendFile(file); + + if (result.dataFiles() != null && result.dataFiles().length > 0) { + for (DataFile file : result.dataFiles()) { + numFiles += 1; + rowDelta.addRows(file); + } + } + + if (result.deleteFiles() != null && result.deleteFiles().length > 0) { + for (DeleteFile file : result.deleteFiles()) { + numFiles += 1; + rowDelta.addDeletes(file); + } } - commitOperation(appendFiles, numFiles, "append", newFlinkJobId, checkpointId); + commitOperation(rowDelta, numFiles, "rowDelta", newFlinkJobId, checkpointId); } private void commitOperation(SnapshotUpdate operation, int numFiles, String description, @@ -266,8 +283,8 @@ private void commitOperation(SnapshotUpdate operation, int numFiles, String d } @Override - public void processElement(StreamRecord element) { - this.dataFilesOfCurrentCheckpoint.add(element.getValue()); + public void processElement(StreamRecord element) { + this.writerResultsOfCurrentCkpt.add(element.getValue()); } @Override @@ -275,7 +292,7 @@ public void endInput() throws IOException { // Flush the buffered data files into 'dataFilesPerCheckpoint' firstly. long currentCheckpointId = Long.MAX_VALUE; dataFilesPerCheckpoint.put(currentCheckpointId, writeToManifest(currentCheckpointId)); - dataFilesOfCurrentCheckpoint.clear(); + writerResultsOfCurrentCkpt.clear(); commitUpToCheckpoint(dataFilesPerCheckpoint, flinkJobId, currentCheckpointId); } @@ -284,14 +301,19 @@ public void endInput() throws IOException { * Write all the complete data files to a newly created manifest file and return the manifest's avro serialized bytes. */ private byte[] writeToManifest(long checkpointId) throws IOException { - if (dataFilesOfCurrentCheckpoint.isEmpty()) { + if (writerResultsOfCurrentCkpt.isEmpty()) { return EMPTY_MANIFEST_DATA; } - OutputFile manifestOutputFile = manifestOutputFileFactory.create(checkpointId); - ManifestFile manifestFile = - FlinkManifestUtil.writeDataFiles(manifestOutputFile, table.spec(), dataFilesOfCurrentCheckpoint); - return SimpleVersionedSerialization.writeVersionAndSerialize(FlinkManifestSerializer.INSTANCE, manifestFile); + WriterResult.Builder builder = WriterResult.builder(); + for (WriterResult writerResult : writerResultsOfCurrentCkpt) { + builder.add(writerResult); + } + + DeltaManifests deltaManifests = FlinkManifestUtil.writeCompletedFiles(builder.build(), + () -> manifestOutputFileFactory.create(checkpointId), table.spec()); + + return SimpleVersionedSerialization.writeVersionAndSerialize(DeltaManifestsSerializer.INSTANCE, deltaManifests); } @Override diff --git a/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java b/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java index 95daa9656c68..713741c21ed9 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java @@ -25,12 +25,12 @@ import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.iceberg.DataFile; import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.WriterResult; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -class IcebergStreamWriter extends AbstractStreamOperator - implements OneInputStreamOperator, BoundedOneInput { +class IcebergStreamWriter extends AbstractStreamOperator + implements OneInputStreamOperator, BoundedOneInput { private static final long serialVersionUID = 1L; @@ -62,9 +62,7 @@ public void open() { @Override public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { // close all open files and emit files to downstream committer operator - for (DataFile dataFile : writer.complete()) { - emit(dataFile); - } + emit(writer.complete()); this.writer = taskWriterFactory.create(); } @@ -87,9 +85,7 @@ public void dispose() throws Exception { public void endInput() throws IOException { // For bounded stream, it may don't enable the checkpoint mechanism so we'd better to emit the remaining // data files to downstream before closing the writer so that we won't miss any of them. - for (DataFile dataFile : writer.complete()) { - emit(dataFile); - } + emit(writer.complete()); } @Override @@ -101,7 +97,7 @@ public String toString() { .toString(); } - private void emit(DataFile dataFile) { - output.collect(new StreamRecord<>(dataFile)); + private void emit(WriterResult writerResult) { + output.collect(new StreamRecord<>(writerResult)); } } diff --git a/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedFanoutWriter.java b/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedFanoutWriter.java deleted file mode 100644 index ad846974adcf..000000000000 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedFanoutWriter.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * 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.sink; - -import java.io.IOException; -import java.util.Map; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.io.BaseTaskWriter; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.OutputFileFactory; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; - -abstract class PartitionedFanoutWriter extends BaseTaskWriter { - private final Map writers = Maps.newHashMap(); - - PartitionedFanoutWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, FileIO io, long targetFileSize) { - super(spec, format, appenderFactory, fileFactory, io, targetFileSize); - } - - /** - * Create a PartitionKey from the values in row. - *

- * Any PartitionKey returned by this method can be reused by the implementation. - * - * @param row a data row - */ - protected abstract PartitionKey partition(T row); - - @Override - public void write(T row) throws IOException { - PartitionKey partitionKey = partition(row); - - RollingFileWriter writer = writers.get(partitionKey); - if (writer == null) { - // NOTICE: we need to copy a new partition key here, in case of messing up the keys in writers. - PartitionKey copiedKey = partitionKey.copy(); - writer = new RollingFileWriter(copiedKey); - writers.put(copiedKey, writer); - } - - writer.add(row); - } - - @Override - public void close() throws IOException { - if (!writers.isEmpty()) { - for (PartitionKey key : writers.keySet()) { - writers.get(key).close(); - } - writers.clear(); - } - } -} diff --git a/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriter.java b/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriter.java new file mode 100644 index 000000000000..a2bc91b62819 --- /dev/null +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriter.java @@ -0,0 +1,159 @@ +/* + * 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.sink; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.List; +import java.util.Map; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.io.DeltaWriter; +import org.apache.iceberg.io.DeltaWriterFactory; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.WriterResult; +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.Tasks; + +class RowDataTaskWriter implements TaskWriter { + + private final PartitionSpec spec; + private final FlinkDeltaWriterFactory deltaWriterFactory; + private final PartitionKey partitionKey; + private final RowDataWrapper rowDataWrapper; + + private final Map> deltaWriterMap; + private final DeltaWriterFactory.Context ctxt; + + RowDataTaskWriter(Schema schema, + RowType flinkSchema, + PartitionSpec spec, + FileFormat format, + OutputFileFactory fileFactory, + FileIO io, + long targetFileSize, + Map tableProperties, + List equalityFieldIds) { + this.deltaWriterFactory = new FlinkDeltaWriterFactory(schema, flinkSchema, spec, + format, fileFactory, io, targetFileSize, tableProperties); + + this.spec = spec; + this.partitionKey = new PartitionKey(spec, schema); + this.rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); + + this.deltaWriterMap = Maps.newHashMap(); + + if (equalityFieldIds != null && equalityFieldIds.size() > 0) { + this.ctxt = DeltaWriterFactory.Context.builder() + .allowEqualityDelete(true) + .equalityFieldIds(equalityFieldIds) + .eqDeleteRowSchema(TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds))) + .build(); + } else { + this.ctxt = DeltaWriterFactory.Context.builder().build(); + } + } + + @Override + public void write(RowData row) { + DeltaWriter deltaWriter; + + if (spec.fields().size() <= 0) { + // Create and cache the delta writer if absent for unpartitioned table. + deltaWriter = deltaWriterMap.get(partitionKey); + if (deltaWriter == null) { + deltaWriter = deltaWriterFactory.createDeltaWriter(null, ctxt); + deltaWriterMap.put(partitionKey, deltaWriter); + } + } else { + // Refresh the current partition key. + partitionKey.partition(rowDataWrapper.wrap(row)); + + // Create and cache the delta writer if absent for partitioned table. + deltaWriter = deltaWriterMap.get(partitionKey); + if (deltaWriter == null) { + // NOTICE: we need to copy a new partition key here, in case of messing up the keys in writers. + PartitionKey copiedKey = partitionKey.copy(); + deltaWriter = deltaWriterFactory.createDeltaWriter(copiedKey, ctxt); + deltaWriterMap.put(copiedKey, deltaWriter); + } + } + + switch (row.getRowKind()) { + case INSERT: + case UPDATE_AFTER: + deltaWriter.writeRow(row); + break; + + case DELETE: + case UPDATE_BEFORE: + deltaWriter.writeEqualityDelete(row); + break; + + default: + throw new UnsupportedOperationException("Unknown row kind: " + row.getRowKind()); + } + } + + @Override + public void abort() { + close(); + + Tasks.foreach(deltaWriterMap.values()) + .throwFailureWhenFinished() + .noRetry() + .run(DeltaWriter::abort); + } + + @Override + public WriterResult complete() { + close(); + + WriterResult.Builder builder = WriterResult.builder(); + for (DeltaWriter writer : deltaWriterMap.values()) { + builder.add(writer.complete()); + } + + return builder.build(); + } + + @Override + public void close() { + Tasks.foreach(deltaWriterMap.values()) + .throwFailureWhenFinished() + .noRetry() + .run(deltaWriter -> { + try { + deltaWriter.close(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + }); + } +} diff --git a/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java b/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java index 72e7a7941b77..e066824bb3ba 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java @@ -19,33 +19,18 @@ package org.apache.iceberg.flink.sink; -import java.io.IOException; -import java.io.Serializable; -import java.io.UncheckedIOException; +import java.util.List; import java.util.Map; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.FileFormat; -import org.apache.iceberg.MetricsConfig; -import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; -import org.apache.iceberg.avro.Avro; import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.flink.RowDataWrapper; -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.io.FileAppender; -import org.apache.iceberg.io.FileAppenderFactory; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; -import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.io.OutputFileFactory; import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.UnpartitionedWriter; -import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; public class RowDataTaskWriterFactory implements TaskWriterFactory { @@ -57,7 +42,8 @@ public class RowDataTaskWriterFactory implements TaskWriterFactory { private final EncryptionManager encryptionManager; private final long targetFileSizeBytes; private final FileFormat format; - private final FileAppenderFactory appenderFactory; + private final Map tableProperties; + private final List equalityFieldIds; private transient OutputFileFactory outputFileFactory; @@ -69,7 +55,8 @@ public RowDataTaskWriterFactory(Schema schema, EncryptionManager encryptionManager, long targetFileSizeBytes, FileFormat format, - Map tableProperties) { + Map tableProperties, + List equalityFieldIds) { this.schema = schema; this.flinkSchema = flinkSchema; this.spec = spec; @@ -78,7 +65,8 @@ public RowDataTaskWriterFactory(Schema schema, this.encryptionManager = encryptionManager; this.targetFileSizeBytes = targetFileSizeBytes; this.format = format; - this.appenderFactory = new FlinkFileAppenderFactory(schema, flinkSchema, tableProperties); + this.tableProperties = tableProperties; + this.equalityFieldIds = equalityFieldIds; } @Override @@ -91,81 +79,7 @@ public TaskWriter create() { Preconditions.checkNotNull(outputFileFactory, "The outputFileFactory shouldn't be null if we have invoked the initialize()."); - if (spec.fields().isEmpty()) { - return new UnpartitionedWriter<>(spec, format, appenderFactory, outputFileFactory, io, targetFileSizeBytes); - } else { - return new RowDataPartitionedFanoutWriter(spec, format, appenderFactory, outputFileFactory, - io, targetFileSizeBytes, schema, flinkSchema); - } - } - - private static class RowDataPartitionedFanoutWriter extends PartitionedFanoutWriter { - - private final PartitionKey partitionKey; - private final RowDataWrapper rowDataWrapper; - - RowDataPartitionedFanoutWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, - RowType flinkSchema) { - super(spec, format, appenderFactory, fileFactory, io, targetFileSize); - this.partitionKey = new PartitionKey(spec, schema); - this.rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); - } - - @Override - protected PartitionKey partition(RowData row) { - partitionKey.partition(rowDataWrapper.wrap(row)); - return partitionKey; - } - } - - public static class FlinkFileAppenderFactory implements FileAppenderFactory, Serializable { - private final Schema schema; - private final RowType flinkSchema; - private final Map props; - - public FlinkFileAppenderFactory(Schema schema, RowType flinkSchema, Map props) { - this.schema = schema; - this.flinkSchema = flinkSchema; - this.props = props; - } - - @Override - public FileAppender newAppender(OutputFile outputFile, FileFormat format) { - MetricsConfig metricsConfig = MetricsConfig.fromProperties(props); - try { - switch (format) { - case AVRO: - return Avro.write(outputFile) - .createWriterFunc(ignore -> new FlinkAvroWriter(flinkSchema)) - .setAll(props) - .schema(schema) - .overwrite() - .build(); - - case ORC: - return ORC.write(outputFile) - .createWriterFunc((iSchema, typDesc) -> FlinkOrcWriter.buildWriter(flinkSchema, iSchema)) - .setAll(props) - .schema(schema) - .overwrite() - .build(); - - case PARQUET: - return Parquet.write(outputFile) - .createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(flinkSchema, msgType)) - .setAll(props) - .metricsConfig(metricsConfig) - .schema(schema) - .overwrite() - .build(); - - default: - throw new UnsupportedOperationException("Cannot write unknown file format: " + format); - } - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } + return new RowDataTaskWriter(schema, flinkSchema, spec, format, outputFileFactory, io, targetFileSizeBytes, + tableProperties, equalityFieldIds); } } diff --git a/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java b/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java index 65a3ca92c4d0..6e827b0f7e5c 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java +++ b/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java @@ -80,7 +80,8 @@ public RowDataRewriter(Table table, boolean caseSensitive, FileIO io, Encryption encryptionManager, Long.MAX_VALUE, format, - table.properties()); + table.properties(), + Lists.newArrayList()); /* TODO: will need to handle the delete case.*/ } public List rewriteDataForTasks(DataStream dataStream, int parallelism) { @@ -131,7 +132,7 @@ public List map(CombinedScanTask task) throws Exception { RowData rowData = iterator.next(); writer.write(rowData); } - return Lists.newArrayList(writer.complete()); + return Lists.newArrayList(writer.complete().dataFiles()); } catch (Throwable originalThrowable) { try { LOG.error("Aborting commit for (subTaskId {}, attemptId {})", subTaskId, attemptId); diff --git a/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java index dc14fb683a91..fdf53b4bc6b9 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java +++ b/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java @@ -39,7 +39,7 @@ import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.IcebergGenerics; import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.sink.RowDataTaskWriterFactory; +import org.apache.iceberg.flink.sink.FlinkFileAppenderFactory; import org.apache.iceberg.hadoop.HadoopInputFile; import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.io.CloseableIterable; @@ -102,8 +102,7 @@ public static DataFile writeFile(Schema schema, PartitionSpec spec, Configuratio Preconditions.checkNotNull(fileFormat, "Cannot determine format for file: %s", filename); RowType flinkSchema = FlinkSchemaUtil.convert(schema); - FileAppenderFactory appenderFactory = new RowDataTaskWriterFactory.FlinkFileAppenderFactory(schema, - flinkSchema, ImmutableMap.of()); + FileAppenderFactory appenderFactory = new FlinkFileAppenderFactory(schema, flinkSchema, ImmutableMap.of()); FileAppender appender = appenderFactory.newAppender(fromPath(path, conf), fileFormat); try (FileAppender closeableAppender = appender) { @@ -116,14 +115,18 @@ public static DataFile writeFile(Schema schema, PartitionSpec spec, Configuratio .build(); } - public static void assertTableRows(String tablePath, List expected) throws IOException { + public static void assertTableRows(Table table, List expected) throws IOException { List expectedRecords = Lists.newArrayList(); for (RowData row : expected) { Integer id = row.isNullAt(0) ? null : row.getInt(0); String data = row.isNullAt(1) ? null : row.getString(1).toString(); expectedRecords.add(createRecord(id, data)); } - assertTableRecords(tablePath, expectedRecords); + assertTableRecords(table, expectedRecords); + } + + public static void assertTableRows(String tablePath, List expected) throws IOException { + assertTableRows(new HadoopTables().load(tablePath), expected); } public static void assertTableRecords(Table table, List expected) throws IOException { diff --git a/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkDeltaWriter.java b/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkDeltaWriter.java new file mode 100644 index 000000000000..83179e96c7a0 --- /dev/null +++ b/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkDeltaWriter.java @@ -0,0 +1,362 @@ +/* + * 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.sink; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Locale; +import java.util.Set; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.AssertHelpers; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestTables; +import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.data.RandomRowData; +import org.apache.iceberg.io.DeltaWriter; +import org.apache.iceberg.io.DeltaWriterFactory; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.io.WriterResult; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.StructLikeSet; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.apache.iceberg.flink.SimpleDataUtil.ROW_TYPE; +import static org.apache.iceberg.flink.SimpleDataUtil.createRowData; + +@RunWith(Parameterized.class) +public class TestFlinkDeltaWriter extends TableTestBase { + private static final String TABLE_NAME = "flink_delta_table"; + private static final int FORMAT_V2 = 2; + + @Rule + public final TemporaryFolder tempFolder = new TemporaryFolder(); + + @Parameterized.Parameters(name = "format = {0}, partitioned = {1}") + public static Object[][] parameters() { + return new Object[][] { + {"avro", false}, + {"parquet", false}, + }; + } + + private final FileFormat format; + private final boolean partitioned; + private Table table; + + public TestFlinkDeltaWriter(String format, boolean partitioned) { + super(FORMAT_V2); + this.format = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH)); + this.partitioned = partitioned; + } + + @Before + public void before() throws IOException { + File tableDir = tempFolder.newFolder(); + Assert.assertTrue(tableDir.delete()); + + if (partitioned) { + this.table = TestTables.create(tableDir, TABLE_NAME, SCHEMA, SPEC, formatVersion); + } else { + this.table = TestTables.create(tableDir, TABLE_NAME, SCHEMA, PartitionSpec.unpartitioned(), formatVersion); + } + } + + @Test + public void testWritePureInsert() { + DeltaWriterFactory writerFactory = createDeltaWriterFactory(); + + DeltaWriterFactory.Context ctxt = DeltaWriterFactory.Context.builder() + .allowPosDelete(false) + .allowEqualityDelete(false) + .build(); + + DeltaWriter deltaWriter = writerFactory.createDeltaWriter(null, ctxt); + + Iterable expectedRecords = RandomGenericData.generate(SCHEMA, 100, 22112234L); + for (RowData record : RandomRowData.generate(SCHEMA, 100, 22112234L)) { + deltaWriter.writeRow(record); + } + + WriterResult result = deltaWriter.complete(); + + Assert.assertEquals(result.dataFiles().length, 1); + Assert.assertEquals(result.deleteFiles().length, 0); + + commitTransaction(result); + + Assert.assertEquals(Sets.newHashSet(expectedRecords), Sets.newHashSet(IcebergGenerics.read(table).build())); + } + + @Test + public void testWriteEqualityDelete() { + DeltaWriterFactory writerFactory = createDeltaWriterFactory(); + + List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); + DeltaWriterFactory.Context ctxt = DeltaWriterFactory.Context.builder() + .allowEqualityDelete(true) + .equalityFieldIds(equalityFieldIds) + .eqDeleteRowSchema(table.schema().select("id")) + .build(); + + // TODO More unit tests to test the partitioned case. + DeltaWriter deltaWriter = writerFactory.createDeltaWriter(null, ctxt); + + RowData record1 = createRowData(1, "aaa"); + RowData record2 = createRowData(2, "bbb"); + RowData record3 = createRowData(3, "ccc"); + + deltaWriter.writeRow(record1); + deltaWriter.writeRow(record2); + + deltaWriter.writeEqualityDelete(record1); + deltaWriter.writeEqualityDelete(record2); + + deltaWriter.writeRow(record3); + + WriterResult result = deltaWriter.complete(); + Assert.assertEquals(result.dataFiles().length, 1); + Assert.assertEquals(result.deleteFiles().length, 1); + commitTransaction(result); + + assertTableRecords(Sets.newHashSet(record3)); + + deltaWriter = writerFactory.createDeltaWriter(null, ctxt); + deltaWriter.writeEqualityDelete(record3); + + result = deltaWriter.complete(); + Assert.assertEquals(result.dataFiles().length, 0); + Assert.assertEquals(result.deleteFiles().length, 1); + commitTransaction(result); + + assertTableRecords(ImmutableSet.of()); + } + + @Test + public void testEqualityDeleteSameRow() { + DeltaWriterFactory writerFactory = createDeltaWriterFactory(); + + List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); + DeltaWriterFactory.Context ctxt = DeltaWriterFactory.Context.builder() + .allowEqualityDelete(true) + .equalityFieldIds(equalityFieldIds) + .eqDeleteRowSchema(table.schema()) + .posDeleteRowSchema(table.schema()) + .build(); + + DeltaWriter deltaWriter1 = writerFactory.createDeltaWriter(null, ctxt); + + RowData record1 = createRowData(1, "aaa"); + + deltaWriter1.writeRow(record1); + deltaWriter1.writeEqualityDelete(record1); + deltaWriter1.writeRow(record1); + deltaWriter1.writeEqualityDelete(record1); + deltaWriter1.writeRow(record1); + + AssertHelpers.assertThrows("Encountered duplicated keys in the same transaction", + ValidationException.class, () -> deltaWriter1.writeRow(record1)); + + WriterResult result = deltaWriter1.complete(); + Assert.assertEquals(result.dataFiles().length, 1); + Assert.assertEquals(result.deleteFiles().length, 1); + commitTransaction(result); + + assertTableRecords(ImmutableSet.of(record1)); + + DeltaWriter deltaWriter2 = writerFactory.createDeltaWriter(null, ctxt); + deltaWriter2.writeRow(record1); + + result = deltaWriter2.complete(); + Assert.assertEquals(result.dataFiles().length, 1); + Assert.assertEquals(result.deleteFiles().length, 0); + commitTransaction(result); + + assertTableRecords(ImmutableSet.of(record1, record1)); + } + + @Test + public void testPositionDelete() { + DeltaWriterFactory writerFactory = createDeltaWriterFactory(); + DeltaWriterFactory.Context ctxt = DeltaWriterFactory.Context.builder() + .allowPosDelete(true) + .build(); + + DeltaWriter deltaWriter = writerFactory.createDeltaWriter(null, ctxt); + + RowData record1 = createRowData(1, "aaa"); + RowData record2 = createRowData(2, "bbb"); + RowData record3 = createRowData(3, "ccc"); + + // Write two records. + deltaWriter.writeRow(record1); + deltaWriter.writeRow(record2); + + WriterResult result = deltaWriter.complete(); + Assert.assertEquals(result.dataFiles().length, 1); + Assert.assertEquals(result.deleteFiles().length, 0); + commitTransaction(result); + + CharSequence dataFilePath = result.dataFiles()[0].path(); + + // Delete the second record. + deltaWriter = writerFactory.createDeltaWriter(null, ctxt); + deltaWriter.writePosDelete(dataFilePath, 1); + + result = deltaWriter.complete(); + Assert.assertEquals(result.dataFiles().length, 0); + Assert.assertEquals(result.deleteFiles().length, 1); + commitTransaction(result); + + assertTableRecords(ImmutableSet.of(record1)); + + // Delete the first record. + deltaWriter = writerFactory.createDeltaWriter(null, ctxt); + deltaWriter.writePosDelete(dataFilePath, 0); + deltaWriter.writeRow(record3); + + result = deltaWriter.complete(); + Assert.assertEquals(result.dataFiles().length, 1); + Assert.assertEquals(result.deleteFiles().length, 1); + commitTransaction(result); + + assertTableRecords(ImmutableSet.of(record3)); + } + + @Test + public void testUpsertSameRow() { + DeltaWriterFactory writerFactory = createDeltaWriterFactory(); + List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); + DeltaWriterFactory.Context ctxt = DeltaWriterFactory.Context.builder() + .allowEqualityDelete(true) + .equalityFieldIds(equalityFieldIds) + .eqDeleteRowSchema(table.schema().select("id")) + .build(); + DeltaWriter deltaWriter1 = writerFactory.createDeltaWriter(null, ctxt); + + RowData record1 = createRowData(1, "aaa"); + RowData record2 = createRowData(1, "bbb"); + RowData record3 = createRowData(1, "ccc"); + RowData record4 = createRowData(1, "ddd"); + RowData record5 = createRowData(1, "eee"); + RowData record6 = createRowData(1, "fff"); + RowData record7 = createRowData(1, "ggg"); + + deltaWriter1.writeRow(record1); + AssertHelpers.assertThrows("Duplicated keys", ValidationException.class, + () -> deltaWriter1.writeRow(record2)); + + // Commit the transaction. + WriterResult result = deltaWriter1.complete(); + Assert.assertEquals(result.dataFiles().length, 1); + Assert.assertEquals(result.deleteFiles().length, 0); + commitTransaction(result); + + assertTableRecords(ImmutableSet.of(record1)); + + DeltaWriter deltaWriter2 = writerFactory.createDeltaWriter(null, ctxt); + + // UPSERT (1, "ccc") + deltaWriter2.writeEqualityDelete(record3); + deltaWriter2.writeRow(record3); + + // INSERT (1, "ddd") + // INSERT (1, "eee") + AssertHelpers.assertThrows("Duplicated keys", ValidationException.class, + () -> deltaWriter2.writeRow(record4)); + AssertHelpers.assertThrows("Duplicated keys", ValidationException.class, + () -> deltaWriter2.writeRow(record5)); + + // UPSERT (1, "fff") + deltaWriter2.writeEqualityDelete(record6); + deltaWriter2.writeRow(record6); + + // INSERT (1, "ggg") + AssertHelpers.assertThrows("Duplicated keys", ValidationException.class, + () -> deltaWriter2.writeRow(record7)); + + // Commit the transaction. + result = deltaWriter2.complete(); + Assert.assertEquals(1, result.dataFiles().length); + // One pos-delete file, and one equality-delete file. + Assert.assertEquals(2, result.deleteFiles().length); + commitTransaction(result); + + assertTableRecords(ImmutableSet.of(record6)); + } + + private void assertTableRecords(Set expectedRowDataSet) { + StructLikeSet expectedSet = StructLikeSet.create(SCHEMA.asStruct()); + for (RowData rowData : expectedRowDataSet) { + RowDataWrapper wrapper = new RowDataWrapper(ROW_TYPE, SCHEMA.asStruct()); + expectedSet.add(wrapper.wrap(rowData)); + } + + StructLikeSet actualSet = StructLikeSet.create(SCHEMA.asStruct()); + Iterables.addAll(actualSet, IcebergGenerics.read(table).build()); + Assert.assertEquals(expectedSet, actualSet); + } + + private void commitTransaction(WriterResult result) { + RowDelta rowDelta = table.newRowDelta(); + + for (DataFile dataFile : result.dataFiles()) { + rowDelta.addRows(dataFile); + } + + for (DeleteFile deleteFile : result.deleteFiles()) { + rowDelta.addDeletes(deleteFile); + } + + rowDelta.commit(); + } + + private DeltaWriterFactory createDeltaWriterFactory() { + OutputFileFactory outputFileFactory = + new OutputFileFactory(table.spec(), format, table.locationProvider(), table.io(), + table.encryption(), 1, 1); + + RowType flinkSchema = FlinkSchemaUtil.convert(table.schema()); + return new FlinkDeltaWriterFactory(table.schema(), flinkSchema, table.spec(), format, outputFileFactory, table.io(), + 128 * 1024 * 1024L, table.properties() + ); + } +} diff --git a/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java index 0022425d8447..4f8be0955ba1 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java +++ b/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java @@ -23,10 +23,11 @@ import java.io.IOException; import java.util.List; import java.util.Locale; -import java.util.Map; +import java.util.Objects; import java.util.stream.Collectors; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.util.FiniteTestSource; @@ -34,35 +35,57 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.util.DataFormatConverters; import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo; -import org.apache.flink.test.util.AbstractTestBase; +import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.flink.types.Row; +import org.apache.flink.types.RowKind; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestTables; +import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.junit.Assert; +import org.junit.Assume; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import static org.apache.iceberg.flink.SimpleDataUtil.FLINK_SCHEMA; +import static org.apache.iceberg.flink.SimpleDataUtil.ROW_TYPE; +import static org.apache.iceberg.flink.SimpleDataUtil.createRecord; + @RunWith(Parameterized.class) -public class TestFlinkIcebergSink extends AbstractTestBase { +public class TestFlinkIcebergSink extends TableTestBase { private static final TypeInformation ROW_TYPE_INFO = new RowTypeInfo( - SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); + FLINK_SCHEMA.getFieldTypes()); private static final DataFormatConverters.RowConverter CONVERTER = new DataFormatConverters.RowConverter( - SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); + FLINK_SCHEMA.getFieldDataTypes()); + + private static final int FORMAT_V2 = 2; + private static final String TABLE_NAME = "flink_delta_table"; + + @ClassRule + public static MiniClusterWithClientResource miniClusterResource = new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(4) + .build()); @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); - private String tablePath; private Table table; private StreamExecutionEnvironment env; private TableLoader tableLoader; @@ -90,6 +113,7 @@ public static Object[][] parameters() { } public TestFlinkIcebergSink(String format, int parallelism, boolean partitioned) { + super(FORMAT_V2); this.format = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH)); this.parallelism = parallelism; this.partitioned = partitioned; @@ -97,21 +121,30 @@ public TestFlinkIcebergSink(String format, int parallelism, boolean partitioned) @Before public void before() throws IOException { - File folder = tempFolder.newFolder(); - String warehouse = folder.getAbsolutePath(); + File tableDir = tempFolder.newFolder(); + Assert.assertTrue(tableDir.delete()); + + // Create 'data' directory firstly so that the parallelism writers won't conflict with it. + File dataTableDir = new File(tableDir, "data"); + Assert.assertTrue(dataTableDir.mkdirs()); - tablePath = warehouse.concat("/test"); - Assert.assertTrue("Should create the table path correctly.", new File(tablePath).mkdir()); + if (partitioned) { + PartitionSpec spec = PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("data", 16).build(); + this.table = TestTables.create(tableDir, TABLE_NAME, SimpleDataUtil.SCHEMA, spec, formatVersion); + } else { + this.table = TestTables.create(tableDir, TABLE_NAME, SimpleDataUtil.SCHEMA, + PartitionSpec.unpartitioned(), formatVersion); + } - Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); - table = SimpleDataUtil.createTable(tablePath, props, partitioned); + // Update table's properties to the given file format. + table.updateProperties().set(TableProperties.DEFAULT_FILE_FORMAT, format.name()).commit(); env = StreamExecutionEnvironment.getExecutionEnvironment() .enableCheckpointing(100) .setParallelism(parallelism) .setMaxParallelism(parallelism); - tableLoader = TableLoader.fromHadoopTable(tablePath); + tableLoader = new TestTableLoader(tableDir); } private List convertToRowData(List rows) { @@ -126,7 +159,7 @@ public void testWriteRowData() throws Exception { Row.of(3, "foo") ); DataStream dataStream = env.addSource(new FiniteTestSource<>(rows), ROW_TYPE_INFO) - .map(CONVERTER::toInternal, RowDataTypeInfo.of(SimpleDataUtil.ROW_TYPE)); + .map(CONVERTER::toInternal, RowDataTypeInfo.of(ROW_TYPE)); FlinkSink.forRowData(dataStream) .table(table) @@ -140,7 +173,7 @@ public void testWriteRowData() throws Exception { // Assert the iceberg table's records. NOTICE: the FiniteTestSource will checkpoint the same rows twice, so it will // commit the same row list into iceberg twice. List expectedRows = Lists.newArrayList(Iterables.concat(convertToRowData(rows), convertToRowData(rows))); - SimpleDataUtil.assertTableRows(tablePath, expectedRows); + assertTableRows(expectedRows); } private void testWriteRow(TableSchema tableSchema) throws Exception { @@ -150,7 +183,7 @@ private void testWriteRow(TableSchema tableSchema) throws Exception { ); DataStream dataStream = env.addSource(new FiniteTestSource<>(rows), ROW_TYPE_INFO); - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + FlinkSink.forRow(dataStream, FLINK_SCHEMA) .table(table) .tableLoader(tableLoader) .tableSchema(tableSchema) @@ -161,7 +194,7 @@ private void testWriteRow(TableSchema tableSchema) throws Exception { env.execute("Test Iceberg DataStream."); List expectedRows = Lists.newArrayList(Iterables.concat(convertToRowData(rows), convertToRowData(rows))); - SimpleDataUtil.assertTableRows(tablePath, expectedRows); + assertTableRows(expectedRows); } @Test @@ -171,6 +204,75 @@ public void testWriteRow() throws Exception { @Test public void testWriteRowWithTableSchema() throws Exception { - testWriteRow(SimpleDataUtil.FLINK_SCHEMA); + testWriteRow(FLINK_SCHEMA); + } + + @Test + public void testUpsertRow() throws Exception { + Assume.assumeFalse("ORC format does not support equality delete.", FileFormat.ORC.equals(format)); + + List rows = Lists.newArrayList( + Row.ofKind(RowKind.INSERT, 1, "aaa"), + Row.ofKind(RowKind.INSERT, 2, "bbb"), + Row.ofKind(RowKind.UPDATE_BEFORE, 1, "aaa"), + Row.ofKind(RowKind.UPDATE_AFTER, 1, "ccc"), + Row.ofKind(RowKind.INSERT, 3, "ccc") + ); + DataStream dataStream = env.addSource(new FiniteTestSource<>(rows), ROW_TYPE_INFO); + + FlinkSink.forRow(dataStream, FLINK_SCHEMA) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .equalityFieldNames(Lists.newArrayList("id")) + .build(); + + env.execute("Execute the upsert program."); + + List rowDataList = Lists.newArrayList( + SimpleDataUtil.createRowData(1, "ccc"), + SimpleDataUtil.createRowData(2, "bbb"), + SimpleDataUtil.createRowData(3, "ccc") + ); + assertTableRows(Lists.newArrayList(Iterables.concat(rowDataList, rowDataList))); + } + + private void assertTableRows(List expectedRows) throws IOException { + Iterable expected = + Iterables.transform(expectedRows, + row -> createRecord(Objects.requireNonNull(row).getInt(0), row.getString(1).toString())); + + table.refresh(); + try (CloseableIterable iterable = IcebergGenerics.read(table).build()) { + // Remove the row identifier. + Iterable actual = Iterables.transform(iterable, + r -> createRecord((Integer) Objects.requireNonNull(r).getField("id"), (String) r.getField("data"))); + + Assert.assertEquals("Should produce the expected record", Sets.newHashSet(expected), Sets.newHashSet(actual)); + } + } + + private static class TestTableLoader implements TableLoader { + private static final long serialVersionUID = 1L; + + private final File testTableDir; + + TestTableLoader(File testTableDir) { + this.testTableDir = testTableDir; + } + + @Override + public void open() { + + } + + @Override + public Table loadTable() { + return TestTables.load(testTableDir, TABLE_NAME); + } + + @Override + public void close() { + + } } } diff --git a/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java b/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java index 9fd15070d7fa..b7502da807ec 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java +++ b/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java @@ -122,16 +122,17 @@ public void testVersionedSerializer() throws IOException { OutputFile outputFile = factory.create(checkpointId); List expectedDataFiles = generateDataFiles(10); - ManifestFile expected = FlinkManifestUtil.writeDataFiles(outputFile, table.spec(), expectedDataFiles); + ManifestFile dataManifest = FlinkManifestUtil.writeDataFiles(outputFile, table.spec(), expectedDataFiles); + DeltaManifests expected = new DeltaManifests(dataManifest, null); byte[] versionedSerializeData = - SimpleVersionedSerialization.writeVersionAndSerialize(FlinkManifestSerializer.INSTANCE, expected); - ManifestFile actual = SimpleVersionedSerialization - .readVersionAndDeSerialize(FlinkManifestSerializer.INSTANCE, versionedSerializeData); - checkManifestFile(expected, actual); + SimpleVersionedSerialization.writeVersionAndSerialize(DeltaManifestsSerializer.INSTANCE, expected); + DeltaManifests actual = SimpleVersionedSerialization + .readVersionAndDeSerialize(DeltaManifestsSerializer.INSTANCE, versionedSerializeData); + checkManifestFile(expected.dataManifest(), actual.dataManifest()); byte[] versionedSerializeData2 = - SimpleVersionedSerialization.writeVersionAndSerialize(FlinkManifestSerializer.INSTANCE, actual); + SimpleVersionedSerialization.writeVersionAndSerialize(DeltaManifestsSerializer.INSTANCE, actual); Assert.assertArrayEquals(versionedSerializeData, versionedSerializeData2); } diff --git a/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java index 583b6f18e336..516f402da1d4 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ b/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -49,6 +49,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.WriterResult; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -113,7 +114,7 @@ public void testCommitTxnWithoutDataFiles() throws Exception { long checkpointId = 0; long timestamp = 0; JobID jobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { harness.setup(); harness.open(); @@ -148,7 +149,7 @@ public void testCommitTxn() throws Exception { long timestamp = 0; JobID jobID = new JobID(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobID)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobID)) { harness.setup(); harness.open(); assertSnapshotSize(0); @@ -157,7 +158,7 @@ public void testCommitTxn() throws Exception { for (int i = 1; i <= 3; i++) { RowData rowData = SimpleDataUtil.createRowData(i, "hello" + i); DataFile dataFile = writeDataFile("data-" + i, ImmutableList.of(rowData)); - harness.processElement(dataFile, ++timestamp); + harness.processElement(WriterResult.create(dataFile), ++timestamp); rows.add(rowData); harness.snapshot(i, ++timestamp); @@ -183,7 +184,7 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { long timestamp = 0; JobID jobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { harness.setup(); harness.open(); @@ -192,7 +193,7 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(dataFile1, ++timestamp); + harness.processElement(WriterResult.create(dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, -1L); // 1. snapshotState for checkpoint#1 @@ -202,7 +203,7 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { RowData row2 = SimpleDataUtil.createRowData(2, "world"); DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); - harness.processElement(dataFile2, ++timestamp); + harness.processElement(WriterResult.create(dataFile2), ++timestamp); assertMaxCommittedCheckpointId(jobId, -1L); // 2. snapshotState for checkpoint#2 @@ -234,7 +235,7 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { long timestamp = 0; JobID jobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { harness.setup(); harness.open(); @@ -243,7 +244,7 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(dataFile1, ++timestamp); + harness.processElement(WriterResult.create(dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, -1L); // 1. snapshotState for checkpoint#1 @@ -253,7 +254,7 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { RowData row2 = SimpleDataUtil.createRowData(2, "world"); DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); - harness.processElement(dataFile2, ++timestamp); + harness.processElement(WriterResult.create(dataFile2), ++timestamp); assertMaxCommittedCheckpointId(jobId, -1L); // 2. snapshotState for checkpoint#2 @@ -283,7 +284,7 @@ public void testRecoveryFromValidSnapshot() throws Exception { OperatorSubtaskState snapshot; JobID jobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { harness.setup(); harness.open(); @@ -294,7 +295,7 @@ public void testRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row)); - harness.processElement(dataFile1, ++timestamp); + harness.processElement(WriterResult.create(dataFile1), ++timestamp); snapshot = harness.snapshot(++checkpointId, ++timestamp); assertFlinkManifests(1); @@ -307,7 +308,7 @@ public void testRecoveryFromValidSnapshot() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { harness.setup(); harness.initializeState(snapshot); harness.open(); @@ -319,7 +320,7 @@ public void testRecoveryFromValidSnapshot() throws Exception { RowData row = SimpleDataUtil.createRowData(2, "world"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); - harness.processElement(dataFile, ++timestamp); + harness.processElement(WriterResult.create(dataFile), ++timestamp); harness.snapshot(++checkpointId, ++timestamp); assertFlinkManifests(1); @@ -342,7 +343,7 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except OperatorSubtaskState snapshot; List expectedRows = Lists.newArrayList(); JobID jobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { harness.setup(); harness.open(); @@ -352,7 +353,7 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(1, "hello"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-1", ImmutableList.of(row)); - harness.processElement(dataFile, ++timestamp); + harness.processElement(WriterResult.create(dataFile), ++timestamp); snapshot = harness.snapshot(++checkpointId, ++timestamp); SimpleDataUtil.assertTableRows(tablePath, ImmutableList.of()); @@ -360,7 +361,7 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except assertFlinkManifests(1); } - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { harness.setup(); harness.initializeState(snapshot); harness.open(); @@ -385,7 +386,7 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(2, "world"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); - harness.processElement(dataFile, ++timestamp); + harness.processElement(WriterResult.create(dataFile), ++timestamp); snapshot = harness.snapshot(++checkpointId, ++timestamp); assertFlinkManifests(1); @@ -393,7 +394,7 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except // Redeploying flink job from external checkpoint. JobID newJobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(newJobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(newJobId)) { harness.setup(); harness.initializeState(snapshot); harness.open(); @@ -409,7 +410,7 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(3, "foo"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-3", ImmutableList.of(row)); - harness.processElement(dataFile, ++timestamp); + harness.processElement(WriterResult.create(dataFile), ++timestamp); harness.snapshot(++checkpointId, ++timestamp); assertFlinkManifests(1); @@ -431,7 +432,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { List tableRows = Lists.newArrayList(); JobID oldJobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(oldJobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(oldJobId)) { harness.setup(); harness.open(); @@ -443,7 +444,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); - harness.processElement(dataFile, ++timestamp); + harness.processElement(WriterResult.create(dataFile), ++timestamp); harness.snapshot(++checkpointId, ++timestamp); assertFlinkManifests(1); @@ -460,7 +461,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { checkpointId = 0; timestamp = 0; JobID newJobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(newJobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(newJobId)) { harness.setup(); harness.open(); @@ -472,7 +473,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile("data-new-1", rows); - harness.processElement(dataFile, ++timestamp); + harness.processElement(WriterResult.create(dataFile), ++timestamp); harness.snapshot(++checkpointId, ++timestamp); assertFlinkManifests(1); @@ -494,7 +495,7 @@ public void testMultipleJobsWriteSameTable() throws Exception { int jobIndex = i % 3; int checkpointId = i / 3; JobID jobId = jobs[jobIndex]; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { harness.setup(); harness.open(); @@ -505,7 +506,7 @@ public void testMultipleJobsWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); - harness.processElement(dataFile, ++timestamp); + harness.processElement(WriterResult.create(dataFile), ++timestamp); harness.snapshot(checkpointId + 1, ++timestamp); assertFlinkManifests(1); @@ -521,7 +522,7 @@ public void testMultipleJobsWriteSameTable() throws Exception { @Test public void testBoundedStream() throws Exception { JobID jobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { harness.setup(); harness.open(); @@ -532,7 +533,7 @@ public void testBoundedStream() throws Exception { List tableRows = Lists.newArrayList(SimpleDataUtil.createRowData(1, "word-1")); DataFile dataFile = writeDataFile("data-1", tableRows); - harness.processElement(dataFile, 1); + harness.processElement(WriterResult.create(dataFile), 1); ((BoundedOneInput) harness.getOneInputOperator()).endInput(); assertFlinkManifests(0); @@ -548,7 +549,7 @@ public void testFlinkManifests() throws Exception { final long checkpoint = 10; JobID jobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { harness.setup(); harness.open(); @@ -557,7 +558,7 @@ public void testFlinkManifests() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(dataFile1, ++timestamp); + harness.processElement(WriterResult.create(dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, -1L); // 1. snapshotState for checkpoint#1 @@ -609,7 +610,7 @@ private void assertSnapshotSize(int expectedSnapshotSize) { Assert.assertEquals(expectedSnapshotSize, Lists.newArrayList(table.snapshots()).size()); } - private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) + private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) throws Exception { TestOperatorFactory factory = TestOperatorFactory.of(tablePath); return new OneInputStreamOperatorTestHarness<>(factory, createEnvironment(jobID)); @@ -629,7 +630,7 @@ private static MockEnvironment createEnvironment(JobID jobID) { } private static class TestOperatorFactory extends AbstractStreamOperatorFactory - implements OneInputStreamOperatorFactory { + implements OneInputStreamOperatorFactory { private final String tablePath; private TestOperatorFactory(String tablePath) { diff --git a/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java index 80771a4d233f..1863b3a8471e 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java +++ b/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java @@ -25,6 +25,8 @@ import java.util.Locale; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.flink.streaming.api.operators.BoundedOneInput; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.table.api.DataTypes; @@ -47,6 +49,7 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.WriterResult; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -101,7 +104,7 @@ public void before() throws IOException { @Test public void testWritingTable() throws Exception { long checkpointId = 1L; - try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { // The first checkpoint testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 1); @@ -109,7 +112,7 @@ public void testWritingTable() throws Exception { testHarness.prepareSnapshotPreBarrier(checkpointId); long expectedDataFiles = partitioned ? 2 : 1; - Assert.assertEquals(expectedDataFiles, testHarness.extractOutputValues().size()); + Assert.assertEquals(expectedDataFiles, extractDataFiles(testHarness).size()); checkpointId = checkpointId + 1; @@ -119,11 +122,11 @@ public void testWritingTable() throws Exception { testHarness.prepareSnapshotPreBarrier(checkpointId); expectedDataFiles = partitioned ? 4 : 2; - Assert.assertEquals(expectedDataFiles, testHarness.extractOutputValues().size()); + Assert.assertEquals(expectedDataFiles, extractDataFiles(testHarness).size()); // Commit the iceberg transaction. AppendFiles appendFiles = table.newAppend(); - testHarness.extractOutputValues().forEach(appendFiles::appendFile); + extractDataFiles(testHarness).forEach(appendFiles::appendFile); appendFiles.commit(); // Assert the table records. @@ -141,34 +144,34 @@ public void testWritingTable() throws Exception { public void testSnapshotTwice() throws Exception { long checkpointId = 1; long timestamp = 1; - try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), timestamp++); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), timestamp); testHarness.prepareSnapshotPreBarrier(checkpointId++); long expectedDataFiles = partitioned ? 2 : 1; - Assert.assertEquals(expectedDataFiles, testHarness.extractOutputValues().size()); + Assert.assertEquals(expectedDataFiles, extractDataFiles(testHarness).size()); // snapshot again immediately. for (int i = 0; i < 5; i++) { testHarness.prepareSnapshotPreBarrier(checkpointId++); - Assert.assertEquals(expectedDataFiles, testHarness.extractOutputValues().size()); + Assert.assertEquals(expectedDataFiles, extractDataFiles(testHarness).size()); } } } @Test public void testTableWithoutSnapshot() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { - Assert.assertEquals(0, testHarness.extractOutputValues().size()); + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { + Assert.assertEquals(0, extractDataFiles(testHarness).size()); } // Even if we closed the iceberg stream writer, there's no orphan data file. Assert.assertEquals(0, scanDataFiles().size()); - try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); // Still not emit the data file yet, because there is no checkpoint. - Assert.assertEquals(0, testHarness.extractOutputValues().size()); + Assert.assertEquals(0, extractDataFiles(testHarness).size()); } // Once we closed the iceberg stream writer, there will left an orphan data file. Assert.assertEquals(1, scanDataFiles().size()); @@ -197,7 +200,7 @@ private Set scanDataFiles() throws IOException { @Test public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); @@ -205,11 +208,11 @@ public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); long expectedDataFiles = partitioned ? 2 : 1; - Assert.assertEquals(expectedDataFiles, testHarness.extractOutputValues().size()); + Assert.assertEquals(expectedDataFiles, extractDataFiles(testHarness).size()); // invoke endInput again. ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); - Assert.assertEquals(expectedDataFiles * 2, testHarness.extractOutputValues().size()); + Assert.assertEquals(expectedDataFiles * 2, extractDataFiles(testHarness).size()); } } @@ -233,23 +236,23 @@ public void testTableWithTargetFileSize() throws Exception { } } - try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { for (RowData row : rows) { testHarness.processElement(row, 1); } // snapshot the operator. testHarness.prepareSnapshotPreBarrier(1); - Assert.assertEquals(8, testHarness.extractOutputValues().size()); + Assert.assertEquals(8, extractDataFiles(testHarness).size()); // Assert that the data file have the expected records. - for (DataFile serDataFile : testHarness.extractOutputValues()) { + for (DataFile serDataFile : extractDataFiles(testHarness)) { Assert.assertEquals(1000, serDataFile.recordCount()); } // Commit the iceberg transaction. AppendFiles appendFiles = table.newAppend(); - testHarness.extractOutputValues().forEach(appendFiles::appendFile); + extractDataFiles(testHarness).forEach(appendFiles::appendFile); appendFiles.commit(); } @@ -294,31 +297,36 @@ public void testPromotedFlinkDataType() throws Exception { record.copy(ImmutableMap.of("tinyint", 3, "smallint", 32767, "int", 103)) ); - try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter(icebergTable, + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter(icebergTable, flinkSchema)) { for (RowData row : rows) { testHarness.processElement(row, 1); } testHarness.prepareSnapshotPreBarrier(1); - Assert.assertEquals(partitioned ? 3 : 1, testHarness.extractOutputValues().size()); + Assert.assertEquals(partitioned ? 3 : 1, extractDataFiles(testHarness).size()); // Commit the iceberg transaction. AppendFiles appendFiles = icebergTable.newAppend(); - testHarness.extractOutputValues().forEach(appendFiles::appendFile); + extractDataFiles(testHarness).forEach(appendFiles::appendFile); appendFiles.commit(); } SimpleDataUtil.assertTableRecords(location, expected); } - private OneInputStreamOperatorTestHarness createIcebergStreamWriter() throws Exception { + private List extractDataFiles(OneInputStreamOperatorTestHarness harness) { + return harness.extractOutputValues().stream() + .map(WriterResult::dataFiles).flatMap(Stream::of).collect(Collectors.toList()); + } + + private OneInputStreamOperatorTestHarness createIcebergStreamWriter() throws Exception { return createIcebergStreamWriter(table, SimpleDataUtil.FLINK_SCHEMA); } - private OneInputStreamOperatorTestHarness createIcebergStreamWriter( + private OneInputStreamOperatorTestHarness createIcebergStreamWriter( Table icebergTable, TableSchema flinkSchema) throws Exception { - IcebergStreamWriter streamWriter = FlinkSink.createStreamWriter(icebergTable, flinkSchema); - OneInputStreamOperatorTestHarness harness = new OneInputStreamOperatorTestHarness<>( + IcebergStreamWriter streamWriter = FlinkSink.createStreamWriter(icebergTable, flinkSchema, null); + OneInputStreamOperatorTestHarness harness = new OneInputStreamOperatorTestHarness<>( streamWriter, 1, 1, 0); harness.setup(); diff --git a/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataTaskWriter.java b/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataTaskWriter.java new file mode 100644 index 000000000000..75ac1ff7f06b --- /dev/null +++ b/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataTaskWriter.java @@ -0,0 +1,29 @@ +/* + * 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.sink; + +import org.junit.Test; + +public class TestRowDataTaskWriter { + + @Test + public void test() { + } +} diff --git a/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java b/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java index 60e6cefd45c0..2b9fe94224ec 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java +++ b/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java @@ -94,13 +94,13 @@ public void testWriteZeroRecord() throws IOException { try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.close(); - DataFile[] dataFiles = taskWriter.complete(); + DataFile[] dataFiles = taskWriter.complete().dataFiles(); Assert.assertNotNull(dataFiles); Assert.assertEquals(0, dataFiles.length); // Close again. taskWriter.close(); - dataFiles = taskWriter.complete(); + dataFiles = taskWriter.complete().dataFiles(); Assert.assertNotNull(dataFiles); Assert.assertEquals(0, dataFiles.length); } @@ -115,7 +115,7 @@ public void testCloseTwice() throws IOException { taskWriter.close(); // The second close int expectedFiles = partitioned ? 2 : 1; - DataFile[] dataFiles = taskWriter.complete(); + DataFile[] dataFiles = taskWriter.complete().dataFiles(); Assert.assertEquals(expectedFiles, dataFiles.length); FileSystem fs = FileSystem.get(CONF); @@ -132,7 +132,7 @@ public void testAbort() throws IOException { taskWriter.write(SimpleDataUtil.createRowData(2, "world")); taskWriter.abort(); - DataFile[] dataFiles = taskWriter.complete(); + DataFile[] dataFiles = taskWriter.complete().dataFiles(); int expectedFiles = partitioned ? 2 : 1; Assert.assertEquals(expectedFiles, dataFiles.length); @@ -152,11 +152,11 @@ public void testCompleteFiles() throws IOException { taskWriter.write(SimpleDataUtil.createRowData(3, "c")); taskWriter.write(SimpleDataUtil.createRowData(4, "d")); - DataFile[] dataFiles = taskWriter.complete(); + DataFile[] dataFiles = taskWriter.complete().dataFiles(); int expectedFiles = partitioned ? 4 : 1; Assert.assertEquals(expectedFiles, dataFiles.length); - dataFiles = taskWriter.complete(); + dataFiles = taskWriter.complete().dataFiles(); Assert.assertEquals(expectedFiles, dataFiles.length); FileSystem fs = FileSystem.get(CONF); @@ -200,7 +200,7 @@ public void testRollingWithTargetFileSize() throws IOException { taskWriter.write(row); } - DataFile[] dataFiles = taskWriter.complete(); + DataFile[] dataFiles = taskWriter.complete().dataFiles(); Assert.assertEquals(8, dataFiles.length); AppendFiles appendFiles = table.newAppend(); @@ -223,7 +223,7 @@ public void testRandomData() throws IOException { } taskWriter.close(); - DataFile[] dataFiles = taskWriter.complete(); + DataFile[] dataFiles = taskWriter.complete().dataFiles(); AppendFiles appendFiles = table.newAppend(); for (DataFile dataFile : dataFiles) { appendFiles.appendFile(dataFile); @@ -239,7 +239,7 @@ private TaskWriter createTaskWriter(long targetFileSize) { TaskWriterFactory taskWriterFactory = new RowDataTaskWriterFactory(table.schema(), (RowType) SimpleDataUtil.FLINK_SCHEMA.toRowDataType().getLogicalType(), table.spec(), table.locationProvider(), table.io(), table.encryption(), - targetFileSize, format, table.properties()); + targetFileSize, format, table.properties(), Lists.newArrayList()); taskWriterFactory.initialize(1, 1); return taskWriterFactory.create(); } diff --git a/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java b/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java index b116faff23d1..9648d493a96f 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java +++ b/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java @@ -29,7 +29,7 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataConverter; -import org.apache.iceberg.flink.sink.RowDataTaskWriterFactory; +import org.apache.iceberg.flink.sink.FlinkFileAppenderFactory; import org.apache.iceberg.io.FileAppender; public class TestFlinkMergingMetrics extends TestMergingMetrics { @@ -38,9 +38,8 @@ public class TestFlinkMergingMetrics extends TestMergingMetrics { protected FileAppender writeAndGetAppender(List records) throws IOException { RowType flinkSchema = FlinkSchemaUtil.convert(SCHEMA); - FileAppender appender = - new RowDataTaskWriterFactory.FlinkFileAppenderFactory(SCHEMA, flinkSchema, new HashMap<>()).newAppender( - org.apache.iceberg.Files.localOutput(temp.newFile()), FileFormat.PARQUET); + FileAppender appender = new FlinkFileAppenderFactory(SCHEMA, flinkSchema, new HashMap<>()).newAppender( + org.apache.iceberg.Files.localOutput(temp.newFile()), FileFormat.PARQUET); try (FileAppender fileAppender = appender) { records.stream().map(r -> RowDataConverter.convert(SCHEMA, r)).forEach(fileAppender::add); } 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 fc35adc63f18..8fe4c9619bb8 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -32,7 +32,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Files; -import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.MetricsConfig; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -41,6 +40,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.avro.AvroSchemaUtil; import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.deletes.DeletesUtil; import org.apache.iceberg.deletes.EqualityDeleteWriter; import org.apache.iceberg.deletes.PositionDeleteWriter; import org.apache.iceberg.encryption.EncryptionKeyMetadata; @@ -58,7 +58,6 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.Types.NestedField; import org.apache.iceberg.util.ArrayUtil; import org.apache.parquet.HadoopReadOptions; import org.apache.parquet.ParquetReadOptions; @@ -272,6 +271,28 @@ public static DeleteWriteBuilder writeDeletes(OutputFile file) { return new DeleteWriteBuilder(file); } + public interface PositionAccessor { + + FILE accessPath(CharSequence path); + + POS accessPos(long pos); + } + + private static class IdentityPositionAccessor implements Parquet.PositionAccessor { + + static final IdentityPositionAccessor INSTANCE = new IdentityPositionAccessor(); + + @Override + public CharSequence accessPath(CharSequence path) { + return path; + } + + @Override + public Long accessPos(long pos) { + return pos; + } + } + public static class DeleteWriteBuilder { private final WriteBuilder appenderBuilder; private final String location; @@ -361,7 +382,7 @@ public DeleteWriteBuilder equalityFieldIds(int... fieldIds) { } public EqualityDeleteWriter buildEqualityWriter() throws IOException { - Preconditions.checkState(rowSchema != null, "Cannot create equality delete file without a schema`"); + 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.checkState(createWriterFunc != null, "Cannot create equality delete file unless createWriterFunc is set"); @@ -379,42 +400,38 @@ public EqualityDeleteWriter buildEqualityWriter() throws IOException { appenderBuilder.build(), FileFormat.PARQUET, location, spec, partition, keyMetadata, equalityFieldIds); } - - public PositionDeleteWriter buildPositionWriter() throws IOException { + public PositionDeleteWriter buildPositionWriter(PositionAccessor positionAccessor) throws IOException { Preconditions.checkState(equalityFieldIds == null, "Cannot create position delete file using delete field ids"); meta("delete-type", "position"); - if (rowSchema != null && createWriterFunc != null) { - // the appender uses the row schema wrapped with position fields - appenderBuilder.schema(new org.apache.iceberg.Schema( - MetadataColumns.DELETE_FILE_PATH, - MetadataColumns.DELETE_FILE_POS, - NestedField.optional( - MetadataColumns.DELETE_FILE_ROW_FIELD_ID, "row", rowSchema.asStruct(), - MetadataColumns.DELETE_FILE_ROW_DOC))); + if (createWriterFunc != null) { + appenderBuilder.schema(DeletesUtil.posDeleteSchema(rowSchema)); appenderBuilder.createWriterFunc(parquetSchema -> { ParquetValueWriter writer = createWriterFunc.apply(parquetSchema); if (writer instanceof StructWriter) { - return new PositionDeleteStructWriter((StructWriter) writer); + return new PositionDeleteStructWriter((StructWriter) writer, positionAccessor); } else { throw new UnsupportedOperationException("Cannot wrap writer for position deletes: " + writer.getClass()); } }); } else { - appenderBuilder.schema(new org.apache.iceberg.Schema( - MetadataColumns.DELETE_FILE_PATH, - MetadataColumns.DELETE_FILE_POS)); + appenderBuilder.schema(DeletesUtil.pathPosSchema()); appenderBuilder.createWriterFunc(parquetSchema -> - new PositionDeleteStructWriter((StructWriter) GenericParquetWriter.buildWriter(parquetSchema))); + new PositionDeleteStructWriter((StructWriter) GenericParquetWriter.buildWriter(parquetSchema), + positionAccessor)); } return new PositionDeleteWriter<>( appenderBuilder.build(), FileFormat.PARQUET, location, spec, partition, keyMetadata); } + + public PositionDeleteWriter buildPositionWriter() throws IOException { + return buildPositionWriter(IdentityPositionAccessor.INSTANCE); + } } private static class ParquetWriteBuilder extends ParquetWriter.Builder> { diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java index c17a2ffa173d..db96b65ad562 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java @@ -154,7 +154,7 @@ public void writeInteger(int repetitionLevel, int value) { column.writeInteger(repetitionLevel, value); } - public void writeLong(int repetitionLevel, long value) { + public void writeLong(int repetitionLevel, long value) { column.writeLong(repetitionLevel, value); } @@ -562,17 +562,21 @@ public Stream metrics() { } public static class PositionDeleteStructWriter extends StructWriter> { - public PositionDeleteStructWriter(StructWriter replacedWriter) { + + private Parquet.PositionAccessor positionAccessor; + + public PositionDeleteStructWriter(StructWriter replacedWriter, Parquet.PositionAccessor accessor) { super(Arrays.asList(replacedWriter.writers)); + this.positionAccessor = accessor; } @Override protected Object get(PositionDelete delete, int index) { switch (index) { case 0: - return delete.path(); + return positionAccessor.accessPath(delete.path()); case 1: - return delete.pos(); + return positionAccessor.accessPos(delete.pos()); case 2: return delete.row(); } diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDeleteWriters.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDeleteWriters.java index 77ceec1e987a..b7d56ccb8976 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDeleteWriters.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDeleteWriters.java @@ -86,10 +86,10 @@ public void testEqualityDeleteWriter() throws IOException { .buildEqualityWriter(); try (EqualityDeleteWriter writer = deleteWriter) { - writer.deleteAll(records); + writer.writeAll(records); } - DeleteFile metadata = deleteWriter.toDeleteFile(); + DeleteFile metadata = deleteWriter.toContentFile(); Assert.assertEquals("Format should be Parquet", FileFormat.PARQUET, metadata.format()); Assert.assertEquals("Should be equality deletes", FileContent.EQUALITY_DELETES, metadata.content()); Assert.assertEquals("Record count should be correct", records.size(), metadata.recordCount()); @@ -139,7 +139,7 @@ public void testPositionDeleteWriter() throws IOException { } } - DeleteFile metadata = deleteWriter.toDeleteFile(); + DeleteFile metadata = deleteWriter.toContentFile(); Assert.assertEquals("Format should be Parquet", FileFormat.PARQUET, metadata.format()); Assert.assertEquals("Should be position deletes", FileContent.POSITION_DELETES, metadata.content()); Assert.assertEquals("Record count should be correct", records.size(), metadata.recordCount()); @@ -186,7 +186,7 @@ public void testPositionDeleteWriterWithEmptyRow() throws IOException { } } - DeleteFile metadata = deleteWriter.toDeleteFile(); + DeleteFile metadata = deleteWriter.toContentFile(); Assert.assertEquals("Format should be Parquet", FileFormat.PARQUET, metadata.format()); Assert.assertEquals("Should be position deletes", FileContent.POSITION_DELETES, metadata.content()); Assert.assertEquals("Record count should be correct", records.size(), metadata.recordCount()); diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/RowDataRewriter.java b/spark/src/main/java/org/apache/iceberg/spark/source/RowDataRewriter.java index 37ca56c700a4..a844536f4a90 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/RowDataRewriter.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/RowDataRewriter.java @@ -33,6 +33,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.io.DataFileWriterFactory; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.io.OutputFileFactory; @@ -104,7 +105,8 @@ private List rewriteDataForTask(CombinedScanTask task) throws Exceptio TaskWriter writer; if (spec.fields().isEmpty()) { - writer = new UnpartitionedWriter<>(spec, format, appenderFactory, fileFactory, io.value(), Long.MAX_VALUE); + writer = new UnpartitionedWriter<>(format, fileFactory, io.value(), Long.MAX_VALUE, + new DataFileWriterFactory<>(appenderFactory, spec)); } else { writer = new SparkPartitionedWriter(spec, format, appenderFactory, fileFactory, io.value(), Long.MAX_VALUE, schema, structType); @@ -120,7 +122,7 @@ private List rewriteDataForTask(CombinedScanTask task) throws Exceptio dataReader = null; writer.close(); - return Lists.newArrayList(writer.complete()); + return Lists.newArrayList(writer.complete().dataFiles()); } catch (Throwable originalThrowable) { try { diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedWriter.java b/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedWriter.java index f81a09926d85..779b4bc00618 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedWriter.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedWriter.java @@ -19,10 +19,12 @@ package org.apache.iceberg.spark.source; +import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.io.DataFileWriterFactory; import org.apache.iceberg.io.FileAppenderFactory; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.OutputFileFactory; @@ -30,7 +32,7 @@ import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.types.StructType; -public class SparkPartitionedWriter extends PartitionedWriter { +public class SparkPartitionedWriter extends PartitionedWriter { private final PartitionKey partitionKey; private final InternalRowWrapper internalRowWrapper; @@ -38,7 +40,7 @@ public SparkPartitionedWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, StructType sparkSchema) { - super(spec, format, appenderFactory, fileFactory, io, targetFileSize); + super(format, fileFactory, io, targetFileSize, new DataFileWriterFactory<>(appenderFactory, spec)); this.partitionKey = new PartitionKey(spec, schema); this.internalRowWrapper = new InternalRowWrapper(sparkSchema); } diff --git a/spark2/src/main/java/org/apache/iceberg/spark/source/Writer.java b/spark2/src/main/java/org/apache/iceberg/spark/source/Writer.java index 6b6739a06a25..b38e207010da 100644 --- a/spark2/src/main/java/org/apache/iceberg/spark/source/Writer.java +++ b/spark2/src/main/java/org/apache/iceberg/spark/source/Writer.java @@ -35,6 +35,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.io.DataFileWriterFactory; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.io.OutputFileFactory; @@ -277,18 +278,18 @@ public DataWriter createDataWriter(int partitionId, long taskId, lo } } - private static class Unpartitioned24Writer extends UnpartitionedWriter + private static class Unpartitioned24Writer extends UnpartitionedWriter implements DataWriter { Unpartitioned24Writer(PartitionSpec spec, FileFormat format, SparkAppenderFactory appenderFactory, OutputFileFactory fileFactory, FileIO fileIo, long targetFileSize) { - super(spec, format, appenderFactory, fileFactory, fileIo, targetFileSize); + super(format, fileFactory, fileIo, targetFileSize, new DataFileWriterFactory<>(appenderFactory, spec)); } @Override public WriterCommitMessage commit() throws IOException { close(); - return new TaskCommit(complete()); + return new TaskCommit(complete().dataFiles()); } } @@ -304,7 +305,7 @@ private static class Partitioned24Writer extends SparkPartitionedWriter implemen public WriterCommitMessage commit() throws IOException { close(); - return new TaskCommit(complete()); + return new TaskCommit(complete().dataFiles()); } } } diff --git a/spark3/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java b/spark3/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java index efbc319197f0..d7963a7d447e 100644 --- a/spark3/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java +++ b/spark3/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java @@ -39,6 +39,7 @@ import org.apache.iceberg.encryption.EncryptionManager; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.DataFileWriterFactory; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.io.OutputFileFactory; @@ -426,18 +427,18 @@ public DataWriter createWriter(int partitionId, long taskId, long e } } - private static class Unpartitioned3Writer extends UnpartitionedWriter + private static class Unpartitioned3Writer extends UnpartitionedWriter implements DataWriter { Unpartitioned3Writer(PartitionSpec spec, FileFormat format, SparkAppenderFactory appenderFactory, OutputFileFactory fileFactory, FileIO io, long targetFileSize) { - super(spec, format, appenderFactory, fileFactory, io, targetFileSize); + super(format, fileFactory, io, targetFileSize, new DataFileWriterFactory<>(appenderFactory, spec)); } @Override public WriterCommitMessage commit() throws IOException { this.close(); - return new TaskCommit(complete()); + return new TaskCommit(complete().dataFiles()); } } @@ -452,7 +453,7 @@ private static class Partitioned3Writer extends SparkPartitionedWriter implement public WriterCommitMessage commit() throws IOException { this.close(); - return new TaskCommit(complete()); + return new TaskCommit(complete().dataFiles()); } } }