From 6de9d80bb74ee7c5bc963a61769539e9ab5978d4 Mon Sep 17 00:00:00 2001 From: huzheng Date: Fri, 23 Oct 2020 18:04:46 +0800 Subject: [PATCH 01/41] Abstract the TaskWriter to accept both INSERT and DELETE records. --- .../org/apache/iceberg/ContentFileWriter.java | 50 ++++++++++ .../org/apache/iceberg/DataFileWriter.java | 84 +++++++++++++++++ .../iceberg/deletes/EqualityDeleteWriter.java | 21 +++-- .../org/apache/iceberg/io/BaseTaskWriter.java | 82 ++++++++++------ .../io/EqualityDeleteWriterFactory.java | 33 +++++++ .../apache/iceberg/io/PartitionedWriter.java | 5 +- .../org/apache/iceberg/io/TaskWriter.java | 3 +- .../apache/iceberg/io/TaskWriterResult.java | 92 ++++++++++++++++++ .../iceberg/io/UnpartitionedWriter.java | 5 +- .../iceberg/avro/TestAvroDeleteWriters.java | 4 +- .../org/apache/iceberg/data/FileHelpers.java | 4 +- .../sink/FlinkEqualityDeleterFactory.java | 94 +++++++++++++++++++ .../flink/sink/IcebergStreamWriter.java | 4 +- .../flink/sink/PartitionedFanoutWriter.java | 7 +- .../iceberg/flink/sink/TestTaskWriters.java | 16 ++-- .../parquet/TestParquetDeleteWriters.java | 4 +- .../iceberg/spark/source/RowDataRewriter.java | 2 +- .../apache/iceberg/spark/source/Writer.java | 4 +- .../iceberg/spark/source/SparkWrite.java | 4 +- 19 files changed, 452 insertions(+), 66 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/ContentFileWriter.java create mode 100644 core/src/main/java/org/apache/iceberg/DataFileWriter.java create mode 100644 core/src/main/java/org/apache/iceberg/io/EqualityDeleteWriterFactory.java create mode 100644 core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java create mode 100644 flink/src/main/java/org/apache/iceberg/flink/sink/FlinkEqualityDeleterFactory.java 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..1f3f29b1c789 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/ContentFileWriter.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg; + +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 {@link Metrics} for this file. Only valid after the file is closed. + */ + Metrics metrics(); + + /** + * Returns the length of this file. + */ + long length(); + + T toContentFile(); +} 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..9cc54ffdad8d --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/DataFileWriter.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +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 Metrics metrics() { + return appender.metrics(); + } + + @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/deletes/EqualityDeleteWriter.java b/core/src/main/java/org/apache/iceberg/deletes/EqualityDeleteWriter.java index 1838ef749b08..75ea9fd91d69 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/EqualityDeleteWriter.java +++ b/core/src/main/java/org/apache/iceberg/deletes/EqualityDeleteWriter.java @@ -19,19 +19,20 @@ 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; +import org.apache.iceberg.Metrics; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.StructLike; import org.apache.iceberg.encryption.EncryptionKeyMetadata; 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,14 +54,21 @@ 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); } + @Override + public Metrics metrics() { + return appender.metrics(); + } + @Override public void close() throws IOException { if (deleteFile == null) { @@ -77,7 +85,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/io/BaseTaskWriter.java b/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java index ca16318d8bdc..696dd0f63c72 100644 --- a/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java @@ -21,19 +21,19 @@ import java.io.Closeable; import java.io.IOException; -import java.util.List; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.ContentFileWriter; import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DataFileWriter; 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 TaskWriterResult.Builder builder; private final PartitionSpec spec; private final FileFormat format; private final FileAppenderFactory appenderFactory; @@ -43,6 +43,7 @@ public abstract class BaseTaskWriter implements TaskWriter { protected BaseTaskWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, OutputFileFactory fileFactory, FileIO io, long targetFileSize) { + this.builder = TaskWriterResult.builder(); this.spec = spec; this.format = format; this.appenderFactory = appenderFactory; @@ -56,34 +57,59 @@ public void abort() throws IOException { close(); // clean up files created by this writer - Tasks.foreach(completedFiles) + TaskWriterResult result = builder.build(); + + Tasks.foreach(result.dataFiles()) + .throwFailureWhenFinished() + .noRetry() + .run(file -> io.deleteFile(file.path().toString())); + + Tasks.foreach(result.deleteFiles()) .throwFailureWhenFinished() .noRetry() .run(file -> io.deleteFile(file.path().toString())); } @Override - public DataFile[] complete() throws IOException { + public TaskWriterResult complete() throws IOException { close(); - return completedFiles.toArray(new DataFile[0]); + return builder.build(); } - protected class RollingFileWriter implements Closeable { + protected class RollingDataFileWriter extends BaseRollingFileWriter { + + public RollingDataFileWriter(PartitionKey partitionKey) { + super(partitionKey); + } + + @Override + ContentFileWriter newContentFileWriter(EncryptedOutputFile outputFile, FileFormat fileFormat) { + FileAppender appender = appenderFactory.newAppender(outputFile.encryptingOutputFile(), fileFormat); + return new DataFileWriter<>(appender, fileFormat, outputFile.encryptingOutputFile().location(), partitionKey(), + spec, outputFile.keyMetadata()); + } + } + + protected abstract class BaseRollingFileWriter implements Closeable { private static final int ROWS_DIVISOR = 1000; private final PartitionKey partitionKey; private EncryptedOutputFile currentFile = null; - private FileAppender currentAppender = null; + private ContentFileWriter currentFileWriter = null; private long currentRows = 0; - public RollingFileWriter(PartitionKey partitionKey) { + public BaseRollingFileWriter(PartitionKey partitionKey) { this.partitionKey = partitionKey; openCurrent(); } - public void add(T record) throws IOException { - this.currentAppender.add(record); + protected PartitionKey partitionKey() { + return partitionKey; + } + + public void add(R record) throws IOException { + this.currentFileWriter.write(record); this.currentRows++; if (shouldRollToNewFile()) { @@ -92,6 +118,8 @@ public void add(T record) throws IOException { } } + abstract ContentFileWriter newContentFileWriter(EncryptedOutputFile outputFile, FileFormat fileFormat); + private void openCurrent() { if (partitionKey == null) { // unpartitioned @@ -100,37 +128,31 @@ private void openCurrent() { // partitioned currentFile = fileFactory.newOutputFile(partitionKey); } - currentAppender = appenderFactory.newAppender(currentFile.encryptingOutputFile(), format); + currentFileWriter = newContentFileWriter(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 && currentAppender.length() >= targetFileSize; + currentRows % ROWS_DIVISOR == 0 && currentFileWriter.length() >= targetFileSize; } + @SuppressWarnings("unchecked") 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 (currentFileWriter != null) { + currentFileWriter.close(); + F contentFile = currentFileWriter.toContentFile(); + Metrics metrics = currentFileWriter.metrics(); + this.currentFileWriter = null; if (metrics.recordCount() == 0L) { io.deleteFile(currentFile.encryptingOutputFile()); + } else if (contentFile instanceof ContentFile) { + builder.add((ContentFile) contentFile); } 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); + throw new RuntimeException(String.format( + "The newly generated content file must be DataFile or DeleteFile: %s", contentFile)); } this.currentFile = null; diff --git a/core/src/main/java/org/apache/iceberg/io/EqualityDeleteWriterFactory.java b/core/src/main/java/org/apache/iceberg/io/EqualityDeleteWriterFactory.java new file mode 100644 index 000000000000..5c1b52d2bbfb --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/io/EqualityDeleteWriterFactory.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.FileFormat; +import org.apache.iceberg.deletes.EqualityDeleteWriter; + +/** + * Factory to create a new {@link EqualityDeleteWriter} to write equality deletions in row-level. + * + * @param data type of the rows to delete. + */ +public interface EqualityDeleteWriterFactory { + + EqualityDeleteWriter newEqualityDeleteWriter(OutputFile outputFile, FileFormat format); +} 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..8161e893e909 100644 --- a/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.Set; +import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; @@ -35,7 +36,7 @@ public abstract class PartitionedWriter extends BaseTaskWriter { private final Set completedPartitions = Sets.newHashSet(); private PartitionKey currentKey = null; - private RollingFileWriter currentWriter = null; + private BaseRollingFileWriter currentWriter = null; public PartitionedWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, OutputFileFactory fileFactory, FileIO io, long targetFileSize) { @@ -70,7 +71,7 @@ public void write(T row) throws IOException { } currentKey = key.copy(); - currentWriter = new RollingFileWriter(currentKey); + currentWriter = new RollingDataFileWriter(currentKey); } currentWriter.add(row); 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..8eb4b7f95e78 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; + TaskWriterResult complete() throws IOException; } diff --git a/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java b/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java new file mode 100644 index 000000000000..0200cd95a51c --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java @@ -0,0 +1,92 @@ +/* + * 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.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.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +public class TaskWriterResult { + private static final TaskWriterResult EMPTY = new TaskWriterResult(ImmutableList.of(), ImmutableList.of()); + + private DataFile[] dataFiles; + private DeleteFile[] deleteFiles; + + TaskWriterResult(DataFile[] dataFiles, DeleteFile[] deleteFiles) { + this.dataFiles = dataFiles; + this.deleteFiles = deleteFiles; + } + + TaskWriterResult(List dataFiles, List deleteFiles) { + this.dataFiles = dataFiles.toArray(new DataFile[0]); + this.deleteFiles = deleteFiles.toArray(new DeleteFile[0]); + } + + static TaskWriterResult empty() { + return EMPTY; + } + + public DataFile[] dataFiles() { + return dataFiles; + } + + public DeleteFile[] deleteFiles() { + return deleteFiles; + } + + 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(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 file: " + contentFile.content()); + } + } + + public TaskWriterResult build() { + return new TaskWriterResult(dataFiles, deleteFiles); + } + } +} 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..7b08761fc5ab 100644 --- a/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java @@ -20,17 +20,18 @@ package org.apache.iceberg.io; import java.io.IOException; +import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionSpec; public class UnpartitionedWriter extends BaseTaskWriter { - private final RollingFileWriter currentWriter; + private final BaseRollingFileWriter 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); + currentWriter = new RollingDataFileWriter(null); } @Override 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..1f8bb66c0cdd 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()); 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..0755ffb8ca18 100644 --- a/data/src/test/java/org/apache/iceberg/data/FileHelpers.java +++ b/data/src/test/java/org/apache/iceberg/data/FileHelpers.java @@ -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/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkEqualityDeleterFactory.java b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkEqualityDeleterFactory.java new file mode 100644 index 000000000000..75a924d5df3c --- /dev/null +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkEqualityDeleterFactory.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.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.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.deletes.EqualityDeleteWriter; +import org.apache.iceberg.flink.data.FlinkAvroWriter; +import org.apache.iceberg.flink.data.FlinkParquetWriters; +import org.apache.iceberg.io.EqualityDeleteWriterFactory; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.parquet.Parquet; + +public class FlinkEqualityDeleterFactory implements EqualityDeleteWriterFactory, Serializable { + private final Schema schema; + private final RowType flinkSchema; + private final PartitionSpec spec; + private final List equalityFieldIds; + private final Map props; + + public FlinkEqualityDeleterFactory(Schema schema, + RowType flinkSchema, + PartitionSpec spec, + List equalityFieldIds, + Map props) { + this.schema = schema; + this.flinkSchema = flinkSchema; + this.spec = spec; + this.equalityFieldIds = equalityFieldIds; + this.props = props; + } + + @Override + public EqualityDeleteWriter newEqualityDeleteWriter(OutputFile outputFile, FileFormat format) { + MetricsConfig metricsConfig = MetricsConfig.fromProperties(props); + try { + switch (format) { + case AVRO: + return Avro.writeDeletes(outputFile) + .createWriterFunc(ignore -> new FlinkAvroWriter(flinkSchema)) + .overwrite() + .setAll(props) + .rowSchema(schema) + .withSpec(spec) + .equalityFieldIds(equalityFieldIds) + .buildEqualityWriter(); + + case PARQUET: + return Parquet.writeDeletes(outputFile) + .createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(flinkSchema, msgType)) + .overwrite() + .setAll(props) + .metricsConfig(metricsConfig) + .rowSchema(schema) + .withSpec(spec) + .equalityFieldIds(equalityFieldIds) + .buildEqualityWriter(); + + case ORC: + 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/IcebergStreamWriter.java b/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java index 95daa9656c68..8e9861171534 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 @@ -62,7 +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()) { + for (DataFile dataFile : writer.complete().dataFiles()) { emit(dataFile); } @@ -87,7 +87,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()) { + for (DataFile dataFile : writer.complete().dataFiles()) { emit(dataFile); } } 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 index ad846974adcf..77eb8d788fc6 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedFanoutWriter.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedFanoutWriter.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.Map; +import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; @@ -31,7 +32,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; abstract class PartitionedFanoutWriter extends BaseTaskWriter { - private final Map writers = Maps.newHashMap(); + private final Map writers = Maps.newHashMap(); PartitionedFanoutWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, OutputFileFactory fileFactory, FileIO io, long targetFileSize) { @@ -51,11 +52,11 @@ abstract class PartitionedFanoutWriter extends BaseTaskWriter { public void write(T row) throws IOException { PartitionKey partitionKey = partition(row); - RollingFileWriter writer = writers.get(partitionKey); + BaseRollingFileWriter 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); + writer = new RollingDataFileWriter(partitionKey); writers.put(copiedKey, writer); } 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..5505f9a1b7c8 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); 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..402d09823a8f 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()); 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..68870c04dbd9 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 @@ -120,7 +120,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/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..dc910b4fae1e 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 @@ -288,7 +288,7 @@ private static class Unpartitioned24Writer extends UnpartitionedWriter Date: Mon, 26 Oct 2020 20:37:48 +0800 Subject: [PATCH 02/41] Introduce DataFileWriterFactory and EqualityDeleteWriterFactory --- .../org/apache/iceberg/ContentFileWriter.java | 3 ++ .../iceberg/ContentFileWriterFactory.java | 28 ++++++++++++ .../apache/iceberg/DataFileWriterFactory.java | 43 +++++++++++++++++++ .../sink/FlinkEqualityDeleterFactory.java | 19 ++++---- 4 files changed, 84 insertions(+), 9 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/ContentFileWriterFactory.java create mode 100644 core/src/main/java/org/apache/iceberg/DataFileWriterFactory.java diff --git a/core/src/main/java/org/apache/iceberg/ContentFileWriter.java b/core/src/main/java/org/apache/iceberg/ContentFileWriter.java index 1f3f29b1c789..c1c0a546fa6f 100644 --- a/core/src/main/java/org/apache/iceberg/ContentFileWriter.java +++ b/core/src/main/java/org/apache/iceberg/ContentFileWriter.java @@ -46,5 +46,8 @@ default void writeAll(Iterable values) { */ long length(); + /** + * Return a {@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..338260b5eec1 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/ContentFileWriterFactory.java @@ -0,0 +1,28 @@ +/* + * 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 org.apache.iceberg.encryption.EncryptedOutputFile; + +public interface ContentFileWriterFactory { + + ContentFileWriter createWriter(EncryptedOutputFile outputFile, FileFormat fileFormat) throws IOException; +} diff --git a/core/src/main/java/org/apache/iceberg/DataFileWriterFactory.java b/core/src/main/java/org/apache/iceberg/DataFileWriterFactory.java new file mode 100644 index 000000000000..dc66349577f4 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/DataFileWriterFactory.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; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.FileAppenderFactory; + +public class DataFileWriterFactory implements ContentFileWriterFactory { + private final FileAppenderFactory appenderFactory; + private final PartitionKey partitionKey; + private final PartitionSpec spec; + + public DataFileWriterFactory(FileAppenderFactory appenderFactory, PartitionKey partitionKey, PartitionSpec spec) { + this.appenderFactory = appenderFactory; + this.partitionKey = partitionKey; + this.spec = spec; + } + + @Override + public ContentFileWriter createWriter(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/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkEqualityDeleterFactory.java b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkEqualityDeleterFactory.java index 75a924d5df3c..654a3adf9670 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkEqualityDeleterFactory.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkEqualityDeleterFactory.java @@ -26,19 +26,20 @@ import java.util.Map; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.ContentFileWriter; +import org.apache.iceberg.ContentFileWriterFactory; +import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.MetricsConfig; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.deletes.EqualityDeleteWriter; +import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.flink.data.FlinkAvroWriter; import org.apache.iceberg.flink.data.FlinkParquetWriters; -import org.apache.iceberg.io.EqualityDeleteWriterFactory; -import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.parquet.Parquet; -public class FlinkEqualityDeleterFactory implements EqualityDeleteWriterFactory, Serializable { +public class FlinkEqualityDeleterFactory implements ContentFileWriterFactory, Serializable { private final Schema schema; private final RowType flinkSchema; private final PartitionSpec spec; @@ -58,12 +59,12 @@ public FlinkEqualityDeleterFactory(Schema schema, } @Override - public EqualityDeleteWriter newEqualityDeleteWriter(OutputFile outputFile, FileFormat format) { + public ContentFileWriter createWriter(EncryptedOutputFile outputFile, FileFormat fileFormat) { MetricsConfig metricsConfig = MetricsConfig.fromProperties(props); try { - switch (format) { + switch (fileFormat) { case AVRO: - return Avro.writeDeletes(outputFile) + return Avro.writeDeletes(outputFile.encryptingOutputFile()) .createWriterFunc(ignore -> new FlinkAvroWriter(flinkSchema)) .overwrite() .setAll(props) @@ -73,7 +74,7 @@ public EqualityDeleteWriter newEqualityDeleteWriter(OutputFile outputFi .buildEqualityWriter(); case PARQUET: - return Parquet.writeDeletes(outputFile) + return Parquet.writeDeletes(outputFile.encryptingOutputFile()) .createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(flinkSchema, msgType)) .overwrite() .setAll(props) @@ -85,7 +86,7 @@ public EqualityDeleteWriter newEqualityDeleteWriter(OutputFile outputFi case ORC: default: - throw new UnsupportedOperationException("Cannot write unknown file format: " + format); + throw new UnsupportedOperationException("Cannot write unknown file format: " + fileFormat); } } catch (IOException e) { throw new UncheckedIOException(e); From a7e2e698e9ec038a97e9de2f27c750685bbc0f49 Mon Sep 17 00:00:00 2001 From: huzheng Date: Mon, 26 Oct 2020 21:20:18 +0800 Subject: [PATCH 03/41] Refactor the BaseTaskWriter to use the generic ContentFileWriterFactory. --- .../iceberg/ContentFileWriterFactory.java | 5 ++- .../apache/iceberg/DataFileWriterFactory.java | 8 ++-- .../org/apache/iceberg/io/BaseTaskWriter.java | 44 +++++-------------- .../apache/iceberg/io/PartitionedWriter.java | 15 +++---- .../iceberg/io/UnpartitionedWriter.java | 15 +++---- .../sink/FlinkEqualityDeleterFactory.java | 7 ++- .../flink/sink/PartitionedFanoutWriter.java | 18 ++++---- .../flink/sink/RowDataTaskWriterFactory.java | 9 ++-- .../iceberg/spark/source/RowDataRewriter.java | 4 +- .../spark/source/SparkPartitionedWriter.java | 6 ++- .../apache/iceberg/spark/source/Writer.java | 5 ++- .../iceberg/spark/source/SparkWrite.java | 5 ++- 12 files changed, 66 insertions(+), 75 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/ContentFileWriterFactory.java b/core/src/main/java/org/apache/iceberg/ContentFileWriterFactory.java index 338260b5eec1..eed0f3ddae89 100644 --- a/core/src/main/java/org/apache/iceberg/ContentFileWriterFactory.java +++ b/core/src/main/java/org/apache/iceberg/ContentFileWriterFactory.java @@ -19,10 +19,11 @@ package org.apache.iceberg; -import java.io.IOException; import org.apache.iceberg.encryption.EncryptedOutputFile; public interface ContentFileWriterFactory { - ContentFileWriter createWriter(EncryptedOutputFile outputFile, FileFormat fileFormat) throws IOException; + ContentFileWriter createWriter(PartitionKey partitionKey, + EncryptedOutputFile outputFile, + FileFormat fileFormat); } diff --git a/core/src/main/java/org/apache/iceberg/DataFileWriterFactory.java b/core/src/main/java/org/apache/iceberg/DataFileWriterFactory.java index dc66349577f4..366970593823 100644 --- a/core/src/main/java/org/apache/iceberg/DataFileWriterFactory.java +++ b/core/src/main/java/org/apache/iceberg/DataFileWriterFactory.java @@ -25,17 +25,17 @@ public class DataFileWriterFactory implements ContentFileWriterFactory { private final FileAppenderFactory appenderFactory; - private final PartitionKey partitionKey; private final PartitionSpec spec; - public DataFileWriterFactory(FileAppenderFactory appenderFactory, PartitionKey partitionKey, PartitionSpec spec) { + public DataFileWriterFactory(FileAppenderFactory appenderFactory, PartitionSpec spec) { this.appenderFactory = appenderFactory; - this.partitionKey = partitionKey; this.spec = spec; } @Override - public ContentFileWriter createWriter(EncryptedOutputFile outputFile, FileFormat fileFormat) { + 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/BaseTaskWriter.java b/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java index 696dd0f63c72..6194d16b69c8 100644 --- a/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java @@ -23,33 +23,29 @@ import java.io.IOException; import org.apache.iceberg.ContentFile; import org.apache.iceberg.ContentFileWriter; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFileWriter; +import org.apache.iceberg.ContentFileWriterFactory; 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.util.Tasks; -public abstract class BaseTaskWriter implements TaskWriter { +public abstract class BaseTaskWriter implements TaskWriter { private final TaskWriterResult.Builder builder; - private final PartitionSpec spec; private final FileFormat format; - private final FileAppenderFactory appenderFactory; private final OutputFileFactory fileFactory; private final FileIO io; private final long targetFileSize; + private final ContentFileWriterFactory writerFactory; - protected BaseTaskWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, FileIO io, long targetFileSize) { + protected BaseTaskWriter(FileFormat format, OutputFileFactory fileFactory, FileIO io, long targetFileSize, + ContentFileWriterFactory writerFactory) { this.builder = TaskWriterResult.builder(); - this.spec = spec; this.format = format; - this.appenderFactory = appenderFactory; this.fileFactory = fileFactory; this.io = io; this.targetFileSize = targetFileSize; + this.writerFactory = writerFactory; } @Override @@ -77,29 +73,15 @@ public TaskWriterResult complete() throws IOException { return builder.build(); } - protected class RollingDataFileWriter extends BaseRollingFileWriter { - - public RollingDataFileWriter(PartitionKey partitionKey) { - super(partitionKey); - } - - @Override - ContentFileWriter newContentFileWriter(EncryptedOutputFile outputFile, FileFormat fileFormat) { - FileAppender appender = appenderFactory.newAppender(outputFile.encryptingOutputFile(), fileFormat); - return new DataFileWriter<>(appender, fileFormat, outputFile.encryptingOutputFile().location(), partitionKey(), - spec, outputFile.keyMetadata()); - } - } - - protected abstract class BaseRollingFileWriter implements Closeable { + protected class RollingFileWriter implements Closeable { private static final int ROWS_DIVISOR = 1000; private final PartitionKey partitionKey; private EncryptedOutputFile currentFile = null; - private ContentFileWriter currentFileWriter = null; + private ContentFileWriter currentFileWriter = null; private long currentRows = 0; - public BaseRollingFileWriter(PartitionKey partitionKey) { + public RollingFileWriter(PartitionKey partitionKey) { this.partitionKey = partitionKey; openCurrent(); } @@ -108,7 +90,7 @@ protected PartitionKey partitionKey() { return partitionKey; } - public void add(R record) throws IOException { + public void add(T record) throws IOException { this.currentFileWriter.write(record); this.currentRows++; @@ -118,8 +100,6 @@ public void add(R record) throws IOException { } } - abstract ContentFileWriter newContentFileWriter(EncryptedOutputFile outputFile, FileFormat fileFormat); - private void openCurrent() { if (partitionKey == null) { // unpartitioned @@ -128,7 +108,7 @@ private void openCurrent() { // partitioned currentFile = fileFactory.newOutputFile(partitionKey); } - currentFileWriter = newContentFileWriter(currentFile, format); + currentFileWriter = writerFactory.createWriter(partitionKey, currentFile, format); currentRows = 0; } @@ -142,7 +122,7 @@ private boolean shouldRollToNewFile() { private void closeCurrent() throws IOException { if (currentFileWriter != null) { currentFileWriter.close(); - F contentFile = currentFileWriter.toContentFile(); + ContentFileT contentFile = currentFileWriter.toContentFile(); Metrics metrics = currentFileWriter.metrics(); this.currentFileWriter = null; 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 8161e893e909..ef786121c4ae 100644 --- a/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java @@ -21,26 +21,25 @@ import java.io.IOException; import java.util.Set; -import org.apache.iceberg.DataFile; +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.slf4j.Logger; import org.slf4j.LoggerFactory; -public abstract class PartitionedWriter extends BaseTaskWriter { +public abstract class PartitionedWriter extends BaseTaskWriter { private static final Logger LOG = LoggerFactory.getLogger(PartitionedWriter.class); private final Set completedPartitions = Sets.newHashSet(); private PartitionKey currentKey = null; - private BaseRollingFileWriter currentWriter = null; + private RollingFileWriter 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) { + super(format, fileFactory, io, targetFileSize, writerFactory); } /** @@ -71,7 +70,7 @@ public void write(T row) throws IOException { } currentKey = key.copy(); - currentWriter = new RollingDataFileWriter(currentKey); + currentWriter = new RollingFileWriter(currentKey); } currentWriter.add(row); 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 7b08761fc5ab..994fe259ef18 100644 --- a/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java @@ -20,18 +20,17 @@ package org.apache.iceberg.io; import java.io.IOException; -import org.apache.iceberg.DataFile; +import org.apache.iceberg.ContentFileWriterFactory; import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -public class UnpartitionedWriter extends BaseTaskWriter { +public class UnpartitionedWriter extends BaseTaskWriter { - private final BaseRollingFileWriter currentWriter; + private final RollingFileWriter currentWriter; - public UnpartitionedWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, FileIO io, long targetFileSize) { - super(spec, format, appenderFactory, fileFactory, io, targetFileSize); - currentWriter = new RollingDataFileWriter(null); + public UnpartitionedWriter(FileFormat format, OutputFileFactory fileFactory, FileIO io, long targetFileSize, + ContentFileWriterFactory writerFactory) { + super(format, fileFactory, io, targetFileSize, writerFactory); + currentWriter = new RollingFileWriter(null); } @Override diff --git a/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkEqualityDeleterFactory.java b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkEqualityDeleterFactory.java index 654a3adf9670..b0a8e9c5d139 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkEqualityDeleterFactory.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkEqualityDeleterFactory.java @@ -31,6 +31,7 @@ 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.avro.Avro; @@ -59,13 +60,16 @@ public FlinkEqualityDeleterFactory(Schema schema, } @Override - public ContentFileWriter createWriter(EncryptedOutputFile outputFile, FileFormat fileFormat) { + public ContentFileWriter createWriter(PartitionKey partitionKey, + EncryptedOutputFile outputFile, + FileFormat fileFormat) { MetricsConfig metricsConfig = MetricsConfig.fromProperties(props); try { switch (fileFormat) { case AVRO: return Avro.writeDeletes(outputFile.encryptingOutputFile()) .createWriterFunc(ignore -> new FlinkAvroWriter(flinkSchema)) + .withPartition(partitionKey) .overwrite() .setAll(props) .rowSchema(schema) @@ -76,6 +80,7 @@ public ContentFileWriter createWriter(EncryptedOutputFile o case PARQUET: return Parquet.writeDeletes(outputFile.encryptingOutputFile()) .createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(flinkSchema, msgType)) + .withPartition(partitionKey) .overwrite() .setAll(props) .metricsConfig(metricsConfig) 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 index 77eb8d788fc6..ebdf4314b11c 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedFanoutWriter.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedFanoutWriter.java @@ -21,22 +21,20 @@ import java.io.IOException; import java.util.Map; -import org.apache.iceberg.DataFile; +import org.apache.iceberg.ContentFileWriterFactory; 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(); +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); + PartitionedFanoutWriter(FileFormat format, OutputFileFactory fileFactory, FileIO io, long targetFileSize, + ContentFileWriterFactory writerFactory) { + super(format, fileFactory, io, targetFileSize, writerFactory); } /** @@ -52,11 +50,11 @@ abstract class PartitionedFanoutWriter extends BaseTaskWriter { public void write(T row) throws IOException { PartitionKey partitionKey = partition(row); - BaseRollingFileWriter writer = writers.get(partitionKey); + 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 RollingDataFileWriter(partitionKey); + writer = new RollingFileWriter(partitionKey); writers.put(copiedKey, writer); } 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..303742b4e86c 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 @@ -25,6 +25,8 @@ import java.util.Map; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFileWriterFactory; import org.apache.iceberg.FileFormat; import org.apache.iceberg.MetricsConfig; import org.apache.iceberg.PartitionKey; @@ -92,14 +94,15 @@ public TaskWriter create() { "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); + return new UnpartitionedWriter<>(format, outputFileFactory, io, + targetFileSizeBytes, new DataFileWriterFactory<>(appenderFactory, spec)); } else { return new RowDataPartitionedFanoutWriter(spec, format, appenderFactory, outputFileFactory, io, targetFileSizeBytes, schema, flinkSchema); } } - private static class RowDataPartitionedFanoutWriter extends PartitionedFanoutWriter { + private static class RowDataPartitionedFanoutWriter extends PartitionedFanoutWriter { private final PartitionKey partitionKey; private final RowDataWrapper rowDataWrapper; @@ -107,7 +110,7 @@ private static class RowDataPartitionedFanoutWriter extends PartitionedFanoutWri RowDataPartitionedFanoutWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, RowType flinkSchema) { - super(spec, format, appenderFactory, fileFactory, io, targetFileSize); + super(format, fileFactory, io, targetFileSize, new DataFileWriterFactory<>(appenderFactory, spec)); this.partitionKey = new PartitionKey(spec, schema); this.rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); } 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 68870c04dbd9..8d3e46f9622c 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 @@ -27,6 +27,7 @@ import java.util.stream.Collectors; import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFileWriterFactory; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -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); 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..7e23bd4bd37e 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,6 +19,8 @@ package org.apache.iceberg.spark.source; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFileWriterFactory; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; @@ -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 dc910b4fae1e..41c08fddabf7 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 @@ -26,6 +26,7 @@ import java.util.Optional; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFileWriterFactory; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.ReplacePartitions; @@ -277,11 +278,11 @@ 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 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 6c5994660dab..7c4b127e4c16 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 @@ -26,6 +26,7 @@ import java.util.Optional; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFileWriterFactory; import org.apache.iceberg.FileFormat; import org.apache.iceberg.OverwriteFiles; import org.apache.iceberg.PartitionSpec; @@ -426,11 +427,11 @@ 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 From 7f06d23448caba2f29561d09ab9125a412721fb6 Mon Sep 17 00:00:00 2001 From: huzheng Date: Mon, 26 Oct 2020 22:00:26 +0800 Subject: [PATCH 04/41] Introduce the MixedTaskWriter to accept both INSERT and DELETE records. --- .../apache/iceberg/io/TaskWriterResult.java | 21 +++ .../flink/sink/RowDataTaskWriterFactory.java | 126 ++++++++++++++++-- 2 files changed, 137 insertions(+), 10 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java b/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java index 0200cd95a51c..4c48e584b920 100644 --- a/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java +++ b/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java @@ -59,6 +59,27 @@ public static Builder builder() { return new Builder(); } + public static TaskWriterResult concat(TaskWriterResult result0, TaskWriterResult result1) { + Builder builder = new Builder(); + for (DataFile dataFile : result0.dataFiles) { + builder.add(dataFile); + } + + for (DataFile dataFile : result1.dataFiles) { + builder.add(dataFile); + } + + for (DeleteFile deleteFile : result0.deleteFiles) { + builder.add(deleteFile); + } + + for (DeleteFile deleteFile : result1.deleteFiles) { + builder.add(deleteFile); + } + + return builder.build(); + } + public static class Builder { private final List dataFiles; private final List deleteFiles; 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 303742b4e86c..08d5006538fb 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 @@ -22,10 +22,11 @@ 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.DataFile; +import org.apache.iceberg.ContentFileWriterFactory; import org.apache.iceberg.DataFileWriterFactory; import org.apache.iceberg.FileFormat; import org.apache.iceberg.MetricsConfig; @@ -45,10 +46,12 @@ import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.io.OutputFileFactory; import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.TaskWriterResult; 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; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; public class RowDataTaskWriterFactory implements TaskWriterFactory { private final Schema schema; @@ -59,6 +62,7 @@ public class RowDataTaskWriterFactory implements TaskWriterFactory { private final EncryptionManager encryptionManager; private final long targetFileSizeBytes; private final FileFormat format; + private final Map tableProperties; private final FileAppenderFactory appenderFactory; private transient OutputFileFactory outputFileFactory; @@ -80,6 +84,7 @@ public RowDataTaskWriterFactory(Schema schema, this.encryptionManager = encryptionManager; this.targetFileSizeBytes = targetFileSizeBytes; this.format = format; + this.tableProperties = tableProperties; this.appenderFactory = new FlinkFileAppenderFactory(schema, flinkSchema, tableProperties); } @@ -94,23 +99,124 @@ public TaskWriter create() { "The outputFileFactory shouldn't be null if we have invoked the initialize()."); if (spec.fields().isEmpty()) { - return new UnpartitionedWriter<>(format, outputFileFactory, io, - targetFileSizeBytes, new DataFileWriterFactory<>(appenderFactory, spec)); + return new MixedUnpartitionedTaskWriter(format, appenderFactory, outputFileFactory, io, targetFileSizeBytes, + schema, flinkSchema, tableProperties); } else { - return new RowDataPartitionedFanoutWriter(spec, format, appenderFactory, outputFileFactory, - io, targetFileSizeBytes, schema, flinkSchema); + return new MixedPartitionedTaskWriter(spec, format, appenderFactory, outputFileFactory, + io, targetFileSizeBytes, schema, flinkSchema, tableProperties); } } - private static class RowDataPartitionedFanoutWriter extends PartitionedFanoutWriter { + private abstract static class BaseMixedTaskWriter implements TaskWriter { + + protected abstract TaskWriter dataTaskWriter(); + + protected abstract TaskWriter deleteTaskWriter(); + + @Override + public void write(RowData row) throws IOException { + switch (row.getRowKind()) { + case INSERT: + case UPDATE_AFTER: + dataTaskWriter().write(row); + break; + case UPDATE_BEFORE: + case DELETE: + deleteTaskWriter().write(row); + break; + default: + throw new UnsupportedOperationException("Unrecognized row kind: " + row.getRowKind()); + } + } + + @Override + public void abort() throws IOException { + dataTaskWriter().abort(); + deleteTaskWriter().abort(); + } + + @Override + public TaskWriterResult complete() throws IOException { + return TaskWriterResult.concat(dataTaskWriter().complete(), deleteTaskWriter().complete()); + } + + @Override + public void close() throws IOException { + dataTaskWriter().close(); + deleteTaskWriter().close(); + } + } + + private static class MixedUnpartitionedTaskWriter extends BaseMixedTaskWriter { + private final TaskWriter dataTaskWriter; + private final TaskWriter deleteTaskWriter; + + MixedUnpartitionedTaskWriter(FileFormat format, FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, + RowType flinkSchema, Map tableProperties) { + this.dataTaskWriter = new UnpartitionedWriter<>(format, fileFactory, io, + targetFileSize, new DataFileWriterFactory<>(appenderFactory, PartitionSpec.unpartitioned())); + + // TODO: set the correct equality field ids. + List equalityIds = ImmutableList.of(); + + this.deleteTaskWriter = new UnpartitionedWriter<>(format, fileFactory, io, targetFileSize, + new FlinkEqualityDeleterFactory(schema, flinkSchema, PartitionSpec.unpartitioned(), equalityIds, + tableProperties)); + } + + @Override + protected TaskWriter dataTaskWriter() { + return dataTaskWriter; + } + + @Override + protected TaskWriter deleteTaskWriter() { + return deleteTaskWriter; + } + } + + private static class MixedPartitionedTaskWriter extends BaseMixedTaskWriter { + private final TaskWriter dataTaskWriter; + private final TaskWriter deleteTaskWriter; + + MixedPartitionedTaskWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, + RowType flinkSchema, Map tableProperties) { + this.dataTaskWriter = + new RowDataPartitionedFanoutWriter<>(spec, format, fileFactory, io, targetFileSize, schema, + flinkSchema, new DataFileWriterFactory<>(appenderFactory, spec)); + + // TODO: set the correct equality field ids. + List equalityIds = ImmutableList.of(); + + this.deleteTaskWriter = + new RowDataPartitionedFanoutWriter<>(spec, format, fileFactory, io, targetFileSize, schema, + flinkSchema, new FlinkEqualityDeleterFactory(schema, flinkSchema, spec, equalityIds, tableProperties)); + + } + + @Override + protected TaskWriter dataTaskWriter() { + return dataTaskWriter; + } + + @Override + protected TaskWriter deleteTaskWriter() { + return deleteTaskWriter; + } + } + + 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(format, fileFactory, io, targetFileSize, new DataFileWriterFactory<>(appenderFactory, spec)); + RowDataPartitionedFanoutWriter(PartitionSpec spec, FileFormat format, OutputFileFactory fileFactory, FileIO io, + long targetFileSize, Schema schema, RowType flinkSchema, + ContentFileWriterFactory contentFileWriter) { + super(format, fileFactory, io, targetFileSize, contentFileWriter); this.partitionKey = new PartitionKey(spec, schema); this.rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); } From 7e4fcf202db29380b3842834753c80dda04e01b5 Mon Sep 17 00:00:00 2001 From: huzheng Date: Tue, 27 Oct 2020 10:44:33 +0800 Subject: [PATCH 05/41] Minor fixes --- .../org/apache/iceberg/io/BaseTaskWriter.java | 7 +- .../io/EqualityDeleteWriterFactory.java | 33 --------- .../org/apache/iceberg/io/TaskWriter.java | 2 +- .../apache/iceberg/io/TaskWriterResult.java | 40 ++++------- .../flink/sink/PartitionedFanoutWriter.java | 2 +- .../flink/sink/RowDataTaskWriterFactory.java | 72 ++++++++++++++----- 6 files changed, 71 insertions(+), 85 deletions(-) delete mode 100644 core/src/main/java/org/apache/iceberg/io/EqualityDeleteWriterFactory.java diff --git a/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java b/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java index 6194d16b69c8..ff385047754d 100644 --- a/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java @@ -86,10 +86,6 @@ public RollingFileWriter(PartitionKey partitionKey) { openCurrent(); } - protected PartitionKey partitionKey() { - return partitionKey; - } - public void add(T record) throws IOException { this.currentFileWriter.write(record); this.currentRows++; @@ -118,7 +114,6 @@ private boolean shouldRollToNewFile() { currentRows % ROWS_DIVISOR == 0 && currentFileWriter.length() >= targetFileSize; } - @SuppressWarnings("unchecked") private void closeCurrent() throws IOException { if (currentFileWriter != null) { currentFileWriter.close(); @@ -129,7 +124,7 @@ private void closeCurrent() throws IOException { if (metrics.recordCount() == 0L) { io.deleteFile(currentFile.encryptingOutputFile()); } else if (contentFile instanceof ContentFile) { - builder.add((ContentFile) contentFile); + builder.add((ContentFile) contentFile); } else { throw new RuntimeException(String.format( "The newly generated content file must be DataFile or DeleteFile: %s", contentFile)); diff --git a/core/src/main/java/org/apache/iceberg/io/EqualityDeleteWriterFactory.java b/core/src/main/java/org/apache/iceberg/io/EqualityDeleteWriterFactory.java deleted file mode 100644 index 5c1b52d2bbfb..000000000000 --- a/core/src/main/java/org/apache/iceberg/io/EqualityDeleteWriterFactory.java +++ /dev/null @@ -1,33 +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 org.apache.iceberg.FileFormat; -import org.apache.iceberg.deletes.EqualityDeleteWriter; - -/** - * Factory to create a new {@link EqualityDeleteWriter} to write equality deletions in row-level. - * - * @param data type of the rows to delete. - */ -public interface EqualityDeleteWriterFactory { - - EqualityDeleteWriter newEqualityDeleteWriter(OutputFile outputFile, FileFormat format); -} 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 8eb4b7f95e78..36a699fe8803 100644 --- a/core/src/main/java/org/apache/iceberg/io/TaskWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/TaskWriter.java @@ -42,7 +42,7 @@ public interface TaskWriter extends Closeable { void abort() throws IOException; /** - * Close the writer and get the completed data files. + * Close the writer and get the completed data/delete files. * * @return the completed data files of this task writer. */ diff --git a/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java b/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java index 4c48e584b920..6917e562c9b0 100644 --- a/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java +++ b/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java @@ -24,29 +24,17 @@ 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.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; public class TaskWriterResult { - private static final TaskWriterResult EMPTY = new TaskWriterResult(ImmutableList.of(), ImmutableList.of()); - private DataFile[] dataFiles; private DeleteFile[] deleteFiles; - TaskWriterResult(DataFile[] dataFiles, DeleteFile[] deleteFiles) { - this.dataFiles = dataFiles; - this.deleteFiles = deleteFiles; - } - - TaskWriterResult(List dataFiles, List deleteFiles) { + private TaskWriterResult(List dataFiles, List deleteFiles) { this.dataFiles = dataFiles.toArray(new DataFile[0]); this.deleteFiles = deleteFiles.toArray(new DeleteFile[0]); } - static TaskWriterResult empty() { - return EMPTY; - } - public DataFile[] dataFiles() { return dataFiles; } @@ -61,21 +49,11 @@ public static Builder builder() { public static TaskWriterResult concat(TaskWriterResult result0, TaskWriterResult result1) { Builder builder = new Builder(); - for (DataFile dataFile : result0.dataFiles) { - builder.add(dataFile); - } - - for (DataFile dataFile : result1.dataFiles) { - builder.add(dataFile); - } - - for (DeleteFile deleteFile : result0.deleteFiles) { - builder.add(deleteFile); - } - for (DeleteFile deleteFile : result1.deleteFiles) { - builder.add(deleteFile); - } + builder.addAll(result0.dataFiles); + builder.addAll(result0.deleteFiles); + builder.addAll(result1.dataFiles); + builder.addAll(result1.deleteFiles); return builder.build(); } @@ -89,6 +67,12 @@ private Builder() { this.deleteFiles = Lists.newArrayList(); } + public void addAll(ContentFile... files) { + for (ContentFile file : files) { + add(file); + } + } + public void add(ContentFile contentFile) { Preconditions.checkNotNull(contentFile, "Content file shouldn't be null."); switch (contentFile.content()) { @@ -102,7 +86,7 @@ public void add(ContentFile contentFile) { break; default: - throw new UnsupportedOperationException("Unknown file: " + contentFile.content()); + throw new UnsupportedOperationException("Unknown file content: " + contentFile.content()); } } 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 index ebdf4314b11c..74c3bd570a91 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedFanoutWriter.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedFanoutWriter.java @@ -54,7 +54,7 @@ public void write(T row) throws IOException { 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(partitionKey); + writer = new RollingFileWriter(copiedKey); writers.put(copiedKey, writer); } 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 08d5006538fb..aee79a96695e 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 @@ -109,21 +109,30 @@ public TaskWriter create() { private abstract static class BaseMixedTaskWriter implements TaskWriter { - protected abstract TaskWriter dataTaskWriter(); + abstract boolean supportDeletion(); - protected abstract TaskWriter deleteTaskWriter(); + abstract TaskWriter dataTaskWriter(); + + abstract TaskWriter deleteTaskWriter(); @Override public void write(RowData row) throws IOException { switch (row.getRowKind()) { + case INSERT: case UPDATE_AFTER: dataTaskWriter().write(row); break; + case UPDATE_BEFORE: case DELETE: - deleteTaskWriter().write(row); + if (supportDeletion()) { + deleteTaskWriter().write(row); + } else { + throw new UnsupportedOperationException("Couldn't accept deletion in task writer."); + } break; + default: throw new UnsupportedOperationException("Unrecognized row kind: " + row.getRowKind()); } @@ -132,57 +141,80 @@ public void write(RowData row) throws IOException { @Override public void abort() throws IOException { dataTaskWriter().abort(); - deleteTaskWriter().abort(); + if (supportDeletion()) { + deleteTaskWriter().abort(); + } } @Override public TaskWriterResult complete() throws IOException { - return TaskWriterResult.concat(dataTaskWriter().complete(), deleteTaskWriter().complete()); + if (supportDeletion()) { + return TaskWriterResult.concat(dataTaskWriter().complete(), deleteTaskWriter().complete()); + } else { + return dataTaskWriter().complete(); + } } @Override public void close() throws IOException { dataTaskWriter().close(); - deleteTaskWriter().close(); + if (supportDeletion()) { + deleteTaskWriter().close(); + } } } private static class MixedUnpartitionedTaskWriter extends BaseMixedTaskWriter { + private final FileFormat format; private final TaskWriter dataTaskWriter; private final TaskWriter deleteTaskWriter; MixedUnpartitionedTaskWriter(FileFormat format, FileAppenderFactory appenderFactory, OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, RowType flinkSchema, Map tableProperties) { + this.format = format; + this.dataTaskWriter = new UnpartitionedWriter<>(format, fileFactory, io, targetFileSize, new DataFileWriterFactory<>(appenderFactory, PartitionSpec.unpartitioned())); // TODO: set the correct equality field ids. List equalityIds = ImmutableList.of(); - this.deleteTaskWriter = new UnpartitionedWriter<>(format, fileFactory, io, targetFileSize, - new FlinkEqualityDeleterFactory(schema, flinkSchema, PartitionSpec.unpartitioned(), equalityIds, - tableProperties)); + if (supportDeletion()) { + this.deleteTaskWriter = new UnpartitionedWriter<>(format, fileFactory, io, targetFileSize, + new FlinkEqualityDeleterFactory(schema, flinkSchema, PartitionSpec.unpartitioned(), equalityIds, + tableProperties)); + } else { + this.deleteTaskWriter = null; + } } @Override - protected TaskWriter dataTaskWriter() { + boolean supportDeletion() { + return FileFormat.PARQUET.equals(format) || FileFormat.AVRO.equals(format); + } + + @Override + TaskWriter dataTaskWriter() { return dataTaskWriter; } @Override - protected TaskWriter deleteTaskWriter() { + TaskWriter deleteTaskWriter() { return deleteTaskWriter; } } private static class MixedPartitionedTaskWriter extends BaseMixedTaskWriter { + private final FileFormat format; private final TaskWriter dataTaskWriter; private final TaskWriter deleteTaskWriter; MixedPartitionedTaskWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, RowType flinkSchema, Map tableProperties) { + this.format = format; + this.dataTaskWriter = new RowDataPartitionedFanoutWriter<>(spec, format, fileFactory, io, targetFileSize, schema, flinkSchema, new DataFileWriterFactory<>(appenderFactory, spec)); @@ -190,19 +222,27 @@ private static class MixedPartitionedTaskWriter extends BaseMixedTaskWriter { // TODO: set the correct equality field ids. List equalityIds = ImmutableList.of(); - this.deleteTaskWriter = - new RowDataPartitionedFanoutWriter<>(spec, format, fileFactory, io, targetFileSize, schema, - flinkSchema, new FlinkEqualityDeleterFactory(schema, flinkSchema, spec, equalityIds, tableProperties)); + if (supportDeletion()) { + this.deleteTaskWriter = + new RowDataPartitionedFanoutWriter<>(spec, format, fileFactory, io, targetFileSize, schema, + flinkSchema, new FlinkEqualityDeleterFactory(schema, flinkSchema, spec, equalityIds, tableProperties)); + } else { + this.deleteTaskWriter = null; + } + } + @Override + boolean supportDeletion() { + return FileFormat.PARQUET.equals(format) || FileFormat.AVRO.equals(format); } @Override - protected TaskWriter dataTaskWriter() { + TaskWriter dataTaskWriter() { return dataTaskWriter; } @Override - protected TaskWriter deleteTaskWriter() { + TaskWriter deleteTaskWriter() { return deleteTaskWriter; } } From 6f742f081bae3b230a99ab20f1413a5f5c3f3f28 Mon Sep 17 00:00:00 2001 From: huzheng Date: Wed, 28 Oct 2020 15:48:38 +0800 Subject: [PATCH 06/41] Add contentFiles in TaskWriterResult. --- .../org/apache/iceberg/io/BaseTaskWriter.java | 7 +---- .../apache/iceberg/io/TaskWriterResult.java | 27 +++++++++++++++++++ 2 files changed, 28 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java b/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java index ff385047754d..5aaa2923e263 100644 --- a/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java @@ -55,12 +55,7 @@ public void abort() throws IOException { // clean up files created by this writer TaskWriterResult result = builder.build(); - Tasks.foreach(result.dataFiles()) - .throwFailureWhenFinished() - .noRetry() - .run(file -> io.deleteFile(file.path().toString())); - - Tasks.foreach(result.deleteFiles()) + Tasks.foreach(result.contentFiles()) .throwFailureWhenFinished() .noRetry() .run(file -> io.deleteFile(file.path().toString())); diff --git a/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java b/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java index 6917e562c9b0..3afa23435428 100644 --- a/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java +++ b/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java @@ -19,7 +19,9 @@ package org.apache.iceberg.io; +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; @@ -43,6 +45,31 @@ 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 Builder builder() { return new Builder(); } From 9a58c5c23db242c5902be45c66d086601fc50e60 Mon Sep 17 00:00:00 2001 From: huzheng Date: Fri, 30 Oct 2020 11:01:03 +0800 Subject: [PATCH 07/41] temp --- .../iceberg/deletes/PositionDeleteWriter.java | 23 ++- ...skWriter.java => BaseFileGroupWriter.java} | 22 ++- .../iceberg/io/BasePartitionWriter.java | 142 ++++++++++++++++++ .../{TaskWriter.java => FileGroupWriter.java} | 14 +- .../apache/iceberg/io/PartitionWriter.java | 35 +++++ .../apache/iceberg/io/PartitionedWriter.java | 2 +- .../iceberg/io/UnpartitionedWriter.java | 12 +- ...askWriterResult.java => WriterResult.java} | 22 ++- .../iceberg/avro/TestAvroDeleteWriters.java | 4 +- .../org/apache/iceberg/data/FileHelpers.java | 2 +- .../FlinkPositionDeleteWriterFactory.java | 87 +++++++++++ .../flink/sink/IcebergStreamWriter.java | 4 +- .../flink/sink/PartitionedFanoutWriter.java | 4 +- .../flink/sink/RowDataTaskWriterFactory.java | 126 +++++++++++----- .../iceberg/flink/sink/TaskWriterFactory.java | 8 +- .../iceberg/flink/sink/TestTaskWriters.java | 64 ++++---- .../parquet/TestParquetDeleteWriters.java | 4 +- .../iceberg/spark/source/RowDataRewriter.java | 4 +- 18 files changed, 474 insertions(+), 105 deletions(-) rename core/src/main/java/org/apache/iceberg/io/{BaseTaskWriter.java => BaseFileGroupWriter.java} (85%) create mode 100644 core/src/main/java/org/apache/iceberg/io/BasePartitionWriter.java rename core/src/main/java/org/apache/iceberg/io/{TaskWriter.java => FileGroupWriter.java} (81%) create mode 100644 core/src/main/java/org/apache/iceberg/io/PartitionWriter.java rename core/src/main/java/org/apache/iceberg/io/{TaskWriterResult.java => WriterResult.java} (85%) create mode 100644 flink/src/main/java/org/apache/iceberg/flink/sink/FlinkPositionDeleteWriterFactory.java 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..a40f711b0c11 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteWriter.java +++ b/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteWriter.java @@ -19,13 +19,14 @@ 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; +import org.apache.iceberg.Metrics; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.StructLike; import org.apache.iceberg.encryption.EncryptionKeyMetadata; @@ -33,7 +34,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,8 +86,24 @@ public Set referencedDataFiles() { return pathSet; } - public DeleteFile toDeleteFile() { + @Override + public DeleteFile toContentFile() { Preconditions.checkState(deleteFile != null, "Cannot create delete file from unclosed writer"); return deleteFile; } + + @Override + public void write(PositionDelete record) { + delete(record.path(), record.pos(), record.row()); + } + + @Override + public Metrics metrics() { + return appender.metrics(); + } + + @Override + public long length() { + return appender.length(); + } } diff --git a/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java b/core/src/main/java/org/apache/iceberg/io/BaseFileGroupWriter.java similarity index 85% rename from core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java rename to core/src/main/java/org/apache/iceberg/io/BaseFileGroupWriter.java index 5aaa2923e263..a2b68dac32de 100644 --- a/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BaseFileGroupWriter.java @@ -30,17 +30,17 @@ import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.util.Tasks; -public abstract class BaseTaskWriter implements TaskWriter { - private final TaskWriterResult.Builder builder; +public abstract class BaseFileGroupWriter implements FileGroupWriter { + private final WriterResult.Builder builder; private final FileFormat format; private final OutputFileFactory fileFactory; private final FileIO io; private final long targetFileSize; private final ContentFileWriterFactory writerFactory; - protected BaseTaskWriter(FileFormat format, OutputFileFactory fileFactory, FileIO io, long targetFileSize, - ContentFileWriterFactory writerFactory) { - this.builder = TaskWriterResult.builder(); + protected BaseFileGroupWriter(FileFormat format, OutputFileFactory fileFactory, FileIO io, long targetFileSize, + ContentFileWriterFactory writerFactory) { + this.builder = WriterResult.builder(); this.format = format; this.fileFactory = fileFactory; this.io = io; @@ -53,7 +53,7 @@ public void abort() throws IOException { close(); // clean up files created by this writer - TaskWriterResult result = builder.build(); + WriterResult result = builder.build(); Tasks.foreach(result.contentFiles()) .throwFailureWhenFinished() @@ -62,7 +62,7 @@ public void abort() throws IOException { } @Override - public TaskWriterResult complete() throws IOException { + public WriterResult complete() throws IOException { close(); return builder.build(); @@ -91,6 +91,14 @@ public void add(T record) throws IOException { } } + public CharSequence currentPath() { + return currentFile != null ? currentFile.encryptingOutputFile().location() : null; + } + + public long currentPos() { + return currentRows; + } + private void openCurrent() { if (partitionKey == null) { // unpartitioned diff --git a/core/src/main/java/org/apache/iceberg/io/BasePartitionWriter.java b/core/src/main/java/org/apache/iceberg/io/BasePartitionWriter.java new file mode 100644 index 000000000000..bbe865ab0096 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/io/BasePartitionWriter.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.io; + +import java.io.IOException; +import java.util.List; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.StructLikeSet; +import org.apache.iceberg.util.StructProjection; + +public abstract class BasePartitionWriter implements PartitionWriter { + + private final FileGroupWriter dataWriter; + private final FileGroupWriter equalityDeleteWriter; + private final FileGroupWriter> positionDeleteWriter; + private final StructProjection projectRow; + private final Schema deleteSchema; + private final List equalityFieldIds; + private final PositionDelete positionDelete = new PositionDelete<>(); + private final StructLikeSet insertedDataSet; + + public BasePartitionWriter(FileGroupWriter dataWriter) { + this(dataWriter, null); + } + + public BasePartitionWriter(FileGroupWriter dataWriter, + FileGroupWriter> positionDeleteWriter) { + this(dataWriter, positionDeleteWriter, null, null, null); + } + + public BasePartitionWriter(FileGroupWriter dataWriter, + FileGroupWriter> positionDeleteWriter, + FileGroupWriter equalityDeleteWriter, + Schema writeSchema, + List equalityFieldIds) { + this.dataWriter = dataWriter; + this.positionDeleteWriter = positionDeleteWriter; + this.equalityDeleteWriter = equalityDeleteWriter; + this.deleteSchema = TypeUtil.select(writeSchema, Sets.newHashSet(equalityFieldIds)); + this.equalityFieldIds = equalityFieldIds; + this.projectRow = StructProjection.create(writeSchema, deleteSchema); + this.insertedDataSet = StructLikeSet.create(deleteSchema.asStruct()); + } + + protected abstract StructLike asStructLike(T record); + + @Override + public void append(T row) throws IOException { + this.dataWriter.write(row); + + if (allowEqualityDelete()) { + // TODO Put the > into the insert data MAP, rather than SET. + insertedDataSet.add(projectRow.wrap(asStructLike(row))); + } + } + + @Override + public void delete(T deleteRow) throws IOException { + if (!allowEqualityDelete()) { + throw new UnsupportedOperationException("Couldn't accept equality deletions."); + } + + if (!insertedDataSet.contains(projectRow.wrap(asStructLike(deleteRow)))) { + this.equalityDeleteWriter.write(deleteRow); + } else { + // TODO Get the correct path and pos from insert data MAP rather than SET. + this.positionDeleteWriter.write(positionDelete.set(dataWriter.currentPath(), dataWriter.currentPos(), deleteRow)); + } + } + + @Override + public void delete(CharSequence path, long pos, T row) throws IOException { + if (!allowPositionDelete()) { + throw new UnsupportedOperationException("Couldn't accept positional deletions."); + } + + this.positionDeleteWriter.write(positionDelete.set(path, pos, row)); + } + + @Override + public void abort() throws IOException { + if (dataWriter != null) { + dataWriter.abort(); + } + + if (equalityDeleteWriter != null) { + equalityDeleteWriter.abort(); + } + + if (positionDeleteWriter != null) { + positionDeleteWriter.abort(); + } + } + + @Override + public WriterResult complete() throws IOException { + WriterResult.Builder builder = WriterResult.builder(); + + if (dataWriter != null) { + builder.addAll(dataWriter.complete().contentFiles()); + } + + if (equalityDeleteWriter != null) { + builder.addAll(equalityDeleteWriter.complete().contentFiles()); + } + + if (positionDeleteWriter != null) { + builder.addAll(positionDeleteWriter.complete().contentFiles()); + } + + return builder.build(); + } + + private boolean allowEqualityDelete() { + return equalityDeleteWriter != null && positionDeleteWriter != null; + } + + private boolean allowPositionDelete() { + return positionDeleteWriter != null; + } +} diff --git a/core/src/main/java/org/apache/iceberg/io/TaskWriter.java b/core/src/main/java/org/apache/iceberg/io/FileGroupWriter.java similarity index 81% rename from core/src/main/java/org/apache/iceberg/io/TaskWriter.java rename to core/src/main/java/org/apache/iceberg/io/FileGroupWriter.java index 36a699fe8803..08f826a4ccdf 100644 --- a/core/src/main/java/org/apache/iceberg/io/TaskWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/FileGroupWriter.java @@ -27,7 +27,7 @@ * * @param to indicate the record data type. */ -public interface TaskWriter extends Closeable { +public interface FileGroupWriter extends Closeable { /** * Write the row into the data files. @@ -41,10 +41,20 @@ public interface TaskWriter extends Closeable { */ void abort() throws IOException; + /** + * Get the path of current opening file. + */ + CharSequence currentPath() throws IOException; + + /** + * Get the row number (starts from 0) of current opening file. + */ + long currentPos() throws IOException; + /** * Close the writer and get the completed data/delete files. * * @return the completed data files of this task writer. */ - TaskWriterResult complete() throws IOException; + WriterResult complete() throws IOException; } diff --git a/core/src/main/java/org/apache/iceberg/io/PartitionWriter.java b/core/src/main/java/org/apache/iceberg/io/PartitionWriter.java new file mode 100644 index 000000000000..b40e0de9510b --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/io/PartitionWriter.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.io; + +import java.io.IOException; + +public interface PartitionWriter { + + void append(T row) throws IOException; + + void delete(T deleteRow) throws IOException; + + void delete(CharSequence path, long pos, T row) throws IOException; + + void abort() throws IOException; + + WriterResult complete() throws IOException; +} 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 ef786121c4ae..3e666ef2b5c1 100644 --- a/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java @@ -29,7 +29,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public abstract class PartitionedWriter extends BaseTaskWriter { +public abstract class PartitionedWriter extends BaseFileGroupWriter { private static final Logger LOG = LoggerFactory.getLogger(PartitionedWriter.class); private final Set completedPartitions = Sets.newHashSet(); 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 994fe259ef18..dce29765990b 100644 --- a/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java @@ -23,7 +23,7 @@ import org.apache.iceberg.ContentFileWriterFactory; import org.apache.iceberg.FileFormat; -public class UnpartitionedWriter extends BaseTaskWriter { +public class UnpartitionedWriter extends BaseFileGroupWriter { private final RollingFileWriter currentWriter; @@ -38,6 +38,16 @@ public void write(T record) throws IOException { currentWriter.add(record); } + @Override + public CharSequence currentPath() throws IOException { + return currentWriter.currentPath(); + } + + @Override + public long currentPos() throws IOException { + return currentWriter.currentPos(); + } + @Override public void close() throws IOException { currentWriter.close(); diff --git a/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java b/core/src/main/java/org/apache/iceberg/io/WriterResult.java similarity index 85% rename from core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java rename to core/src/main/java/org/apache/iceberg/io/WriterResult.java index 3afa23435428..7aa3b833b38d 100644 --- a/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java +++ b/core/src/main/java/org/apache/iceberg/io/WriterResult.java @@ -28,11 +28,11 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -public class TaskWriterResult { +public class WriterResult { private DataFile[] dataFiles; private DeleteFile[] deleteFiles; - private TaskWriterResult(List dataFiles, List deleteFiles) { + private WriterResult(List dataFiles, List deleteFiles) { this.dataFiles = dataFiles.toArray(new DataFile[0]); this.deleteFiles = deleteFiles.toArray(new DeleteFile[0]); } @@ -74,7 +74,7 @@ public static Builder builder() { return new Builder(); } - public static TaskWriterResult concat(TaskWriterResult result0, TaskWriterResult result1) { + public static WriterResult concat(WriterResult result0, WriterResult result1) { Builder builder = new Builder(); builder.addAll(result0.dataFiles); @@ -94,13 +94,19 @@ private Builder() { this.deleteFiles = Lists.newArrayList(); } - public void addAll(ContentFile... files) { - for (ContentFile file : files) { + public void addAll(Iterable> iterable) { + for (ContentFile file : iterable) { add(file); } } - public void add(ContentFile contentFile) { + public void addAll(ContentFile... files) { + for (ContentFile file : files) { + add(file); + } + } + + public void add(ContentFile contentFile) { Preconditions.checkNotNull(contentFile, "Content file shouldn't be null."); switch (contentFile.content()) { case DATA: @@ -117,8 +123,8 @@ public void add(ContentFile contentFile) { } } - public TaskWriterResult build() { - return new TaskWriterResult(dataFiles, deleteFiles); + public WriterResult build() { + return new WriterResult(dataFiles, deleteFiles); } } } 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 1f8bb66c0cdd..d6184ee05317 100644 --- a/core/src/test/java/org/apache/iceberg/avro/TestAvroDeleteWriters.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestAvroDeleteWriters.java @@ -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/test/java/org/apache/iceberg/data/FileHelpers.java b/data/src/test/java/org/apache/iceberg/data/FileHelpers.java index 0755ffb8ca18..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) diff --git a/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkPositionDeleteWriterFactory.java b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkPositionDeleteWriterFactory.java new file mode 100644 index 000000000000..23ea8badc3e4 --- /dev/null +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkPositionDeleteWriterFactory.java @@ -0,0 +1,87 @@ +/* + * 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.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.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.parquet.Parquet; + +public class FlinkPositionDeleteWriterFactory + implements ContentFileWriterFactory>, Serializable { + private final Schema schema; + private final PartitionSpec spec; + private final Map props; + + public FlinkPositionDeleteWriterFactory(Schema schema, + PartitionSpec spec, + Map props) { + this.schema = schema; + this.spec = spec; + this.props = props; + } + + + @Override + public ContentFileWriter> createWriter(PartitionKey partitionKey, + EncryptedOutputFile outputFile, + FileFormat fileFormat) { + try { + switch (fileFormat) { + case AVRO: + return Avro.writeDeletes(outputFile.encryptingOutputFile()) + .rowSchema(schema) + .withSpec(spec) + .setAll(props) + .withPartition(partitionKey) + .overwrite() + .buildPositionWriter(); + + + case PARQUET: + return Parquet.writeDeletes(outputFile.encryptingOutputFile()) + .rowSchema(schema) + .withSpec(spec) + .setAll(props) + .withPartition(partitionKey) + .overwrite() + .buildPositionWriter(); + + case ORC: + default: + throw new UnsupportedOperationException(); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } +} 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 8e9861171534..bd87a51dc687 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 @@ -26,7 +26,7 @@ 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.FileGroupWriter; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; class IcebergStreamWriter extends AbstractStreamOperator @@ -37,7 +37,7 @@ class IcebergStreamWriter extends AbstractStreamOperator private final String fullTableName; private final TaskWriterFactory taskWriterFactory; - private transient TaskWriter writer; + private transient FileGroupWriter writer; private transient int subTaskId; private transient int attemptId; 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 index 74c3bd570a91..2059352eb3c9 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedFanoutWriter.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedFanoutWriter.java @@ -24,12 +24,12 @@ import org.apache.iceberg.ContentFileWriterFactory; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.io.BaseTaskWriter; +import org.apache.iceberg.io.BaseFileGroupWriter; 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 { +abstract class PartitionedFanoutWriter extends BaseFileGroupWriter { private final Map writers = Maps.newHashMap(); PartitionedFanoutWriter(FileFormat format, OutputFileFactory fileFactory, FileIO io, long targetFileSize, 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 aee79a96695e..430e88cf5131 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 @@ -24,6 +24,7 @@ import java.io.UncheckedIOException; import java.util.List; import java.util.Map; +import java.util.Set; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.ContentFileWriterFactory; @@ -34,6 +35,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.deletes.PositionDelete; import org.apache.iceberg.encryption.EncryptionManager; import org.apache.iceberg.flink.RowDataWrapper; import org.apache.iceberg.flink.data.FlinkAvroWriter; @@ -45,13 +47,14 @@ 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.TaskWriterResult; +import org.apache.iceberg.io.FileGroupWriter; +import org.apache.iceberg.io.WriterResult; 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; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; public class RowDataTaskWriterFactory implements TaskWriterFactory { private final Schema schema; @@ -66,6 +69,7 @@ public class RowDataTaskWriterFactory implements TaskWriterFactory { private final FileAppenderFactory appenderFactory; private transient OutputFileFactory outputFileFactory; + private transient Set eqDeleteCache; public RowDataTaskWriterFactory(Schema schema, RowType flinkSchema, @@ -91,29 +95,30 @@ public RowDataTaskWriterFactory(Schema schema, @Override public void initialize(int taskId, int attemptId) { this.outputFileFactory = new OutputFileFactory(spec, format, locations, io, encryptionManager, taskId, attemptId); + this.eqDeleteCache = Sets.newHashSet(); } @Override - public TaskWriter create() { + public FileGroupWriter create() { Preconditions.checkNotNull(outputFileFactory, "The outputFileFactory shouldn't be null if we have invoked the initialize()."); if (spec.fields().isEmpty()) { - return new MixedUnpartitionedTaskWriter(format, appenderFactory, outputFileFactory, io, targetFileSizeBytes, + return new MixedUnpartitionedFileGroupWriter(format, appenderFactory, outputFileFactory, io, targetFileSizeBytes, schema, flinkSchema, tableProperties); } else { - return new MixedPartitionedTaskWriter(spec, format, appenderFactory, outputFileFactory, + return new MixedPartitionedFileGroupWriter(spec, format, appenderFactory, outputFileFactory, io, targetFileSizeBytes, schema, flinkSchema, tableProperties); } } - private abstract static class BaseMixedTaskWriter implements TaskWriter { + private abstract static class BaseMixedFileGroupWriter implements FileGroupWriter { abstract boolean supportDeletion(); - abstract TaskWriter dataTaskWriter(); + abstract FileGroupWriter dataTaskWriter(); - abstract TaskWriter deleteTaskWriter(); + abstract FileGroupWriter deleteTaskWriter(); @Override public void write(RowData row) throws IOException { @@ -147,9 +152,9 @@ public void abort() throws IOException { } @Override - public TaskWriterResult complete() throws IOException { + public WriterResult complete() throws IOException { if (supportDeletion()) { - return TaskWriterResult.concat(dataTaskWriter().complete(), deleteTaskWriter().complete()); + return WriterResult.concat(dataTaskWriter().complete(), deleteTaskWriter().complete()); } else { return dataTaskWriter().complete(); } @@ -164,28 +169,28 @@ public void close() throws IOException { } } - private static class MixedUnpartitionedTaskWriter extends BaseMixedTaskWriter { + private static class MixedUnpartitionedFileGroupWriter extends BaseMixedFileGroupWriter { private final FileFormat format; - private final TaskWriter dataTaskWriter; - private final TaskWriter deleteTaskWriter; + private final FileGroupWriter dataFileGroupWriter; + private final FileGroupWriter deleteFileGroupWriter; - MixedUnpartitionedTaskWriter(FileFormat format, FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, - RowType flinkSchema, Map tableProperties) { + MixedUnpartitionedFileGroupWriter(FileFormat format, FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, + RowType flinkSchema, Map tableProperties) { this.format = format; - this.dataTaskWriter = new UnpartitionedWriter<>(format, fileFactory, io, + this.dataFileGroupWriter = new UnpartitionedWriter<>(format, fileFactory, io, targetFileSize, new DataFileWriterFactory<>(appenderFactory, PartitionSpec.unpartitioned())); // TODO: set the correct equality field ids. List equalityIds = ImmutableList.of(); if (supportDeletion()) { - this.deleteTaskWriter = new UnpartitionedWriter<>(format, fileFactory, io, targetFileSize, + this.deleteFileGroupWriter = new UnpartitionedWriter<>(format, fileFactory, io, targetFileSize, new FlinkEqualityDeleterFactory(schema, flinkSchema, PartitionSpec.unpartitioned(), equalityIds, tableProperties)); } else { - this.deleteTaskWriter = null; + this.deleteFileGroupWriter = null; } } @@ -195,27 +200,28 @@ boolean supportDeletion() { } @Override - TaskWriter dataTaskWriter() { - return dataTaskWriter; + FileGroupWriter dataTaskWriter() { + return dataFileGroupWriter; } @Override - TaskWriter deleteTaskWriter() { - return deleteTaskWriter; + FileGroupWriter deleteTaskWriter() { + return deleteFileGroupWriter; } } - private static class MixedPartitionedTaskWriter extends BaseMixedTaskWriter { + private static class MixedPartitionedFileGroupWriter extends BaseMixedFileGroupWriter { private final FileFormat format; - private final TaskWriter dataTaskWriter; - private final TaskWriter deleteTaskWriter; + private final FileGroupWriter dataFileGroupWriter; + private final FileGroupWriter deleteFileGroupWriter; + private final FileGroupWriter> posDelFileGroupWriter; - MixedPartitionedTaskWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, - RowType flinkSchema, Map tableProperties) { + MixedPartitionedFileGroupWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, + RowType flinkSchema, Map tableProperties) { this.format = format; - this.dataTaskWriter = + this.dataFileGroupWriter = new RowDataPartitionedFanoutWriter<>(spec, format, fileFactory, io, targetFileSize, schema, flinkSchema, new DataFileWriterFactory<>(appenderFactory, spec)); @@ -223,11 +229,15 @@ private static class MixedPartitionedTaskWriter extends BaseMixedTaskWriter { List equalityIds = ImmutableList.of(); if (supportDeletion()) { - this.deleteTaskWriter = + this.deleteFileGroupWriter = new RowDataPartitionedFanoutWriter<>(spec, format, fileFactory, io, targetFileSize, schema, flinkSchema, new FlinkEqualityDeleterFactory(schema, flinkSchema, spec, equalityIds, tableProperties)); + this.posDelFileGroupWriter = + new PosPartitionedFanoutWriter<>(spec, format, fileFactory, io, targetFileSize, schema, + flinkSchema, new FlinkPositionDeleteWriterFactory(schema, spec, tableProperties)); } else { - this.deleteTaskWriter = null; + this.deleteFileGroupWriter = null; + this.posDelFileGroupWriter = null; } } @@ -237,13 +247,36 @@ boolean supportDeletion() { } @Override - TaskWriter dataTaskWriter() { - return dataTaskWriter; + public void write(RowData row) throws IOException { + switch (row.getRowKind()) { + + case INSERT: + case UPDATE_AFTER: + dataTaskWriter().write(row); + break; + + case UPDATE_BEFORE: + case DELETE: + if (supportDeletion()) { + deleteTaskWriter().write(row); + } else { + throw new UnsupportedOperationException("Couldn't accept deletion in task writer."); + } + break; + + default: + throw new UnsupportedOperationException("Unrecognized row kind: " + row.getRowKind()); + } } @Override - TaskWriter deleteTaskWriter() { - return deleteTaskWriter; + FileGroupWriter dataTaskWriter() { + return dataFileGroupWriter; + } + + @Override + FileGroupWriter deleteTaskWriter() { + return deleteFileGroupWriter; } } @@ -268,6 +301,27 @@ protected PartitionKey partition(RowData row) { } } + private static class PosPartitionedFanoutWriter + extends PartitionedFanoutWriter> { + + private final PartitionKey partitionKey; + private final RowDataWrapper rowDataWrapper; + + PosPartitionedFanoutWriter(PartitionSpec spec, FileFormat format, OutputFileFactory fileFactory, FileIO io, + long targetFileSize, Schema schema, RowType flinkSchema, + ContentFileWriterFactory> contentFileWriter) { + super(format, fileFactory, io, targetFileSize, contentFileWriter); + this.partitionKey = new PartitionKey(spec, schema); + this.rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); + } + + @Override + protected PartitionKey partition(PositionDelete row) { + partitionKey.partition(rowDataWrapper.wrap(row.row())); + return partitionKey; + } + } + public static class FlinkFileAppenderFactory implements FileAppenderFactory, Serializable { private final Schema schema; private final RowType flinkSchema; diff --git a/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java b/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java index 9d56ec6a812a..f66f2e18df7d 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java @@ -20,10 +20,10 @@ package org.apache.iceberg.flink.sink; import java.io.Serializable; -import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.FileGroupWriter; /** - * Factory to create {@link TaskWriter} + * Factory to create {@link FileGroupWriter} * * @param data type of record. */ @@ -38,9 +38,9 @@ public interface TaskWriterFactory extends Serializable { void initialize(int taskId, int attemptId); /** - * Initialize a {@link TaskWriter} with given task id and attempt id. + * Initialize a {@link FileGroupWriter} with given task id and attempt id. * * @return a newly created task writer. */ - TaskWriter create(); + FileGroupWriter create(); } 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 5505f9a1b7c8..c17da5825c74 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 @@ -37,7 +37,7 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.data.RandomRowData; -import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.FileGroupWriter; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.Assert; @@ -91,16 +91,16 @@ public void before() throws IOException { @Test public void testWriteZeroRecord() throws IOException { - try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { - taskWriter.close(); + try (FileGroupWriter fileGroupWriter = createTaskWriter(TARGET_FILE_SIZE)) { + fileGroupWriter.close(); - DataFile[] dataFiles = taskWriter.complete().dataFiles(); + DataFile[] dataFiles = fileGroupWriter.complete().dataFiles(); Assert.assertNotNull(dataFiles); Assert.assertEquals(0, dataFiles.length); // Close again. - taskWriter.close(); - dataFiles = taskWriter.complete().dataFiles(); + fileGroupWriter.close(); + dataFiles = fileGroupWriter.complete().dataFiles(); Assert.assertNotNull(dataFiles); Assert.assertEquals(0, dataFiles.length); } @@ -108,14 +108,14 @@ public void testWriteZeroRecord() throws IOException { @Test public void testCloseTwice() throws IOException { - try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { - taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); - taskWriter.write(SimpleDataUtil.createRowData(2, "world")); - taskWriter.close(); // The first close - taskWriter.close(); // The second close + try (FileGroupWriter fileGroupWriter = createTaskWriter(TARGET_FILE_SIZE)) { + fileGroupWriter.write(SimpleDataUtil.createRowData(1, "hello")); + fileGroupWriter.write(SimpleDataUtil.createRowData(2, "world")); + fileGroupWriter.close(); // The first close + fileGroupWriter.close(); // The second close int expectedFiles = partitioned ? 2 : 1; - DataFile[] dataFiles = taskWriter.complete().dataFiles(); + DataFile[] dataFiles = fileGroupWriter.complete().dataFiles(); Assert.assertEquals(expectedFiles, dataFiles.length); FileSystem fs = FileSystem.get(CONF); @@ -127,12 +127,12 @@ public void testCloseTwice() throws IOException { @Test public void testAbort() throws IOException { - try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { - taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); - taskWriter.write(SimpleDataUtil.createRowData(2, "world")); + try (FileGroupWriter fileGroupWriter = createTaskWriter(TARGET_FILE_SIZE)) { + fileGroupWriter.write(SimpleDataUtil.createRowData(1, "hello")); + fileGroupWriter.write(SimpleDataUtil.createRowData(2, "world")); - taskWriter.abort(); - DataFile[] dataFiles = taskWriter.complete().dataFiles(); + fileGroupWriter.abort(); + DataFile[] dataFiles = fileGroupWriter.complete().dataFiles(); int expectedFiles = partitioned ? 2 : 1; Assert.assertEquals(expectedFiles, dataFiles.length); @@ -146,17 +146,17 @@ public void testAbort() throws IOException { @Test public void testCompleteFiles() throws IOException { - try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { - taskWriter.write(SimpleDataUtil.createRowData(1, "a")); - taskWriter.write(SimpleDataUtil.createRowData(2, "b")); - taskWriter.write(SimpleDataUtil.createRowData(3, "c")); - taskWriter.write(SimpleDataUtil.createRowData(4, "d")); + try (FileGroupWriter fileGroupWriter = createTaskWriter(TARGET_FILE_SIZE)) { + fileGroupWriter.write(SimpleDataUtil.createRowData(1, "a")); + fileGroupWriter.write(SimpleDataUtil.createRowData(2, "b")); + fileGroupWriter.write(SimpleDataUtil.createRowData(3, "c")); + fileGroupWriter.write(SimpleDataUtil.createRowData(4, "d")); - DataFile[] dataFiles = taskWriter.complete().dataFiles(); + DataFile[] dataFiles = fileGroupWriter.complete().dataFiles(); int expectedFiles = partitioned ? 4 : 1; Assert.assertEquals(expectedFiles, dataFiles.length); - dataFiles = taskWriter.complete().dataFiles(); + dataFiles = fileGroupWriter.complete().dataFiles(); Assert.assertEquals(expectedFiles, dataFiles.length); FileSystem fs = FileSystem.get(CONF); @@ -186,7 +186,7 @@ public void testRollingWithTargetFileSize() throws IOException { if (format == FileFormat.ORC) { return; } - try (TaskWriter taskWriter = createTaskWriter(4)) { + try (FileGroupWriter fileGroupWriter = createTaskWriter(4)) { List rows = Lists.newArrayListWithCapacity(8000); List records = Lists.newArrayListWithCapacity(8000); for (int i = 0; i < 2000; i++) { @@ -197,10 +197,10 @@ public void testRollingWithTargetFileSize() throws IOException { } for (RowData row : rows) { - taskWriter.write(row); + fileGroupWriter.write(row); } - DataFile[] dataFiles = taskWriter.complete().dataFiles(); + DataFile[] dataFiles = fileGroupWriter.complete().dataFiles(); Assert.assertEquals(8, dataFiles.length); AppendFiles appendFiles = table.newAppend(); @@ -216,14 +216,14 @@ public void testRollingWithTargetFileSize() throws IOException { @Test public void testRandomData() throws IOException { - try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { + try (FileGroupWriter fileGroupWriter = createTaskWriter(TARGET_FILE_SIZE)) { Iterable rows = RandomRowData.generate(SimpleDataUtil.SCHEMA, 100, 1996); for (RowData row : rows) { - taskWriter.write(row); + fileGroupWriter.write(row); } - taskWriter.close(); - DataFile[] dataFiles = taskWriter.complete().dataFiles(); + fileGroupWriter.close(); + DataFile[] dataFiles = fileGroupWriter.complete().dataFiles(); AppendFiles appendFiles = table.newAppend(); for (DataFile dataFile : dataFiles) { appendFiles.appendFile(dataFile); @@ -235,7 +235,7 @@ public void testRandomData() throws IOException { } } - private TaskWriter createTaskWriter(long targetFileSize) { + private FileGroupWriter createTaskWriter(long targetFileSize) { TaskWriterFactory taskWriterFactory = new RowDataTaskWriterFactory(table.schema(), (RowType) SimpleDataUtil.FLINK_SCHEMA.toRowDataType().getLogicalType(), table.spec(), table.locationProvider(), table.io(), table.encryption(), 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 402d09823a8f..b7d56ccb8976 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDeleteWriters.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDeleteWriters.java @@ -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 8d3e46f9622c..d8050ceb356c 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 @@ -37,7 +37,7 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.io.OutputFileFactory; -import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.FileGroupWriter; import org.apache.iceberg.io.UnpartitionedWriter; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.SparkSchemaUtil; @@ -103,7 +103,7 @@ private List rewriteDataForTask(CombinedScanTask task) throws Exceptio OutputFileFactory fileFactory = new OutputFileFactory( spec, format, locations, io.value(), encryptionManager.value(), partitionId, taskId); - TaskWriter writer; + FileGroupWriter writer; if (spec.fields().isEmpty()) { writer = new UnpartitionedWriter<>(format, fileFactory, io.value(), Long.MAX_VALUE, new DataFileWriterFactory<>(appenderFactory, spec)); From 47972c8ad27221c9dd5b186e19180ee50573ee30 Mon Sep 17 00:00:00 2001 From: huzheng Date: Thu, 5 Nov 2020 10:59:15 +0800 Subject: [PATCH 08/41] Introduce the BasePartitionWriter and StructLikeMap --- .../iceberg/io/BasePartitionWriter.java | 55 +++++--- .../apache/iceberg/io/PartitionWriter.java | 4 + .../apache/iceberg/util/StructLikeMap.java | 127 ++++++++++++++++++ .../flink/sink/RowDataTaskWriterFactory.java | 4 - 4 files changed, 170 insertions(+), 20 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/util/StructLikeMap.java diff --git a/core/src/main/java/org/apache/iceberg/io/BasePartitionWriter.java b/core/src/main/java/org/apache/iceberg/io/BasePartitionWriter.java index bbe865ab0096..7340abb8c8de 100644 --- a/core/src/main/java/org/apache/iceberg/io/BasePartitionWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BasePartitionWriter.java @@ -26,7 +26,7 @@ import org.apache.iceberg.deletes.PositionDelete; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.util.StructLikeSet; +import org.apache.iceberg.util.StructLikeMap; import org.apache.iceberg.util.StructProjection; public abstract class BasePartitionWriter implements PartitionWriter { @@ -35,10 +35,30 @@ public abstract class BasePartitionWriter implements PartitionWriter { private final FileGroupWriter equalityDeleteWriter; private final FileGroupWriter> positionDeleteWriter; private final StructProjection projectRow; - private final Schema deleteSchema; - private final List equalityFieldIds; private final PositionDelete positionDelete = new PositionDelete<>(); - private final StructLikeSet insertedDataSet; + private final StructLikeMap insertedRowMap; + + private static class FilePos { + private final CharSequence path; + private final long pos; + + private FilePos(CharSequence path, long pos) { + this.path = path; + this.pos = pos; + } + + private static FilePos create(CharSequence path, long pos) { + return new FilePos(path, pos); + } + + private CharSequence path() { + return path; + } + + private long pos() { + return pos; + } + } public BasePartitionWriter(FileGroupWriter dataWriter) { this(dataWriter, null); @@ -52,27 +72,27 @@ public BasePartitionWriter(FileGroupWriter dataWriter, public BasePartitionWriter(FileGroupWriter dataWriter, FileGroupWriter> positionDeleteWriter, FileGroupWriter equalityDeleteWriter, - Schema writeSchema, + Schema tableSchema, List equalityFieldIds) { this.dataWriter = dataWriter; this.positionDeleteWriter = positionDeleteWriter; this.equalityDeleteWriter = equalityDeleteWriter; - this.deleteSchema = TypeUtil.select(writeSchema, Sets.newHashSet(equalityFieldIds)); - this.equalityFieldIds = equalityFieldIds; - this.projectRow = StructProjection.create(writeSchema, deleteSchema); - this.insertedDataSet = StructLikeSet.create(deleteSchema.asStruct()); + + Schema deleteSchema = TypeUtil.select(tableSchema, Sets.newHashSet(equalityFieldIds)); + this.projectRow = StructProjection.create(tableSchema, deleteSchema); + this.insertedRowMap = StructLikeMap.create(deleteSchema.asStruct()); } protected abstract StructLike asStructLike(T record); @Override public void append(T row) throws IOException { - this.dataWriter.write(row); - if (allowEqualityDelete()) { - // TODO Put the > into the insert data MAP, rather than SET. - insertedDataSet.add(projectRow.wrap(asStructLike(row))); + FilePos filePos = FilePos.create(dataWriter.currentPath(), dataWriter.currentPos()); + insertedRowMap.put(projectRow.wrap(asStructLike(row)), filePos); } + + this.dataWriter.write(row); } @Override @@ -81,11 +101,14 @@ public void delete(T deleteRow) throws IOException { throw new UnsupportedOperationException("Couldn't accept equality deletions."); } - if (!insertedDataSet.contains(projectRow.wrap(asStructLike(deleteRow)))) { + FilePos existingFilePos = insertedRowMap.get(projectRow.wrap(asStructLike(deleteRow))); + + if (existingFilePos == null) { + // Delete the row which has been written before this writer. this.equalityDeleteWriter.write(deleteRow); } else { - // TODO Get the correct path and pos from insert data MAP rather than SET. - this.positionDeleteWriter.write(positionDelete.set(dataWriter.currentPath(), dataWriter.currentPos(), deleteRow)); + // Delete the row which was written in current writer. + this.positionDeleteWriter.write(positionDelete.set(existingFilePos.path(), existingFilePos.pos(), deleteRow)); } } diff --git a/core/src/main/java/org/apache/iceberg/io/PartitionWriter.java b/core/src/main/java/org/apache/iceberg/io/PartitionWriter.java index b40e0de9510b..7c01c6570dee 100644 --- a/core/src/main/java/org/apache/iceberg/io/PartitionWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/PartitionWriter.java @@ -27,6 +27,10 @@ public interface PartitionWriter { void delete(T deleteRow) throws IOException; + default void delete(CharSequence path, long pos) throws IOException { + delete(path, pos, null); + } + void delete(CharSequence path, long pos, T row) throws IOException; void abort() throws IOException; 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..f45b85a2a3c6 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/util/StructLikeMap.java @@ -0,0 +1,127 @@ +/* + * 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.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 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/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java b/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java index 430e88cf5131..bc7d0082cea9 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 @@ -24,7 +24,6 @@ import java.io.UncheckedIOException; import java.util.List; import java.util.Map; -import java.util.Set; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.ContentFileWriterFactory; @@ -54,7 +53,6 @@ import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; public class RowDataTaskWriterFactory implements TaskWriterFactory { private final Schema schema; @@ -69,7 +67,6 @@ public class RowDataTaskWriterFactory implements TaskWriterFactory { private final FileAppenderFactory appenderFactory; private transient OutputFileFactory outputFileFactory; - private transient Set eqDeleteCache; public RowDataTaskWriterFactory(Schema schema, RowType flinkSchema, @@ -95,7 +92,6 @@ public RowDataTaskWriterFactory(Schema schema, @Override public void initialize(int taskId, int attemptId) { this.outputFileFactory = new OutputFileFactory(spec, format, locations, io, encryptionManager, taskId, attemptId); - this.eqDeleteCache = Sets.newHashSet(); } @Override From 8b7bb62886be8c12e29781fa67de60166d0580ae Mon Sep 17 00:00:00 2001 From: huzheng Date: Thu, 5 Nov 2020 14:37:52 +0800 Subject: [PATCH 09/41] temp2 --- .../iceberg/io/BasePartitionWriter.java | 31 +++++--- ...riter.java => BaseRollingFilesWriter.java} | 7 +- .../iceberg/io/PartitionFanoutTaskWriter.java | 39 ++++++++++ .../io/PartitionGroupingTaskWriter.java | 39 ++++++++++ .../apache/iceberg/io/PartitionedWriter.java | 2 +- ...oupWriter.java => RollingFilesWriter.java} | 2 +- .../org/apache/iceberg/io/TaskWriter.java | 31 ++++++++ .../iceberg/io/UnpartitionedTaskWriter.java | 40 +++++++++++ .../iceberg/io/UnpartitionedWriter.java | 2 +- .../flink/sink/IcebergStreamWriter.java | 4 +- .../flink/sink/PartitionedFanoutWriter.java | 4 +- .../flink/sink/RowDataTaskWriterFactory.java | 72 +++++++++---------- .../iceberg/flink/sink/TaskWriterFactory.java | 8 +-- .../iceberg/flink/sink/TestTaskWriters.java | 64 ++++++++--------- .../iceberg/spark/source/RowDataRewriter.java | 4 +- 15 files changed, 256 insertions(+), 93 deletions(-) rename core/src/main/java/org/apache/iceberg/io/{BaseFileGroupWriter.java => BaseRollingFilesWriter.java} (92%) create mode 100644 core/src/main/java/org/apache/iceberg/io/PartitionFanoutTaskWriter.java create mode 100644 core/src/main/java/org/apache/iceberg/io/PartitionGroupingTaskWriter.java rename core/src/main/java/org/apache/iceberg/io/{FileGroupWriter.java => RollingFilesWriter.java} (96%) create mode 100644 core/src/main/java/org/apache/iceberg/io/TaskWriter.java create mode 100644 core/src/main/java/org/apache/iceberg/io/UnpartitionedTaskWriter.java diff --git a/core/src/main/java/org/apache/iceberg/io/BasePartitionWriter.java b/core/src/main/java/org/apache/iceberg/io/BasePartitionWriter.java index 7340abb8c8de..1141d70ec7b6 100644 --- a/core/src/main/java/org/apache/iceberg/io/BasePartitionWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BasePartitionWriter.java @@ -24,6 +24,7 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.StructLike; 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.Sets; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.StructLikeMap; @@ -31,9 +32,9 @@ public abstract class BasePartitionWriter implements PartitionWriter { - private final FileGroupWriter dataWriter; - private final FileGroupWriter equalityDeleteWriter; - private final FileGroupWriter> positionDeleteWriter; + private final RollingFilesWriter dataWriter; + private final RollingFilesWriter equalityDeleteWriter; + private final RollingFilesWriter> positionDeleteWriter; private final StructProjection projectRow; private final PositionDelete positionDelete = new PositionDelete<>(); private final StructLikeMap insertedRowMap; @@ -60,20 +61,32 @@ private long pos() { } } - public BasePartitionWriter(FileGroupWriter dataWriter) { + public BasePartitionWriter(RollingFilesWriter dataWriter) { this(dataWriter, null); } - public BasePartitionWriter(FileGroupWriter dataWriter, - FileGroupWriter> positionDeleteWriter) { + public BasePartitionWriter(RollingFilesWriter dataWriter, + RollingFilesWriter> positionDeleteWriter) { this(dataWriter, positionDeleteWriter, null, null, null); } - public BasePartitionWriter(FileGroupWriter dataWriter, - FileGroupWriter> positionDeleteWriter, - FileGroupWriter equalityDeleteWriter, + public BasePartitionWriter(RollingFilesWriter dataWriter, + RollingFilesWriter> positionDeleteWriter, + RollingFilesWriter equalityDeleteWriter, Schema tableSchema, List equalityFieldIds) { + Preconditions.checkNotNull(dataWriter, "Data writer should always not be null."); + if (positionDeleteWriter == null) { + // Only accept INSERT records. + Preconditions.checkArgument(equalityDeleteWriter == null); + } + + if (equalityDeleteWriter != null) { + // Only accept equality delete records. + Preconditions.checkNotNull(positionDeleteWriter, + "Position delete writer shouldn't be null when writing equality deletions."); + } + this.dataWriter = dataWriter; this.positionDeleteWriter = positionDeleteWriter; this.equalityDeleteWriter = equalityDeleteWriter; diff --git a/core/src/main/java/org/apache/iceberg/io/BaseFileGroupWriter.java b/core/src/main/java/org/apache/iceberg/io/BaseRollingFilesWriter.java similarity index 92% rename from core/src/main/java/org/apache/iceberg/io/BaseFileGroupWriter.java rename to core/src/main/java/org/apache/iceberg/io/BaseRollingFilesWriter.java index a2b68dac32de..1cb7f47c1485 100644 --- a/core/src/main/java/org/apache/iceberg/io/BaseFileGroupWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BaseRollingFilesWriter.java @@ -30,7 +30,7 @@ import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.util.Tasks; -public abstract class BaseFileGroupWriter implements FileGroupWriter { +public abstract class BaseRollingFilesWriter implements RollingFilesWriter { private final WriterResult.Builder builder; private final FileFormat format; private final OutputFileFactory fileFactory; @@ -38,8 +38,9 @@ public abstract class BaseFileGroupWriter implements FileGroupW private final long targetFileSize; private final ContentFileWriterFactory writerFactory; - protected BaseFileGroupWriter(FileFormat format, OutputFileFactory fileFactory, FileIO io, long targetFileSize, - ContentFileWriterFactory writerFactory) { + protected BaseRollingFilesWriter(FileFormat format, OutputFileFactory fileFactory, + FileIO io, long targetFileSize, + ContentFileWriterFactory writerFactory) { this.builder = WriterResult.builder(); this.format = format; this.fileFactory = fileFactory; diff --git a/core/src/main/java/org/apache/iceberg/io/PartitionFanoutTaskWriter.java b/core/src/main/java/org/apache/iceberg/io/PartitionFanoutTaskWriter.java new file mode 100644 index 000000000000..de9ba3ad741b --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/io/PartitionFanoutTaskWriter.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.io; + +import java.io.IOException; + +public class PartitionFanoutTaskWriter implements TaskWriter { + @Override + public PartitionWriter route(T row) { + return null; + } + + @Override + public void abort() throws IOException { + + } + + @Override + public WriterResult complete() throws IOException { + return null; + } +} diff --git a/core/src/main/java/org/apache/iceberg/io/PartitionGroupingTaskWriter.java b/core/src/main/java/org/apache/iceberg/io/PartitionGroupingTaskWriter.java new file mode 100644 index 000000000000..3fa2cdaf6d05 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/io/PartitionGroupingTaskWriter.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.io; + +import java.io.IOException; + +public class PartitionGroupingTaskWriter implements TaskWriter { + @Override + public PartitionWriter route(T row) { + return null; + } + + @Override + public void abort() throws IOException { + + } + + @Override + public WriterResult complete() throws IOException { + return null; + } +} 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 3e666ef2b5c1..96a5d340afa2 100644 --- a/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java @@ -29,7 +29,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public abstract class PartitionedWriter extends BaseFileGroupWriter { +public abstract class PartitionedWriter extends BaseRollingFilesWriter { private static final Logger LOG = LoggerFactory.getLogger(PartitionedWriter.class); private final Set completedPartitions = Sets.newHashSet(); diff --git a/core/src/main/java/org/apache/iceberg/io/FileGroupWriter.java b/core/src/main/java/org/apache/iceberg/io/RollingFilesWriter.java similarity index 96% rename from core/src/main/java/org/apache/iceberg/io/FileGroupWriter.java rename to core/src/main/java/org/apache/iceberg/io/RollingFilesWriter.java index 08f826a4ccdf..124cdf9c97ef 100644 --- a/core/src/main/java/org/apache/iceberg/io/FileGroupWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/RollingFilesWriter.java @@ -27,7 +27,7 @@ * * @param to indicate the record data type. */ -public interface FileGroupWriter extends Closeable { +public interface RollingFilesWriter extends Closeable { /** * Write the row into the data files. diff --git a/core/src/main/java/org/apache/iceberg/io/TaskWriter.java b/core/src/main/java/org/apache/iceberg/io/TaskWriter.java new file mode 100644 index 000000000000..b1e814cd19d1 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/io/TaskWriter.java @@ -0,0 +1,31 @@ +/* + * 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; + +public interface TaskWriter { + + PartitionWriter route(T row); + + void abort() throws IOException; + + WriterResult complete() throws IOException; +} diff --git a/core/src/main/java/org/apache/iceberg/io/UnpartitionedTaskWriter.java b/core/src/main/java/org/apache/iceberg/io/UnpartitionedTaskWriter.java new file mode 100644 index 000000000000..eb888f7956f0 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/io/UnpartitionedTaskWriter.java @@ -0,0 +1,40 @@ +/* + * 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; + +public class UnpartitionedTaskWriter implements TaskWriter { + + @Override + public PartitionWriter route(T row) { + return null; + } + + @Override + public void abort() throws IOException { + + } + + @Override + public WriterResult complete() throws IOException { + return null; + } +} 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 dce29765990b..521d03b618f5 100644 --- a/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java @@ -23,7 +23,7 @@ import org.apache.iceberg.ContentFileWriterFactory; import org.apache.iceberg.FileFormat; -public class UnpartitionedWriter extends BaseFileGroupWriter { +public class UnpartitionedWriter extends BaseRollingFilesWriter { private final RollingFileWriter currentWriter; 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 bd87a51dc687..b7b9bc501910 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 @@ -26,7 +26,7 @@ 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.FileGroupWriter; +import org.apache.iceberg.io.RollingFilesWriter; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; class IcebergStreamWriter extends AbstractStreamOperator @@ -37,7 +37,7 @@ class IcebergStreamWriter extends AbstractStreamOperator private final String fullTableName; private final TaskWriterFactory taskWriterFactory; - private transient FileGroupWriter writer; + private transient RollingFilesWriter writer; private transient int subTaskId; private transient int attemptId; 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 index 2059352eb3c9..ad792eb3ab51 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedFanoutWriter.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedFanoutWriter.java @@ -24,12 +24,12 @@ import org.apache.iceberg.ContentFileWriterFactory; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.io.BaseFileGroupWriter; +import org.apache.iceberg.io.BaseRollingFilesWriter; 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 BaseFileGroupWriter { +abstract class PartitionedFanoutWriter extends BaseRollingFilesWriter { private final Map writers = Maps.newHashMap(); PartitionedFanoutWriter(FileFormat format, OutputFileFactory fileFactory, FileIO io, long targetFileSize, 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 bc7d0082cea9..23521792c824 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 @@ -46,7 +46,7 @@ import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.io.OutputFileFactory; -import org.apache.iceberg.io.FileGroupWriter; +import org.apache.iceberg.io.RollingFilesWriter; import org.apache.iceberg.io.WriterResult; import org.apache.iceberg.io.UnpartitionedWriter; import org.apache.iceberg.orc.ORC; @@ -95,26 +95,26 @@ public void initialize(int taskId, int attemptId) { } @Override - public FileGroupWriter create() { + public RollingFilesWriter create() { Preconditions.checkNotNull(outputFileFactory, "The outputFileFactory shouldn't be null if we have invoked the initialize()."); if (spec.fields().isEmpty()) { - return new MixedUnpartitionedFileGroupWriter(format, appenderFactory, outputFileFactory, io, targetFileSizeBytes, + return new MixedUnpartitionedRollingFilesWriter(format, appenderFactory, outputFileFactory, io, targetFileSizeBytes, schema, flinkSchema, tableProperties); } else { - return new MixedPartitionedFileGroupWriter(spec, format, appenderFactory, outputFileFactory, + return new MixedPartitionedRollingFilesWriter(spec, format, appenderFactory, outputFileFactory, io, targetFileSizeBytes, schema, flinkSchema, tableProperties); } } - private abstract static class BaseMixedFileGroupWriter implements FileGroupWriter { + private abstract static class BaseMixedRollingFilesWriter implements RollingFilesWriter { abstract boolean supportDeletion(); - abstract FileGroupWriter dataTaskWriter(); + abstract RollingFilesWriter dataTaskWriter(); - abstract FileGroupWriter deleteTaskWriter(); + abstract RollingFilesWriter deleteTaskWriter(); @Override public void write(RowData row) throws IOException { @@ -165,28 +165,28 @@ public void close() throws IOException { } } - private static class MixedUnpartitionedFileGroupWriter extends BaseMixedFileGroupWriter { + private static class MixedUnpartitionedRollingFilesWriter extends BaseMixedRollingFilesWriter { private final FileFormat format; - private final FileGroupWriter dataFileGroupWriter; - private final FileGroupWriter deleteFileGroupWriter; + private final RollingFilesWriter dataRollingFilesWriter; + private final RollingFilesWriter deleteRollingFilesWriter; - MixedUnpartitionedFileGroupWriter(FileFormat format, FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, - RowType flinkSchema, Map tableProperties) { + MixedUnpartitionedRollingFilesWriter(FileFormat format, FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, + RowType flinkSchema, Map tableProperties) { this.format = format; - this.dataFileGroupWriter = new UnpartitionedWriter<>(format, fileFactory, io, + this.dataRollingFilesWriter = new UnpartitionedWriter<>(format, fileFactory, io, targetFileSize, new DataFileWriterFactory<>(appenderFactory, PartitionSpec.unpartitioned())); // TODO: set the correct equality field ids. List equalityIds = ImmutableList.of(); if (supportDeletion()) { - this.deleteFileGroupWriter = new UnpartitionedWriter<>(format, fileFactory, io, targetFileSize, + this.deleteRollingFilesWriter = new UnpartitionedWriter<>(format, fileFactory, io, targetFileSize, new FlinkEqualityDeleterFactory(schema, flinkSchema, PartitionSpec.unpartitioned(), equalityIds, tableProperties)); } else { - this.deleteFileGroupWriter = null; + this.deleteRollingFilesWriter = null; } } @@ -196,28 +196,28 @@ boolean supportDeletion() { } @Override - FileGroupWriter dataTaskWriter() { - return dataFileGroupWriter; + RollingFilesWriter dataTaskWriter() { + return dataRollingFilesWriter; } @Override - FileGroupWriter deleteTaskWriter() { - return deleteFileGroupWriter; + RollingFilesWriter deleteTaskWriter() { + return deleteRollingFilesWriter; } } - private static class MixedPartitionedFileGroupWriter extends BaseMixedFileGroupWriter { + private static class MixedPartitionedRollingFilesWriter extends BaseMixedRollingFilesWriter { private final FileFormat format; - private final FileGroupWriter dataFileGroupWriter; - private final FileGroupWriter deleteFileGroupWriter; - private final FileGroupWriter> posDelFileGroupWriter; + private final RollingFilesWriter dataRollingFilesWriter; + private final RollingFilesWriter deleteRollingFilesWriter; + private final RollingFilesWriter> posDelRollingFilesWriter; - MixedPartitionedFileGroupWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, - RowType flinkSchema, Map tableProperties) { + MixedPartitionedRollingFilesWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, + RowType flinkSchema, Map tableProperties) { this.format = format; - this.dataFileGroupWriter = + this.dataRollingFilesWriter = new RowDataPartitionedFanoutWriter<>(spec, format, fileFactory, io, targetFileSize, schema, flinkSchema, new DataFileWriterFactory<>(appenderFactory, spec)); @@ -225,15 +225,15 @@ private static class MixedPartitionedFileGroupWriter extends BaseMixedFileGroupW List equalityIds = ImmutableList.of(); if (supportDeletion()) { - this.deleteFileGroupWriter = + this.deleteRollingFilesWriter = new RowDataPartitionedFanoutWriter<>(spec, format, fileFactory, io, targetFileSize, schema, flinkSchema, new FlinkEqualityDeleterFactory(schema, flinkSchema, spec, equalityIds, tableProperties)); - this.posDelFileGroupWriter = + this.posDelRollingFilesWriter = new PosPartitionedFanoutWriter<>(spec, format, fileFactory, io, targetFileSize, schema, flinkSchema, new FlinkPositionDeleteWriterFactory(schema, spec, tableProperties)); } else { - this.deleteFileGroupWriter = null; - this.posDelFileGroupWriter = null; + this.deleteRollingFilesWriter = null; + this.posDelRollingFilesWriter = null; } } @@ -266,13 +266,13 @@ public void write(RowData row) throws IOException { } @Override - FileGroupWriter dataTaskWriter() { - return dataFileGroupWriter; + RollingFilesWriter dataTaskWriter() { + return dataRollingFilesWriter; } @Override - FileGroupWriter deleteTaskWriter() { - return deleteFileGroupWriter; + RollingFilesWriter deleteTaskWriter() { + return deleteRollingFilesWriter; } } diff --git a/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java b/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java index f66f2e18df7d..f1102075f76f 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java @@ -20,10 +20,10 @@ package org.apache.iceberg.flink.sink; import java.io.Serializable; -import org.apache.iceberg.io.FileGroupWriter; +import org.apache.iceberg.io.RollingFilesWriter; /** - * Factory to create {@link FileGroupWriter} + * Factory to create {@link RollingFilesWriter} * * @param data type of record. */ @@ -38,9 +38,9 @@ public interface TaskWriterFactory extends Serializable { void initialize(int taskId, int attemptId); /** - * Initialize a {@link FileGroupWriter} with given task id and attempt id. + * Initialize a {@link RollingFilesWriter} with given task id and attempt id. * * @return a newly created task writer. */ - FileGroupWriter create(); + RollingFilesWriter create(); } 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 c17da5825c74..6271776f9961 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 @@ -37,7 +37,7 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.data.RandomRowData; -import org.apache.iceberg.io.FileGroupWriter; +import org.apache.iceberg.io.RollingFilesWriter; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.Assert; @@ -91,16 +91,16 @@ public void before() throws IOException { @Test public void testWriteZeroRecord() throws IOException { - try (FileGroupWriter fileGroupWriter = createTaskWriter(TARGET_FILE_SIZE)) { - fileGroupWriter.close(); + try (RollingFilesWriter rollingFilesWriter = createTaskWriter(TARGET_FILE_SIZE)) { + rollingFilesWriter.close(); - DataFile[] dataFiles = fileGroupWriter.complete().dataFiles(); + DataFile[] dataFiles = rollingFilesWriter.complete().dataFiles(); Assert.assertNotNull(dataFiles); Assert.assertEquals(0, dataFiles.length); // Close again. - fileGroupWriter.close(); - dataFiles = fileGroupWriter.complete().dataFiles(); + rollingFilesWriter.close(); + dataFiles = rollingFilesWriter.complete().dataFiles(); Assert.assertNotNull(dataFiles); Assert.assertEquals(0, dataFiles.length); } @@ -108,14 +108,14 @@ public void testWriteZeroRecord() throws IOException { @Test public void testCloseTwice() throws IOException { - try (FileGroupWriter fileGroupWriter = createTaskWriter(TARGET_FILE_SIZE)) { - fileGroupWriter.write(SimpleDataUtil.createRowData(1, "hello")); - fileGroupWriter.write(SimpleDataUtil.createRowData(2, "world")); - fileGroupWriter.close(); // The first close - fileGroupWriter.close(); // The second close + try (RollingFilesWriter rollingFilesWriter = createTaskWriter(TARGET_FILE_SIZE)) { + rollingFilesWriter.write(SimpleDataUtil.createRowData(1, "hello")); + rollingFilesWriter.write(SimpleDataUtil.createRowData(2, "world")); + rollingFilesWriter.close(); // The first close + rollingFilesWriter.close(); // The second close int expectedFiles = partitioned ? 2 : 1; - DataFile[] dataFiles = fileGroupWriter.complete().dataFiles(); + DataFile[] dataFiles = rollingFilesWriter.complete().dataFiles(); Assert.assertEquals(expectedFiles, dataFiles.length); FileSystem fs = FileSystem.get(CONF); @@ -127,12 +127,12 @@ public void testCloseTwice() throws IOException { @Test public void testAbort() throws IOException { - try (FileGroupWriter fileGroupWriter = createTaskWriter(TARGET_FILE_SIZE)) { - fileGroupWriter.write(SimpleDataUtil.createRowData(1, "hello")); - fileGroupWriter.write(SimpleDataUtil.createRowData(2, "world")); + try (RollingFilesWriter rollingFilesWriter = createTaskWriter(TARGET_FILE_SIZE)) { + rollingFilesWriter.write(SimpleDataUtil.createRowData(1, "hello")); + rollingFilesWriter.write(SimpleDataUtil.createRowData(2, "world")); - fileGroupWriter.abort(); - DataFile[] dataFiles = fileGroupWriter.complete().dataFiles(); + rollingFilesWriter.abort(); + DataFile[] dataFiles = rollingFilesWriter.complete().dataFiles(); int expectedFiles = partitioned ? 2 : 1; Assert.assertEquals(expectedFiles, dataFiles.length); @@ -146,17 +146,17 @@ public void testAbort() throws IOException { @Test public void testCompleteFiles() throws IOException { - try (FileGroupWriter fileGroupWriter = createTaskWriter(TARGET_FILE_SIZE)) { - fileGroupWriter.write(SimpleDataUtil.createRowData(1, "a")); - fileGroupWriter.write(SimpleDataUtil.createRowData(2, "b")); - fileGroupWriter.write(SimpleDataUtil.createRowData(3, "c")); - fileGroupWriter.write(SimpleDataUtil.createRowData(4, "d")); + try (RollingFilesWriter rollingFilesWriter = createTaskWriter(TARGET_FILE_SIZE)) { + rollingFilesWriter.write(SimpleDataUtil.createRowData(1, "a")); + rollingFilesWriter.write(SimpleDataUtil.createRowData(2, "b")); + rollingFilesWriter.write(SimpleDataUtil.createRowData(3, "c")); + rollingFilesWriter.write(SimpleDataUtil.createRowData(4, "d")); - DataFile[] dataFiles = fileGroupWriter.complete().dataFiles(); + DataFile[] dataFiles = rollingFilesWriter.complete().dataFiles(); int expectedFiles = partitioned ? 4 : 1; Assert.assertEquals(expectedFiles, dataFiles.length); - dataFiles = fileGroupWriter.complete().dataFiles(); + dataFiles = rollingFilesWriter.complete().dataFiles(); Assert.assertEquals(expectedFiles, dataFiles.length); FileSystem fs = FileSystem.get(CONF); @@ -186,7 +186,7 @@ public void testRollingWithTargetFileSize() throws IOException { if (format == FileFormat.ORC) { return; } - try (FileGroupWriter fileGroupWriter = createTaskWriter(4)) { + try (RollingFilesWriter rollingFilesWriter = createTaskWriter(4)) { List rows = Lists.newArrayListWithCapacity(8000); List records = Lists.newArrayListWithCapacity(8000); for (int i = 0; i < 2000; i++) { @@ -197,10 +197,10 @@ public void testRollingWithTargetFileSize() throws IOException { } for (RowData row : rows) { - fileGroupWriter.write(row); + rollingFilesWriter.write(row); } - DataFile[] dataFiles = fileGroupWriter.complete().dataFiles(); + DataFile[] dataFiles = rollingFilesWriter.complete().dataFiles(); Assert.assertEquals(8, dataFiles.length); AppendFiles appendFiles = table.newAppend(); @@ -216,14 +216,14 @@ public void testRollingWithTargetFileSize() throws IOException { @Test public void testRandomData() throws IOException { - try (FileGroupWriter fileGroupWriter = createTaskWriter(TARGET_FILE_SIZE)) { + try (RollingFilesWriter rollingFilesWriter = createTaskWriter(TARGET_FILE_SIZE)) { Iterable rows = RandomRowData.generate(SimpleDataUtil.SCHEMA, 100, 1996); for (RowData row : rows) { - fileGroupWriter.write(row); + rollingFilesWriter.write(row); } - fileGroupWriter.close(); - DataFile[] dataFiles = fileGroupWriter.complete().dataFiles(); + rollingFilesWriter.close(); + DataFile[] dataFiles = rollingFilesWriter.complete().dataFiles(); AppendFiles appendFiles = table.newAppend(); for (DataFile dataFile : dataFiles) { appendFiles.appendFile(dataFile); @@ -235,7 +235,7 @@ public void testRandomData() throws IOException { } } - private FileGroupWriter createTaskWriter(long targetFileSize) { + private RollingFilesWriter createTaskWriter(long targetFileSize) { TaskWriterFactory taskWriterFactory = new RowDataTaskWriterFactory(table.schema(), (RowType) SimpleDataUtil.FLINK_SCHEMA.toRowDataType().getLogicalType(), table.spec(), table.locationProvider(), table.io(), table.encryption(), 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 d8050ceb356c..28db58acfb4b 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 @@ -37,7 +37,7 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.io.OutputFileFactory; -import org.apache.iceberg.io.FileGroupWriter; +import org.apache.iceberg.io.RollingFilesWriter; import org.apache.iceberg.io.UnpartitionedWriter; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.SparkSchemaUtil; @@ -103,7 +103,7 @@ private List rewriteDataForTask(CombinedScanTask task) throws Exceptio OutputFileFactory fileFactory = new OutputFileFactory( spec, format, locations, io.value(), encryptionManager.value(), partitionId, taskId); - FileGroupWriter writer; + RollingFilesWriter writer; if (spec.fields().isEmpty()) { writer = new UnpartitionedWriter<>(format, fileFactory, io.value(), Long.MAX_VALUE, new DataFileWriterFactory<>(appenderFactory, spec)); From f29e37db792d1f21620094b205f40377d7633f86 Mon Sep 17 00:00:00 2001 From: huzheng Date: Thu, 5 Nov 2020 14:55:12 +0800 Subject: [PATCH 10/41] Revert "temp2" This reverts commit e0e4b2d0d4acd8c5fe28d69b73fb567c33a3ec21. --- ...esWriter.java => BaseFileGroupWriter.java} | 7 +- .../iceberg/io/BasePartitionWriter.java | 31 +++----- ...gFilesWriter.java => FileGroupWriter.java} | 2 +- .../iceberg/io/PartitionFanoutTaskWriter.java | 39 ---------- .../io/PartitionGroupingTaskWriter.java | 39 ---------- .../apache/iceberg/io/PartitionedWriter.java | 2 +- .../org/apache/iceberg/io/TaskWriter.java | 31 -------- .../iceberg/io/UnpartitionedTaskWriter.java | 40 ----------- .../iceberg/io/UnpartitionedWriter.java | 2 +- .../flink/sink/IcebergStreamWriter.java | 4 +- .../flink/sink/PartitionedFanoutWriter.java | 4 +- .../flink/sink/RowDataTaskWriterFactory.java | 72 +++++++++---------- .../iceberg/flink/sink/TaskWriterFactory.java | 8 +-- .../iceberg/flink/sink/TestTaskWriters.java | 64 ++++++++--------- .../iceberg/spark/source/RowDataRewriter.java | 4 +- 15 files changed, 93 insertions(+), 256 deletions(-) rename core/src/main/java/org/apache/iceberg/io/{BaseRollingFilesWriter.java => BaseFileGroupWriter.java} (92%) rename core/src/main/java/org/apache/iceberg/io/{RollingFilesWriter.java => FileGroupWriter.java} (96%) delete mode 100644 core/src/main/java/org/apache/iceberg/io/PartitionFanoutTaskWriter.java delete mode 100644 core/src/main/java/org/apache/iceberg/io/PartitionGroupingTaskWriter.java delete mode 100644 core/src/main/java/org/apache/iceberg/io/TaskWriter.java delete mode 100644 core/src/main/java/org/apache/iceberg/io/UnpartitionedTaskWriter.java diff --git a/core/src/main/java/org/apache/iceberg/io/BaseRollingFilesWriter.java b/core/src/main/java/org/apache/iceberg/io/BaseFileGroupWriter.java similarity index 92% rename from core/src/main/java/org/apache/iceberg/io/BaseRollingFilesWriter.java rename to core/src/main/java/org/apache/iceberg/io/BaseFileGroupWriter.java index 1cb7f47c1485..a2b68dac32de 100644 --- a/core/src/main/java/org/apache/iceberg/io/BaseRollingFilesWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BaseFileGroupWriter.java @@ -30,7 +30,7 @@ import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.util.Tasks; -public abstract class BaseRollingFilesWriter implements RollingFilesWriter { +public abstract class BaseFileGroupWriter implements FileGroupWriter { private final WriterResult.Builder builder; private final FileFormat format; private final OutputFileFactory fileFactory; @@ -38,9 +38,8 @@ public abstract class BaseRollingFilesWriter implements Rolling private final long targetFileSize; private final ContentFileWriterFactory writerFactory; - protected BaseRollingFilesWriter(FileFormat format, OutputFileFactory fileFactory, - FileIO io, long targetFileSize, - ContentFileWriterFactory writerFactory) { + protected BaseFileGroupWriter(FileFormat format, OutputFileFactory fileFactory, FileIO io, long targetFileSize, + ContentFileWriterFactory writerFactory) { this.builder = WriterResult.builder(); this.format = format; this.fileFactory = fileFactory; diff --git a/core/src/main/java/org/apache/iceberg/io/BasePartitionWriter.java b/core/src/main/java/org/apache/iceberg/io/BasePartitionWriter.java index 1141d70ec7b6..7340abb8c8de 100644 --- a/core/src/main/java/org/apache/iceberg/io/BasePartitionWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BasePartitionWriter.java @@ -24,7 +24,6 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.StructLike; 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.Sets; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.StructLikeMap; @@ -32,9 +31,9 @@ public abstract class BasePartitionWriter implements PartitionWriter { - private final RollingFilesWriter dataWriter; - private final RollingFilesWriter equalityDeleteWriter; - private final RollingFilesWriter> positionDeleteWriter; + private final FileGroupWriter dataWriter; + private final FileGroupWriter equalityDeleteWriter; + private final FileGroupWriter> positionDeleteWriter; private final StructProjection projectRow; private final PositionDelete positionDelete = new PositionDelete<>(); private final StructLikeMap insertedRowMap; @@ -61,32 +60,20 @@ private long pos() { } } - public BasePartitionWriter(RollingFilesWriter dataWriter) { + public BasePartitionWriter(FileGroupWriter dataWriter) { this(dataWriter, null); } - public BasePartitionWriter(RollingFilesWriter dataWriter, - RollingFilesWriter> positionDeleteWriter) { + public BasePartitionWriter(FileGroupWriter dataWriter, + FileGroupWriter> positionDeleteWriter) { this(dataWriter, positionDeleteWriter, null, null, null); } - public BasePartitionWriter(RollingFilesWriter dataWriter, - RollingFilesWriter> positionDeleteWriter, - RollingFilesWriter equalityDeleteWriter, + public BasePartitionWriter(FileGroupWriter dataWriter, + FileGroupWriter> positionDeleteWriter, + FileGroupWriter equalityDeleteWriter, Schema tableSchema, List equalityFieldIds) { - Preconditions.checkNotNull(dataWriter, "Data writer should always not be null."); - if (positionDeleteWriter == null) { - // Only accept INSERT records. - Preconditions.checkArgument(equalityDeleteWriter == null); - } - - if (equalityDeleteWriter != null) { - // Only accept equality delete records. - Preconditions.checkNotNull(positionDeleteWriter, - "Position delete writer shouldn't be null when writing equality deletions."); - } - this.dataWriter = dataWriter; this.positionDeleteWriter = positionDeleteWriter; this.equalityDeleteWriter = equalityDeleteWriter; diff --git a/core/src/main/java/org/apache/iceberg/io/RollingFilesWriter.java b/core/src/main/java/org/apache/iceberg/io/FileGroupWriter.java similarity index 96% rename from core/src/main/java/org/apache/iceberg/io/RollingFilesWriter.java rename to core/src/main/java/org/apache/iceberg/io/FileGroupWriter.java index 124cdf9c97ef..08f826a4ccdf 100644 --- a/core/src/main/java/org/apache/iceberg/io/RollingFilesWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/FileGroupWriter.java @@ -27,7 +27,7 @@ * * @param to indicate the record data type. */ -public interface RollingFilesWriter extends Closeable { +public interface FileGroupWriter extends Closeable { /** * Write the row into the data files. diff --git a/core/src/main/java/org/apache/iceberg/io/PartitionFanoutTaskWriter.java b/core/src/main/java/org/apache/iceberg/io/PartitionFanoutTaskWriter.java deleted file mode 100644 index de9ba3ad741b..000000000000 --- a/core/src/main/java/org/apache/iceberg/io/PartitionFanoutTaskWriter.java +++ /dev/null @@ -1,39 +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.IOException; - -public class PartitionFanoutTaskWriter implements TaskWriter { - @Override - public PartitionWriter route(T row) { - return null; - } - - @Override - public void abort() throws IOException { - - } - - @Override - public WriterResult complete() throws IOException { - return null; - } -} diff --git a/core/src/main/java/org/apache/iceberg/io/PartitionGroupingTaskWriter.java b/core/src/main/java/org/apache/iceberg/io/PartitionGroupingTaskWriter.java deleted file mode 100644 index 3fa2cdaf6d05..000000000000 --- a/core/src/main/java/org/apache/iceberg/io/PartitionGroupingTaskWriter.java +++ /dev/null @@ -1,39 +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.IOException; - -public class PartitionGroupingTaskWriter implements TaskWriter { - @Override - public PartitionWriter route(T row) { - return null; - } - - @Override - public void abort() throws IOException { - - } - - @Override - public WriterResult complete() throws IOException { - return null; - } -} 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 96a5d340afa2..3e666ef2b5c1 100644 --- a/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java @@ -29,7 +29,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public abstract class PartitionedWriter extends BaseRollingFilesWriter { +public abstract class PartitionedWriter extends BaseFileGroupWriter { private static final Logger LOG = LoggerFactory.getLogger(PartitionedWriter.class); private final Set completedPartitions = Sets.newHashSet(); diff --git a/core/src/main/java/org/apache/iceberg/io/TaskWriter.java b/core/src/main/java/org/apache/iceberg/io/TaskWriter.java deleted file mode 100644 index b1e814cd19d1..000000000000 --- a/core/src/main/java/org/apache/iceberg/io/TaskWriter.java +++ /dev/null @@ -1,31 +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.IOException; - -public interface TaskWriter { - - PartitionWriter route(T row); - - void abort() throws IOException; - - WriterResult complete() throws IOException; -} diff --git a/core/src/main/java/org/apache/iceberg/io/UnpartitionedTaskWriter.java b/core/src/main/java/org/apache/iceberg/io/UnpartitionedTaskWriter.java deleted file mode 100644 index eb888f7956f0..000000000000 --- a/core/src/main/java/org/apache/iceberg/io/UnpartitionedTaskWriter.java +++ /dev/null @@ -1,40 +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.IOException; - -public class UnpartitionedTaskWriter implements TaskWriter { - - @Override - public PartitionWriter route(T row) { - return null; - } - - @Override - public void abort() throws IOException { - - } - - @Override - public WriterResult complete() throws IOException { - return null; - } -} 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 521d03b618f5..dce29765990b 100644 --- a/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java @@ -23,7 +23,7 @@ import org.apache.iceberg.ContentFileWriterFactory; import org.apache.iceberg.FileFormat; -public class UnpartitionedWriter extends BaseRollingFilesWriter { +public class UnpartitionedWriter extends BaseFileGroupWriter { private final RollingFileWriter currentWriter; 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 b7b9bc501910..bd87a51dc687 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 @@ -26,7 +26,7 @@ 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.RollingFilesWriter; +import org.apache.iceberg.io.FileGroupWriter; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; class IcebergStreamWriter extends AbstractStreamOperator @@ -37,7 +37,7 @@ class IcebergStreamWriter extends AbstractStreamOperator private final String fullTableName; private final TaskWriterFactory taskWriterFactory; - private transient RollingFilesWriter writer; + private transient FileGroupWriter writer; private transient int subTaskId; private transient int attemptId; 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 index ad792eb3ab51..2059352eb3c9 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedFanoutWriter.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedFanoutWriter.java @@ -24,12 +24,12 @@ import org.apache.iceberg.ContentFileWriterFactory; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.io.BaseRollingFilesWriter; +import org.apache.iceberg.io.BaseFileGroupWriter; 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 BaseRollingFilesWriter { +abstract class PartitionedFanoutWriter extends BaseFileGroupWriter { private final Map writers = Maps.newHashMap(); PartitionedFanoutWriter(FileFormat format, OutputFileFactory fileFactory, FileIO io, long targetFileSize, 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 23521792c824..bc7d0082cea9 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 @@ -46,7 +46,7 @@ import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.io.OutputFileFactory; -import org.apache.iceberg.io.RollingFilesWriter; +import org.apache.iceberg.io.FileGroupWriter; import org.apache.iceberg.io.WriterResult; import org.apache.iceberg.io.UnpartitionedWriter; import org.apache.iceberg.orc.ORC; @@ -95,26 +95,26 @@ public void initialize(int taskId, int attemptId) { } @Override - public RollingFilesWriter create() { + public FileGroupWriter create() { Preconditions.checkNotNull(outputFileFactory, "The outputFileFactory shouldn't be null if we have invoked the initialize()."); if (spec.fields().isEmpty()) { - return new MixedUnpartitionedRollingFilesWriter(format, appenderFactory, outputFileFactory, io, targetFileSizeBytes, + return new MixedUnpartitionedFileGroupWriter(format, appenderFactory, outputFileFactory, io, targetFileSizeBytes, schema, flinkSchema, tableProperties); } else { - return new MixedPartitionedRollingFilesWriter(spec, format, appenderFactory, outputFileFactory, + return new MixedPartitionedFileGroupWriter(spec, format, appenderFactory, outputFileFactory, io, targetFileSizeBytes, schema, flinkSchema, tableProperties); } } - private abstract static class BaseMixedRollingFilesWriter implements RollingFilesWriter { + private abstract static class BaseMixedFileGroupWriter implements FileGroupWriter { abstract boolean supportDeletion(); - abstract RollingFilesWriter dataTaskWriter(); + abstract FileGroupWriter dataTaskWriter(); - abstract RollingFilesWriter deleteTaskWriter(); + abstract FileGroupWriter deleteTaskWriter(); @Override public void write(RowData row) throws IOException { @@ -165,28 +165,28 @@ public void close() throws IOException { } } - private static class MixedUnpartitionedRollingFilesWriter extends BaseMixedRollingFilesWriter { + private static class MixedUnpartitionedFileGroupWriter extends BaseMixedFileGroupWriter { private final FileFormat format; - private final RollingFilesWriter dataRollingFilesWriter; - private final RollingFilesWriter deleteRollingFilesWriter; + private final FileGroupWriter dataFileGroupWriter; + private final FileGroupWriter deleteFileGroupWriter; - MixedUnpartitionedRollingFilesWriter(FileFormat format, FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, - RowType flinkSchema, Map tableProperties) { + MixedUnpartitionedFileGroupWriter(FileFormat format, FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, + RowType flinkSchema, Map tableProperties) { this.format = format; - this.dataRollingFilesWriter = new UnpartitionedWriter<>(format, fileFactory, io, + this.dataFileGroupWriter = new UnpartitionedWriter<>(format, fileFactory, io, targetFileSize, new DataFileWriterFactory<>(appenderFactory, PartitionSpec.unpartitioned())); // TODO: set the correct equality field ids. List equalityIds = ImmutableList.of(); if (supportDeletion()) { - this.deleteRollingFilesWriter = new UnpartitionedWriter<>(format, fileFactory, io, targetFileSize, + this.deleteFileGroupWriter = new UnpartitionedWriter<>(format, fileFactory, io, targetFileSize, new FlinkEqualityDeleterFactory(schema, flinkSchema, PartitionSpec.unpartitioned(), equalityIds, tableProperties)); } else { - this.deleteRollingFilesWriter = null; + this.deleteFileGroupWriter = null; } } @@ -196,28 +196,28 @@ boolean supportDeletion() { } @Override - RollingFilesWriter dataTaskWriter() { - return dataRollingFilesWriter; + FileGroupWriter dataTaskWriter() { + return dataFileGroupWriter; } @Override - RollingFilesWriter deleteTaskWriter() { - return deleteRollingFilesWriter; + FileGroupWriter deleteTaskWriter() { + return deleteFileGroupWriter; } } - private static class MixedPartitionedRollingFilesWriter extends BaseMixedRollingFilesWriter { + private static class MixedPartitionedFileGroupWriter extends BaseMixedFileGroupWriter { private final FileFormat format; - private final RollingFilesWriter dataRollingFilesWriter; - private final RollingFilesWriter deleteRollingFilesWriter; - private final RollingFilesWriter> posDelRollingFilesWriter; + private final FileGroupWriter dataFileGroupWriter; + private final FileGroupWriter deleteFileGroupWriter; + private final FileGroupWriter> posDelFileGroupWriter; - MixedPartitionedRollingFilesWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, - RowType flinkSchema, Map tableProperties) { + MixedPartitionedFileGroupWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, + RowType flinkSchema, Map tableProperties) { this.format = format; - this.dataRollingFilesWriter = + this.dataFileGroupWriter = new RowDataPartitionedFanoutWriter<>(spec, format, fileFactory, io, targetFileSize, schema, flinkSchema, new DataFileWriterFactory<>(appenderFactory, spec)); @@ -225,15 +225,15 @@ private static class MixedPartitionedRollingFilesWriter extends BaseMixedRolling List equalityIds = ImmutableList.of(); if (supportDeletion()) { - this.deleteRollingFilesWriter = + this.deleteFileGroupWriter = new RowDataPartitionedFanoutWriter<>(spec, format, fileFactory, io, targetFileSize, schema, flinkSchema, new FlinkEqualityDeleterFactory(schema, flinkSchema, spec, equalityIds, tableProperties)); - this.posDelRollingFilesWriter = + this.posDelFileGroupWriter = new PosPartitionedFanoutWriter<>(spec, format, fileFactory, io, targetFileSize, schema, flinkSchema, new FlinkPositionDeleteWriterFactory(schema, spec, tableProperties)); } else { - this.deleteRollingFilesWriter = null; - this.posDelRollingFilesWriter = null; + this.deleteFileGroupWriter = null; + this.posDelFileGroupWriter = null; } } @@ -266,13 +266,13 @@ public void write(RowData row) throws IOException { } @Override - RollingFilesWriter dataTaskWriter() { - return dataRollingFilesWriter; + FileGroupWriter dataTaskWriter() { + return dataFileGroupWriter; } @Override - RollingFilesWriter deleteTaskWriter() { - return deleteRollingFilesWriter; + FileGroupWriter deleteTaskWriter() { + return deleteFileGroupWriter; } } diff --git a/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java b/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java index f1102075f76f..f66f2e18df7d 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java @@ -20,10 +20,10 @@ package org.apache.iceberg.flink.sink; import java.io.Serializable; -import org.apache.iceberg.io.RollingFilesWriter; +import org.apache.iceberg.io.FileGroupWriter; /** - * Factory to create {@link RollingFilesWriter} + * Factory to create {@link FileGroupWriter} * * @param data type of record. */ @@ -38,9 +38,9 @@ public interface TaskWriterFactory extends Serializable { void initialize(int taskId, int attemptId); /** - * Initialize a {@link RollingFilesWriter} with given task id and attempt id. + * Initialize a {@link FileGroupWriter} with given task id and attempt id. * * @return a newly created task writer. */ - RollingFilesWriter create(); + FileGroupWriter create(); } 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 6271776f9961..c17da5825c74 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 @@ -37,7 +37,7 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.data.RandomRowData; -import org.apache.iceberg.io.RollingFilesWriter; +import org.apache.iceberg.io.FileGroupWriter; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.Assert; @@ -91,16 +91,16 @@ public void before() throws IOException { @Test public void testWriteZeroRecord() throws IOException { - try (RollingFilesWriter rollingFilesWriter = createTaskWriter(TARGET_FILE_SIZE)) { - rollingFilesWriter.close(); + try (FileGroupWriter fileGroupWriter = createTaskWriter(TARGET_FILE_SIZE)) { + fileGroupWriter.close(); - DataFile[] dataFiles = rollingFilesWriter.complete().dataFiles(); + DataFile[] dataFiles = fileGroupWriter.complete().dataFiles(); Assert.assertNotNull(dataFiles); Assert.assertEquals(0, dataFiles.length); // Close again. - rollingFilesWriter.close(); - dataFiles = rollingFilesWriter.complete().dataFiles(); + fileGroupWriter.close(); + dataFiles = fileGroupWriter.complete().dataFiles(); Assert.assertNotNull(dataFiles); Assert.assertEquals(0, dataFiles.length); } @@ -108,14 +108,14 @@ public void testWriteZeroRecord() throws IOException { @Test public void testCloseTwice() throws IOException { - try (RollingFilesWriter rollingFilesWriter = createTaskWriter(TARGET_FILE_SIZE)) { - rollingFilesWriter.write(SimpleDataUtil.createRowData(1, "hello")); - rollingFilesWriter.write(SimpleDataUtil.createRowData(2, "world")); - rollingFilesWriter.close(); // The first close - rollingFilesWriter.close(); // The second close + try (FileGroupWriter fileGroupWriter = createTaskWriter(TARGET_FILE_SIZE)) { + fileGroupWriter.write(SimpleDataUtil.createRowData(1, "hello")); + fileGroupWriter.write(SimpleDataUtil.createRowData(2, "world")); + fileGroupWriter.close(); // The first close + fileGroupWriter.close(); // The second close int expectedFiles = partitioned ? 2 : 1; - DataFile[] dataFiles = rollingFilesWriter.complete().dataFiles(); + DataFile[] dataFiles = fileGroupWriter.complete().dataFiles(); Assert.assertEquals(expectedFiles, dataFiles.length); FileSystem fs = FileSystem.get(CONF); @@ -127,12 +127,12 @@ public void testCloseTwice() throws IOException { @Test public void testAbort() throws IOException { - try (RollingFilesWriter rollingFilesWriter = createTaskWriter(TARGET_FILE_SIZE)) { - rollingFilesWriter.write(SimpleDataUtil.createRowData(1, "hello")); - rollingFilesWriter.write(SimpleDataUtil.createRowData(2, "world")); + try (FileGroupWriter fileGroupWriter = createTaskWriter(TARGET_FILE_SIZE)) { + fileGroupWriter.write(SimpleDataUtil.createRowData(1, "hello")); + fileGroupWriter.write(SimpleDataUtil.createRowData(2, "world")); - rollingFilesWriter.abort(); - DataFile[] dataFiles = rollingFilesWriter.complete().dataFiles(); + fileGroupWriter.abort(); + DataFile[] dataFiles = fileGroupWriter.complete().dataFiles(); int expectedFiles = partitioned ? 2 : 1; Assert.assertEquals(expectedFiles, dataFiles.length); @@ -146,17 +146,17 @@ public void testAbort() throws IOException { @Test public void testCompleteFiles() throws IOException { - try (RollingFilesWriter rollingFilesWriter = createTaskWriter(TARGET_FILE_SIZE)) { - rollingFilesWriter.write(SimpleDataUtil.createRowData(1, "a")); - rollingFilesWriter.write(SimpleDataUtil.createRowData(2, "b")); - rollingFilesWriter.write(SimpleDataUtil.createRowData(3, "c")); - rollingFilesWriter.write(SimpleDataUtil.createRowData(4, "d")); + try (FileGroupWriter fileGroupWriter = createTaskWriter(TARGET_FILE_SIZE)) { + fileGroupWriter.write(SimpleDataUtil.createRowData(1, "a")); + fileGroupWriter.write(SimpleDataUtil.createRowData(2, "b")); + fileGroupWriter.write(SimpleDataUtil.createRowData(3, "c")); + fileGroupWriter.write(SimpleDataUtil.createRowData(4, "d")); - DataFile[] dataFiles = rollingFilesWriter.complete().dataFiles(); + DataFile[] dataFiles = fileGroupWriter.complete().dataFiles(); int expectedFiles = partitioned ? 4 : 1; Assert.assertEquals(expectedFiles, dataFiles.length); - dataFiles = rollingFilesWriter.complete().dataFiles(); + dataFiles = fileGroupWriter.complete().dataFiles(); Assert.assertEquals(expectedFiles, dataFiles.length); FileSystem fs = FileSystem.get(CONF); @@ -186,7 +186,7 @@ public void testRollingWithTargetFileSize() throws IOException { if (format == FileFormat.ORC) { return; } - try (RollingFilesWriter rollingFilesWriter = createTaskWriter(4)) { + try (FileGroupWriter fileGroupWriter = createTaskWriter(4)) { List rows = Lists.newArrayListWithCapacity(8000); List records = Lists.newArrayListWithCapacity(8000); for (int i = 0; i < 2000; i++) { @@ -197,10 +197,10 @@ public void testRollingWithTargetFileSize() throws IOException { } for (RowData row : rows) { - rollingFilesWriter.write(row); + fileGroupWriter.write(row); } - DataFile[] dataFiles = rollingFilesWriter.complete().dataFiles(); + DataFile[] dataFiles = fileGroupWriter.complete().dataFiles(); Assert.assertEquals(8, dataFiles.length); AppendFiles appendFiles = table.newAppend(); @@ -216,14 +216,14 @@ public void testRollingWithTargetFileSize() throws IOException { @Test public void testRandomData() throws IOException { - try (RollingFilesWriter rollingFilesWriter = createTaskWriter(TARGET_FILE_SIZE)) { + try (FileGroupWriter fileGroupWriter = createTaskWriter(TARGET_FILE_SIZE)) { Iterable rows = RandomRowData.generate(SimpleDataUtil.SCHEMA, 100, 1996); for (RowData row : rows) { - rollingFilesWriter.write(row); + fileGroupWriter.write(row); } - rollingFilesWriter.close(); - DataFile[] dataFiles = rollingFilesWriter.complete().dataFiles(); + fileGroupWriter.close(); + DataFile[] dataFiles = fileGroupWriter.complete().dataFiles(); AppendFiles appendFiles = table.newAppend(); for (DataFile dataFile : dataFiles) { appendFiles.appendFile(dataFile); @@ -235,7 +235,7 @@ public void testRandomData() throws IOException { } } - private RollingFilesWriter createTaskWriter(long targetFileSize) { + private FileGroupWriter createTaskWriter(long targetFileSize) { TaskWriterFactory taskWriterFactory = new RowDataTaskWriterFactory(table.schema(), (RowType) SimpleDataUtil.FLINK_SCHEMA.toRowDataType().getLogicalType(), table.spec(), table.locationProvider(), table.io(), table.encryption(), 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 28db58acfb4b..d8050ceb356c 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 @@ -37,7 +37,7 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.io.OutputFileFactory; -import org.apache.iceberg.io.RollingFilesWriter; +import org.apache.iceberg.io.FileGroupWriter; import org.apache.iceberg.io.UnpartitionedWriter; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.SparkSchemaUtil; @@ -103,7 +103,7 @@ private List rewriteDataForTask(CombinedScanTask task) throws Exceptio OutputFileFactory fileFactory = new OutputFileFactory( spec, format, locations, io.value(), encryptionManager.value(), partitionId, taskId); - RollingFilesWriter writer; + FileGroupWriter writer; if (spec.fields().isEmpty()) { writer = new UnpartitionedWriter<>(format, fileFactory, io.value(), Long.MAX_VALUE, new DataFileWriterFactory<>(appenderFactory, spec)); From cce17ce1fdfecb8f1700cf3b5754a5aeeedbb065 Mon Sep 17 00:00:00 2001 From: huzheng Date: Thu, 5 Nov 2020 14:55:19 +0800 Subject: [PATCH 11/41] Revert "Introduce the BasePartitionWriter and StructLikeMap" This reverts commit 4563fb90691752a3dc06d788278d9953f752fa65. --- .../iceberg/io/BasePartitionWriter.java | 55 +++----- .../apache/iceberg/io/PartitionWriter.java | 4 - .../apache/iceberg/util/StructLikeMap.java | 127 ------------------ .../flink/sink/RowDataTaskWriterFactory.java | 4 + 4 files changed, 20 insertions(+), 170 deletions(-) delete mode 100644 core/src/main/java/org/apache/iceberg/util/StructLikeMap.java diff --git a/core/src/main/java/org/apache/iceberg/io/BasePartitionWriter.java b/core/src/main/java/org/apache/iceberg/io/BasePartitionWriter.java index 7340abb8c8de..bbe865ab0096 100644 --- a/core/src/main/java/org/apache/iceberg/io/BasePartitionWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BasePartitionWriter.java @@ -26,7 +26,7 @@ import org.apache.iceberg.deletes.PositionDelete; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.util.StructLikeMap; +import org.apache.iceberg.util.StructLikeSet; import org.apache.iceberg.util.StructProjection; public abstract class BasePartitionWriter implements PartitionWriter { @@ -35,30 +35,10 @@ public abstract class BasePartitionWriter implements PartitionWriter { private final FileGroupWriter equalityDeleteWriter; private final FileGroupWriter> positionDeleteWriter; private final StructProjection projectRow; + private final Schema deleteSchema; + private final List equalityFieldIds; private final PositionDelete positionDelete = new PositionDelete<>(); - private final StructLikeMap insertedRowMap; - - private static class FilePos { - private final CharSequence path; - private final long pos; - - private FilePos(CharSequence path, long pos) { - this.path = path; - this.pos = pos; - } - - private static FilePos create(CharSequence path, long pos) { - return new FilePos(path, pos); - } - - private CharSequence path() { - return path; - } - - private long pos() { - return pos; - } - } + private final StructLikeSet insertedDataSet; public BasePartitionWriter(FileGroupWriter dataWriter) { this(dataWriter, null); @@ -72,27 +52,27 @@ public BasePartitionWriter(FileGroupWriter dataWriter, public BasePartitionWriter(FileGroupWriter dataWriter, FileGroupWriter> positionDeleteWriter, FileGroupWriter equalityDeleteWriter, - Schema tableSchema, + Schema writeSchema, List equalityFieldIds) { this.dataWriter = dataWriter; this.positionDeleteWriter = positionDeleteWriter; this.equalityDeleteWriter = equalityDeleteWriter; - - Schema deleteSchema = TypeUtil.select(tableSchema, Sets.newHashSet(equalityFieldIds)); - this.projectRow = StructProjection.create(tableSchema, deleteSchema); - this.insertedRowMap = StructLikeMap.create(deleteSchema.asStruct()); + this.deleteSchema = TypeUtil.select(writeSchema, Sets.newHashSet(equalityFieldIds)); + this.equalityFieldIds = equalityFieldIds; + this.projectRow = StructProjection.create(writeSchema, deleteSchema); + this.insertedDataSet = StructLikeSet.create(deleteSchema.asStruct()); } protected abstract StructLike asStructLike(T record); @Override public void append(T row) throws IOException { + this.dataWriter.write(row); + if (allowEqualityDelete()) { - FilePos filePos = FilePos.create(dataWriter.currentPath(), dataWriter.currentPos()); - insertedRowMap.put(projectRow.wrap(asStructLike(row)), filePos); + // TODO Put the > into the insert data MAP, rather than SET. + insertedDataSet.add(projectRow.wrap(asStructLike(row))); } - - this.dataWriter.write(row); } @Override @@ -101,14 +81,11 @@ public void delete(T deleteRow) throws IOException { throw new UnsupportedOperationException("Couldn't accept equality deletions."); } - FilePos existingFilePos = insertedRowMap.get(projectRow.wrap(asStructLike(deleteRow))); - - if (existingFilePos == null) { - // Delete the row which has been written before this writer. + if (!insertedDataSet.contains(projectRow.wrap(asStructLike(deleteRow)))) { this.equalityDeleteWriter.write(deleteRow); } else { - // Delete the row which was written in current writer. - this.positionDeleteWriter.write(positionDelete.set(existingFilePos.path(), existingFilePos.pos(), deleteRow)); + // TODO Get the correct path and pos from insert data MAP rather than SET. + this.positionDeleteWriter.write(positionDelete.set(dataWriter.currentPath(), dataWriter.currentPos(), deleteRow)); } } diff --git a/core/src/main/java/org/apache/iceberg/io/PartitionWriter.java b/core/src/main/java/org/apache/iceberg/io/PartitionWriter.java index 7c01c6570dee..b40e0de9510b 100644 --- a/core/src/main/java/org/apache/iceberg/io/PartitionWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/PartitionWriter.java @@ -27,10 +27,6 @@ public interface PartitionWriter { void delete(T deleteRow) throws IOException; - default void delete(CharSequence path, long pos) throws IOException { - delete(path, pos, null); - } - void delete(CharSequence path, long pos, T row) throws IOException; void abort() throws IOException; diff --git a/core/src/main/java/org/apache/iceberg/util/StructLikeMap.java b/core/src/main/java/org/apache/iceberg/util/StructLikeMap.java deleted file mode 100644 index f45b85a2a3c6..000000000000 --- a/core/src/main/java/org/apache/iceberg/util/StructLikeMap.java +++ /dev/null @@ -1,127 +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.util; - -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 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/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java b/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java index bc7d0082cea9..430e88cf5131 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 @@ -24,6 +24,7 @@ import java.io.UncheckedIOException; import java.util.List; import java.util.Map; +import java.util.Set; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.ContentFileWriterFactory; @@ -53,6 +54,7 @@ import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; public class RowDataTaskWriterFactory implements TaskWriterFactory { private final Schema schema; @@ -67,6 +69,7 @@ public class RowDataTaskWriterFactory implements TaskWriterFactory { private final FileAppenderFactory appenderFactory; private transient OutputFileFactory outputFileFactory; + private transient Set eqDeleteCache; public RowDataTaskWriterFactory(Schema schema, RowType flinkSchema, @@ -92,6 +95,7 @@ public RowDataTaskWriterFactory(Schema schema, @Override public void initialize(int taskId, int attemptId) { this.outputFileFactory = new OutputFileFactory(spec, format, locations, io, encryptionManager, taskId, attemptId); + this.eqDeleteCache = Sets.newHashSet(); } @Override From b286bddb527ce2c18d22369604ae322ad4d51f9b Mon Sep 17 00:00:00 2001 From: huzheng Date: Thu, 5 Nov 2020 14:55:28 +0800 Subject: [PATCH 12/41] Revert "temp" This reverts commit afda48d1e4208d6c2089bc433de43565c119c2d0. --- .../iceberg/deletes/PositionDeleteWriter.java | 23 +-- .../iceberg/io/BasePartitionWriter.java | 142 ------------------ ...leGroupWriter.java => BaseTaskWriter.java} | 22 +-- .../apache/iceberg/io/PartitionWriter.java | 35 ----- .../apache/iceberg/io/PartitionedWriter.java | 2 +- .../{FileGroupWriter.java => TaskWriter.java} | 14 +- ...riterResult.java => TaskWriterResult.java} | 22 +-- .../iceberg/io/UnpartitionedWriter.java | 12 +- .../iceberg/avro/TestAvroDeleteWriters.java | 4 +- .../org/apache/iceberg/data/FileHelpers.java | 2 +- .../FlinkPositionDeleteWriterFactory.java | 87 ----------- .../flink/sink/IcebergStreamWriter.java | 4 +- .../flink/sink/PartitionedFanoutWriter.java | 4 +- .../flink/sink/RowDataTaskWriterFactory.java | 126 +++++----------- .../iceberg/flink/sink/TaskWriterFactory.java | 8 +- .../iceberg/flink/sink/TestTaskWriters.java | 64 ++++---- .../parquet/TestParquetDeleteWriters.java | 4 +- .../iceberg/spark/source/RowDataRewriter.java | 4 +- 18 files changed, 105 insertions(+), 474 deletions(-) delete mode 100644 core/src/main/java/org/apache/iceberg/io/BasePartitionWriter.java rename core/src/main/java/org/apache/iceberg/io/{BaseFileGroupWriter.java => BaseTaskWriter.java} (85%) delete mode 100644 core/src/main/java/org/apache/iceberg/io/PartitionWriter.java rename core/src/main/java/org/apache/iceberg/io/{FileGroupWriter.java => TaskWriter.java} (81%) rename core/src/main/java/org/apache/iceberg/io/{WriterResult.java => TaskWriterResult.java} (85%) delete mode 100644 flink/src/main/java/org/apache/iceberg/flink/sink/FlinkPositionDeleteWriterFactory.java 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 a40f711b0c11..7bacdc07ba83 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteWriter.java +++ b/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteWriter.java @@ -19,14 +19,13 @@ 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; -import org.apache.iceberg.Metrics; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.StructLike; import org.apache.iceberg.encryption.EncryptionKeyMetadata; @@ -34,7 +33,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.util.CharSequenceSet; -public class PositionDeleteWriter implements ContentFileWriter> { +public class PositionDeleteWriter implements Closeable { private final FileAppender appender; private final FileFormat format; private final String location; @@ -86,24 +85,8 @@ public Set referencedDataFiles() { return pathSet; } - @Override - public DeleteFile toContentFile() { + public DeleteFile toDeleteFile() { Preconditions.checkState(deleteFile != null, "Cannot create delete file from unclosed writer"); return deleteFile; } - - @Override - public void write(PositionDelete record) { - delete(record.path(), record.pos(), record.row()); - } - - @Override - public Metrics metrics() { - return appender.metrics(); - } - - @Override - public long length() { - return appender.length(); - } } diff --git a/core/src/main/java/org/apache/iceberg/io/BasePartitionWriter.java b/core/src/main/java/org/apache/iceberg/io/BasePartitionWriter.java deleted file mode 100644 index bbe865ab0096..000000000000 --- a/core/src/main/java/org/apache/iceberg/io/BasePartitionWriter.java +++ /dev/null @@ -1,142 +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.IOException; -import java.util.List; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.deletes.PositionDelete; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.util.StructLikeSet; -import org.apache.iceberg.util.StructProjection; - -public abstract class BasePartitionWriter implements PartitionWriter { - - private final FileGroupWriter dataWriter; - private final FileGroupWriter equalityDeleteWriter; - private final FileGroupWriter> positionDeleteWriter; - private final StructProjection projectRow; - private final Schema deleteSchema; - private final List equalityFieldIds; - private final PositionDelete positionDelete = new PositionDelete<>(); - private final StructLikeSet insertedDataSet; - - public BasePartitionWriter(FileGroupWriter dataWriter) { - this(dataWriter, null); - } - - public BasePartitionWriter(FileGroupWriter dataWriter, - FileGroupWriter> positionDeleteWriter) { - this(dataWriter, positionDeleteWriter, null, null, null); - } - - public BasePartitionWriter(FileGroupWriter dataWriter, - FileGroupWriter> positionDeleteWriter, - FileGroupWriter equalityDeleteWriter, - Schema writeSchema, - List equalityFieldIds) { - this.dataWriter = dataWriter; - this.positionDeleteWriter = positionDeleteWriter; - this.equalityDeleteWriter = equalityDeleteWriter; - this.deleteSchema = TypeUtil.select(writeSchema, Sets.newHashSet(equalityFieldIds)); - this.equalityFieldIds = equalityFieldIds; - this.projectRow = StructProjection.create(writeSchema, deleteSchema); - this.insertedDataSet = StructLikeSet.create(deleteSchema.asStruct()); - } - - protected abstract StructLike asStructLike(T record); - - @Override - public void append(T row) throws IOException { - this.dataWriter.write(row); - - if (allowEqualityDelete()) { - // TODO Put the > into the insert data MAP, rather than SET. - insertedDataSet.add(projectRow.wrap(asStructLike(row))); - } - } - - @Override - public void delete(T deleteRow) throws IOException { - if (!allowEqualityDelete()) { - throw new UnsupportedOperationException("Couldn't accept equality deletions."); - } - - if (!insertedDataSet.contains(projectRow.wrap(asStructLike(deleteRow)))) { - this.equalityDeleteWriter.write(deleteRow); - } else { - // TODO Get the correct path and pos from insert data MAP rather than SET. - this.positionDeleteWriter.write(positionDelete.set(dataWriter.currentPath(), dataWriter.currentPos(), deleteRow)); - } - } - - @Override - public void delete(CharSequence path, long pos, T row) throws IOException { - if (!allowPositionDelete()) { - throw new UnsupportedOperationException("Couldn't accept positional deletions."); - } - - this.positionDeleteWriter.write(positionDelete.set(path, pos, row)); - } - - @Override - public void abort() throws IOException { - if (dataWriter != null) { - dataWriter.abort(); - } - - if (equalityDeleteWriter != null) { - equalityDeleteWriter.abort(); - } - - if (positionDeleteWriter != null) { - positionDeleteWriter.abort(); - } - } - - @Override - public WriterResult complete() throws IOException { - WriterResult.Builder builder = WriterResult.builder(); - - if (dataWriter != null) { - builder.addAll(dataWriter.complete().contentFiles()); - } - - if (equalityDeleteWriter != null) { - builder.addAll(equalityDeleteWriter.complete().contentFiles()); - } - - if (positionDeleteWriter != null) { - builder.addAll(positionDeleteWriter.complete().contentFiles()); - } - - return builder.build(); - } - - private boolean allowEqualityDelete() { - return equalityDeleteWriter != null && positionDeleteWriter != null; - } - - private boolean allowPositionDelete() { - return positionDeleteWriter != null; - } -} diff --git a/core/src/main/java/org/apache/iceberg/io/BaseFileGroupWriter.java b/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java similarity index 85% rename from core/src/main/java/org/apache/iceberg/io/BaseFileGroupWriter.java rename to core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java index a2b68dac32de..5aaa2923e263 100644 --- a/core/src/main/java/org/apache/iceberg/io/BaseFileGroupWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java @@ -30,17 +30,17 @@ import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.util.Tasks; -public abstract class BaseFileGroupWriter implements FileGroupWriter { - private final WriterResult.Builder builder; +public abstract class BaseTaskWriter implements TaskWriter { + private final TaskWriterResult.Builder builder; private final FileFormat format; private final OutputFileFactory fileFactory; private final FileIO io; private final long targetFileSize; private final ContentFileWriterFactory writerFactory; - protected BaseFileGroupWriter(FileFormat format, OutputFileFactory fileFactory, FileIO io, long targetFileSize, - ContentFileWriterFactory writerFactory) { - this.builder = WriterResult.builder(); + protected BaseTaskWriter(FileFormat format, OutputFileFactory fileFactory, FileIO io, long targetFileSize, + ContentFileWriterFactory writerFactory) { + this.builder = TaskWriterResult.builder(); this.format = format; this.fileFactory = fileFactory; this.io = io; @@ -53,7 +53,7 @@ public void abort() throws IOException { close(); // clean up files created by this writer - WriterResult result = builder.build(); + TaskWriterResult result = builder.build(); Tasks.foreach(result.contentFiles()) .throwFailureWhenFinished() @@ -62,7 +62,7 @@ public void abort() throws IOException { } @Override - public WriterResult complete() throws IOException { + public TaskWriterResult complete() throws IOException { close(); return builder.build(); @@ -91,14 +91,6 @@ public void add(T record) throws IOException { } } - public CharSequence currentPath() { - return currentFile != null ? currentFile.encryptingOutputFile().location() : null; - } - - public long currentPos() { - return currentRows; - } - private void openCurrent() { if (partitionKey == null) { // unpartitioned diff --git a/core/src/main/java/org/apache/iceberg/io/PartitionWriter.java b/core/src/main/java/org/apache/iceberg/io/PartitionWriter.java deleted file mode 100644 index b40e0de9510b..000000000000 --- a/core/src/main/java/org/apache/iceberg/io/PartitionWriter.java +++ /dev/null @@ -1,35 +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.IOException; - -public interface PartitionWriter { - - void append(T row) throws IOException; - - void delete(T deleteRow) throws IOException; - - void delete(CharSequence path, long pos, T row) throws IOException; - - void abort() throws IOException; - - WriterResult complete() throws IOException; -} 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 3e666ef2b5c1..ef786121c4ae 100644 --- a/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java @@ -29,7 +29,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public abstract class PartitionedWriter extends BaseFileGroupWriter { +public abstract class PartitionedWriter extends BaseTaskWriter { private static final Logger LOG = LoggerFactory.getLogger(PartitionedWriter.class); private final Set completedPartitions = Sets.newHashSet(); diff --git a/core/src/main/java/org/apache/iceberg/io/FileGroupWriter.java b/core/src/main/java/org/apache/iceberg/io/TaskWriter.java similarity index 81% rename from core/src/main/java/org/apache/iceberg/io/FileGroupWriter.java rename to core/src/main/java/org/apache/iceberg/io/TaskWriter.java index 08f826a4ccdf..36a699fe8803 100644 --- a/core/src/main/java/org/apache/iceberg/io/FileGroupWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/TaskWriter.java @@ -27,7 +27,7 @@ * * @param to indicate the record data type. */ -public interface FileGroupWriter extends Closeable { +public interface TaskWriter extends Closeable { /** * Write the row into the data files. @@ -41,20 +41,10 @@ public interface FileGroupWriter extends Closeable { */ void abort() throws IOException; - /** - * Get the path of current opening file. - */ - CharSequence currentPath() throws IOException; - - /** - * Get the row number (starts from 0) of current opening file. - */ - long currentPos() throws IOException; - /** * Close the writer and get the completed data/delete files. * * @return the completed data files of this task writer. */ - WriterResult complete() throws IOException; + TaskWriterResult complete() throws IOException; } diff --git a/core/src/main/java/org/apache/iceberg/io/WriterResult.java b/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java similarity index 85% rename from core/src/main/java/org/apache/iceberg/io/WriterResult.java rename to core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java index 7aa3b833b38d..3afa23435428 100644 --- a/core/src/main/java/org/apache/iceberg/io/WriterResult.java +++ b/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java @@ -28,11 +28,11 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -public class WriterResult { +public class TaskWriterResult { private DataFile[] dataFiles; private DeleteFile[] deleteFiles; - private WriterResult(List dataFiles, List deleteFiles) { + private TaskWriterResult(List dataFiles, List deleteFiles) { this.dataFiles = dataFiles.toArray(new DataFile[0]); this.deleteFiles = deleteFiles.toArray(new DeleteFile[0]); } @@ -74,7 +74,7 @@ public static Builder builder() { return new Builder(); } - public static WriterResult concat(WriterResult result0, WriterResult result1) { + public static TaskWriterResult concat(TaskWriterResult result0, TaskWriterResult result1) { Builder builder = new Builder(); builder.addAll(result0.dataFiles); @@ -94,19 +94,13 @@ private Builder() { this.deleteFiles = Lists.newArrayList(); } - public void addAll(Iterable> iterable) { - for (ContentFile file : iterable) { + public void addAll(ContentFile... files) { + for (ContentFile file : files) { add(file); } } - public void addAll(ContentFile... files) { - for (ContentFile file : files) { - add(file); - } - } - - public void add(ContentFile contentFile) { + public void add(ContentFile contentFile) { Preconditions.checkNotNull(contentFile, "Content file shouldn't be null."); switch (contentFile.content()) { case DATA: @@ -123,8 +117,8 @@ public void add(ContentFile contentFile) { } } - public WriterResult build() { - return new WriterResult(dataFiles, deleteFiles); + public TaskWriterResult build() { + return new TaskWriterResult(dataFiles, deleteFiles); } } } 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 dce29765990b..994fe259ef18 100644 --- a/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java @@ -23,7 +23,7 @@ import org.apache.iceberg.ContentFileWriterFactory; import org.apache.iceberg.FileFormat; -public class UnpartitionedWriter extends BaseFileGroupWriter { +public class UnpartitionedWriter extends BaseTaskWriter { private final RollingFileWriter currentWriter; @@ -38,16 +38,6 @@ public void write(T record) throws IOException { currentWriter.add(record); } - @Override - public CharSequence currentPath() throws IOException { - return currentWriter.currentPath(); - } - - @Override - public long currentPos() throws IOException { - return currentWriter.currentPos(); - } - @Override public void close() throws IOException { currentWriter.close(); 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 d6184ee05317..1f8bb66c0cdd 100644 --- a/core/src/test/java/org/apache/iceberg/avro/TestAvroDeleteWriters.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestAvroDeleteWriters.java @@ -138,7 +138,7 @@ public void testPositionDeleteWriter() throws IOException { } } - DeleteFile metadata = deleteWriter.toContentFile(); + DeleteFile metadata = deleteWriter.toDeleteFile(); 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.toContentFile(); + DeleteFile metadata = deleteWriter.toDeleteFile(); 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/test/java/org/apache/iceberg/data/FileHelpers.java b/data/src/test/java/org/apache/iceberg/data/FileHelpers.java index 43d3ca474ccf..0755ffb8ca18 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.toContentFile(), writer.referencedDataFiles()); + return Pair.of(writer.toDeleteFile(), writer.referencedDataFiles()); } public static DeleteFile writeDeleteFile(Table table, OutputFile out, List deletes, Schema deleteRowSchema) diff --git a/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkPositionDeleteWriterFactory.java b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkPositionDeleteWriterFactory.java deleted file mode 100644 index 23ea8badc3e4..000000000000 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkPositionDeleteWriterFactory.java +++ /dev/null @@ -1,87 +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.io.Serializable; -import java.io.UncheckedIOException; -import java.util.Map; -import org.apache.flink.table.data.RowData; -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.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.deletes.PositionDelete; -import org.apache.iceberg.encryption.EncryptedOutputFile; -import org.apache.iceberg.parquet.Parquet; - -public class FlinkPositionDeleteWriterFactory - implements ContentFileWriterFactory>, Serializable { - private final Schema schema; - private final PartitionSpec spec; - private final Map props; - - public FlinkPositionDeleteWriterFactory(Schema schema, - PartitionSpec spec, - Map props) { - this.schema = schema; - this.spec = spec; - this.props = props; - } - - - @Override - public ContentFileWriter> createWriter(PartitionKey partitionKey, - EncryptedOutputFile outputFile, - FileFormat fileFormat) { - try { - switch (fileFormat) { - case AVRO: - return Avro.writeDeletes(outputFile.encryptingOutputFile()) - .rowSchema(schema) - .withSpec(spec) - .setAll(props) - .withPartition(partitionKey) - .overwrite() - .buildPositionWriter(); - - - case PARQUET: - return Parquet.writeDeletes(outputFile.encryptingOutputFile()) - .rowSchema(schema) - .withSpec(spec) - .setAll(props) - .withPartition(partitionKey) - .overwrite() - .buildPositionWriter(); - - case ORC: - default: - throw new UnsupportedOperationException(); - } - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } -} 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 bd87a51dc687..8e9861171534 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 @@ -26,7 +26,7 @@ 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.FileGroupWriter; +import org.apache.iceberg.io.TaskWriter; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; class IcebergStreamWriter extends AbstractStreamOperator @@ -37,7 +37,7 @@ class IcebergStreamWriter extends AbstractStreamOperator private final String fullTableName; private final TaskWriterFactory taskWriterFactory; - private transient FileGroupWriter writer; + private transient TaskWriter writer; private transient int subTaskId; private transient int attemptId; 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 index 2059352eb3c9..74c3bd570a91 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedFanoutWriter.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedFanoutWriter.java @@ -24,12 +24,12 @@ import org.apache.iceberg.ContentFileWriterFactory; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.io.BaseFileGroupWriter; +import org.apache.iceberg.io.BaseTaskWriter; 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 BaseFileGroupWriter { +abstract class PartitionedFanoutWriter extends BaseTaskWriter { private final Map writers = Maps.newHashMap(); PartitionedFanoutWriter(FileFormat format, OutputFileFactory fileFactory, FileIO io, long targetFileSize, 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 430e88cf5131..aee79a96695e 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 @@ -24,7 +24,6 @@ import java.io.UncheckedIOException; import java.util.List; import java.util.Map; -import java.util.Set; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.ContentFileWriterFactory; @@ -35,7 +34,6 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.deletes.PositionDelete; import org.apache.iceberg.encryption.EncryptionManager; import org.apache.iceberg.flink.RowDataWrapper; import org.apache.iceberg.flink.data.FlinkAvroWriter; @@ -47,14 +45,13 @@ import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.io.OutputFileFactory; -import org.apache.iceberg.io.FileGroupWriter; -import org.apache.iceberg.io.WriterResult; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.TaskWriterResult; 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; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; public class RowDataTaskWriterFactory implements TaskWriterFactory { private final Schema schema; @@ -69,7 +66,6 @@ public class RowDataTaskWriterFactory implements TaskWriterFactory { private final FileAppenderFactory appenderFactory; private transient OutputFileFactory outputFileFactory; - private transient Set eqDeleteCache; public RowDataTaskWriterFactory(Schema schema, RowType flinkSchema, @@ -95,30 +91,29 @@ public RowDataTaskWriterFactory(Schema schema, @Override public void initialize(int taskId, int attemptId) { this.outputFileFactory = new OutputFileFactory(spec, format, locations, io, encryptionManager, taskId, attemptId); - this.eqDeleteCache = Sets.newHashSet(); } @Override - public FileGroupWriter create() { + public TaskWriter create() { Preconditions.checkNotNull(outputFileFactory, "The outputFileFactory shouldn't be null if we have invoked the initialize()."); if (spec.fields().isEmpty()) { - return new MixedUnpartitionedFileGroupWriter(format, appenderFactory, outputFileFactory, io, targetFileSizeBytes, + return new MixedUnpartitionedTaskWriter(format, appenderFactory, outputFileFactory, io, targetFileSizeBytes, schema, flinkSchema, tableProperties); } else { - return new MixedPartitionedFileGroupWriter(spec, format, appenderFactory, outputFileFactory, + return new MixedPartitionedTaskWriter(spec, format, appenderFactory, outputFileFactory, io, targetFileSizeBytes, schema, flinkSchema, tableProperties); } } - private abstract static class BaseMixedFileGroupWriter implements FileGroupWriter { + private abstract static class BaseMixedTaskWriter implements TaskWriter { abstract boolean supportDeletion(); - abstract FileGroupWriter dataTaskWriter(); + abstract TaskWriter dataTaskWriter(); - abstract FileGroupWriter deleteTaskWriter(); + abstract TaskWriter deleteTaskWriter(); @Override public void write(RowData row) throws IOException { @@ -152,9 +147,9 @@ public void abort() throws IOException { } @Override - public WriterResult complete() throws IOException { + public TaskWriterResult complete() throws IOException { if (supportDeletion()) { - return WriterResult.concat(dataTaskWriter().complete(), deleteTaskWriter().complete()); + return TaskWriterResult.concat(dataTaskWriter().complete(), deleteTaskWriter().complete()); } else { return dataTaskWriter().complete(); } @@ -169,28 +164,28 @@ public void close() throws IOException { } } - private static class MixedUnpartitionedFileGroupWriter extends BaseMixedFileGroupWriter { + private static class MixedUnpartitionedTaskWriter extends BaseMixedTaskWriter { private final FileFormat format; - private final FileGroupWriter dataFileGroupWriter; - private final FileGroupWriter deleteFileGroupWriter; + private final TaskWriter dataTaskWriter; + private final TaskWriter deleteTaskWriter; - MixedUnpartitionedFileGroupWriter(FileFormat format, FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, - RowType flinkSchema, Map tableProperties) { + MixedUnpartitionedTaskWriter(FileFormat format, FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, + RowType flinkSchema, Map tableProperties) { this.format = format; - this.dataFileGroupWriter = new UnpartitionedWriter<>(format, fileFactory, io, + this.dataTaskWriter = new UnpartitionedWriter<>(format, fileFactory, io, targetFileSize, new DataFileWriterFactory<>(appenderFactory, PartitionSpec.unpartitioned())); // TODO: set the correct equality field ids. List equalityIds = ImmutableList.of(); if (supportDeletion()) { - this.deleteFileGroupWriter = new UnpartitionedWriter<>(format, fileFactory, io, targetFileSize, + this.deleteTaskWriter = new UnpartitionedWriter<>(format, fileFactory, io, targetFileSize, new FlinkEqualityDeleterFactory(schema, flinkSchema, PartitionSpec.unpartitioned(), equalityIds, tableProperties)); } else { - this.deleteFileGroupWriter = null; + this.deleteTaskWriter = null; } } @@ -200,28 +195,27 @@ boolean supportDeletion() { } @Override - FileGroupWriter dataTaskWriter() { - return dataFileGroupWriter; + TaskWriter dataTaskWriter() { + return dataTaskWriter; } @Override - FileGroupWriter deleteTaskWriter() { - return deleteFileGroupWriter; + TaskWriter deleteTaskWriter() { + return deleteTaskWriter; } } - private static class MixedPartitionedFileGroupWriter extends BaseMixedFileGroupWriter { + private static class MixedPartitionedTaskWriter extends BaseMixedTaskWriter { private final FileFormat format; - private final FileGroupWriter dataFileGroupWriter; - private final FileGroupWriter deleteFileGroupWriter; - private final FileGroupWriter> posDelFileGroupWriter; + private final TaskWriter dataTaskWriter; + private final TaskWriter deleteTaskWriter; - MixedPartitionedFileGroupWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, - RowType flinkSchema, Map tableProperties) { + MixedPartitionedTaskWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, + RowType flinkSchema, Map tableProperties) { this.format = format; - this.dataFileGroupWriter = + this.dataTaskWriter = new RowDataPartitionedFanoutWriter<>(spec, format, fileFactory, io, targetFileSize, schema, flinkSchema, new DataFileWriterFactory<>(appenderFactory, spec)); @@ -229,15 +223,11 @@ private static class MixedPartitionedFileGroupWriter extends BaseMixedFileGroupW List equalityIds = ImmutableList.of(); if (supportDeletion()) { - this.deleteFileGroupWriter = + this.deleteTaskWriter = new RowDataPartitionedFanoutWriter<>(spec, format, fileFactory, io, targetFileSize, schema, flinkSchema, new FlinkEqualityDeleterFactory(schema, flinkSchema, spec, equalityIds, tableProperties)); - this.posDelFileGroupWriter = - new PosPartitionedFanoutWriter<>(spec, format, fileFactory, io, targetFileSize, schema, - flinkSchema, new FlinkPositionDeleteWriterFactory(schema, spec, tableProperties)); } else { - this.deleteFileGroupWriter = null; - this.posDelFileGroupWriter = null; + this.deleteTaskWriter = null; } } @@ -247,36 +237,13 @@ boolean supportDeletion() { } @Override - public void write(RowData row) throws IOException { - switch (row.getRowKind()) { - - case INSERT: - case UPDATE_AFTER: - dataTaskWriter().write(row); - break; - - case UPDATE_BEFORE: - case DELETE: - if (supportDeletion()) { - deleteTaskWriter().write(row); - } else { - throw new UnsupportedOperationException("Couldn't accept deletion in task writer."); - } - break; - - default: - throw new UnsupportedOperationException("Unrecognized row kind: " + row.getRowKind()); - } + TaskWriter dataTaskWriter() { + return dataTaskWriter; } @Override - FileGroupWriter dataTaskWriter() { - return dataFileGroupWriter; - } - - @Override - FileGroupWriter deleteTaskWriter() { - return deleteFileGroupWriter; + TaskWriter deleteTaskWriter() { + return deleteTaskWriter; } } @@ -301,27 +268,6 @@ protected PartitionKey partition(RowData row) { } } - private static class PosPartitionedFanoutWriter - extends PartitionedFanoutWriter> { - - private final PartitionKey partitionKey; - private final RowDataWrapper rowDataWrapper; - - PosPartitionedFanoutWriter(PartitionSpec spec, FileFormat format, OutputFileFactory fileFactory, FileIO io, - long targetFileSize, Schema schema, RowType flinkSchema, - ContentFileWriterFactory> contentFileWriter) { - super(format, fileFactory, io, targetFileSize, contentFileWriter); - this.partitionKey = new PartitionKey(spec, schema); - this.rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); - } - - @Override - protected PartitionKey partition(PositionDelete row) { - partitionKey.partition(rowDataWrapper.wrap(row.row())); - return partitionKey; - } - } - public static class FlinkFileAppenderFactory implements FileAppenderFactory, Serializable { private final Schema schema; private final RowType flinkSchema; diff --git a/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java b/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java index f66f2e18df7d..9d56ec6a812a 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java @@ -20,10 +20,10 @@ package org.apache.iceberg.flink.sink; import java.io.Serializable; -import org.apache.iceberg.io.FileGroupWriter; +import org.apache.iceberg.io.TaskWriter; /** - * Factory to create {@link FileGroupWriter} + * Factory to create {@link TaskWriter} * * @param data type of record. */ @@ -38,9 +38,9 @@ public interface TaskWriterFactory extends Serializable { void initialize(int taskId, int attemptId); /** - * Initialize a {@link FileGroupWriter} with given task id and attempt id. + * Initialize a {@link TaskWriter} with given task id and attempt id. * * @return a newly created task writer. */ - FileGroupWriter create(); + TaskWriter create(); } 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 c17da5825c74..5505f9a1b7c8 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 @@ -37,7 +37,7 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.data.RandomRowData; -import org.apache.iceberg.io.FileGroupWriter; +import org.apache.iceberg.io.TaskWriter; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.Assert; @@ -91,16 +91,16 @@ public void before() throws IOException { @Test public void testWriteZeroRecord() throws IOException { - try (FileGroupWriter fileGroupWriter = createTaskWriter(TARGET_FILE_SIZE)) { - fileGroupWriter.close(); + try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { + taskWriter.close(); - DataFile[] dataFiles = fileGroupWriter.complete().dataFiles(); + DataFile[] dataFiles = taskWriter.complete().dataFiles(); Assert.assertNotNull(dataFiles); Assert.assertEquals(0, dataFiles.length); // Close again. - fileGroupWriter.close(); - dataFiles = fileGroupWriter.complete().dataFiles(); + taskWriter.close(); + dataFiles = taskWriter.complete().dataFiles(); Assert.assertNotNull(dataFiles); Assert.assertEquals(0, dataFiles.length); } @@ -108,14 +108,14 @@ public void testWriteZeroRecord() throws IOException { @Test public void testCloseTwice() throws IOException { - try (FileGroupWriter fileGroupWriter = createTaskWriter(TARGET_FILE_SIZE)) { - fileGroupWriter.write(SimpleDataUtil.createRowData(1, "hello")); - fileGroupWriter.write(SimpleDataUtil.createRowData(2, "world")); - fileGroupWriter.close(); // The first close - fileGroupWriter.close(); // The second close + try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { + taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); + taskWriter.write(SimpleDataUtil.createRowData(2, "world")); + taskWriter.close(); // The first close + taskWriter.close(); // The second close int expectedFiles = partitioned ? 2 : 1; - DataFile[] dataFiles = fileGroupWriter.complete().dataFiles(); + DataFile[] dataFiles = taskWriter.complete().dataFiles(); Assert.assertEquals(expectedFiles, dataFiles.length); FileSystem fs = FileSystem.get(CONF); @@ -127,12 +127,12 @@ public void testCloseTwice() throws IOException { @Test public void testAbort() throws IOException { - try (FileGroupWriter fileGroupWriter = createTaskWriter(TARGET_FILE_SIZE)) { - fileGroupWriter.write(SimpleDataUtil.createRowData(1, "hello")); - fileGroupWriter.write(SimpleDataUtil.createRowData(2, "world")); + try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { + taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); + taskWriter.write(SimpleDataUtil.createRowData(2, "world")); - fileGroupWriter.abort(); - DataFile[] dataFiles = fileGroupWriter.complete().dataFiles(); + taskWriter.abort(); + DataFile[] dataFiles = taskWriter.complete().dataFiles(); int expectedFiles = partitioned ? 2 : 1; Assert.assertEquals(expectedFiles, dataFiles.length); @@ -146,17 +146,17 @@ public void testAbort() throws IOException { @Test public void testCompleteFiles() throws IOException { - try (FileGroupWriter fileGroupWriter = createTaskWriter(TARGET_FILE_SIZE)) { - fileGroupWriter.write(SimpleDataUtil.createRowData(1, "a")); - fileGroupWriter.write(SimpleDataUtil.createRowData(2, "b")); - fileGroupWriter.write(SimpleDataUtil.createRowData(3, "c")); - fileGroupWriter.write(SimpleDataUtil.createRowData(4, "d")); + try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { + taskWriter.write(SimpleDataUtil.createRowData(1, "a")); + taskWriter.write(SimpleDataUtil.createRowData(2, "b")); + taskWriter.write(SimpleDataUtil.createRowData(3, "c")); + taskWriter.write(SimpleDataUtil.createRowData(4, "d")); - DataFile[] dataFiles = fileGroupWriter.complete().dataFiles(); + DataFile[] dataFiles = taskWriter.complete().dataFiles(); int expectedFiles = partitioned ? 4 : 1; Assert.assertEquals(expectedFiles, dataFiles.length); - dataFiles = fileGroupWriter.complete().dataFiles(); + dataFiles = taskWriter.complete().dataFiles(); Assert.assertEquals(expectedFiles, dataFiles.length); FileSystem fs = FileSystem.get(CONF); @@ -186,7 +186,7 @@ public void testRollingWithTargetFileSize() throws IOException { if (format == FileFormat.ORC) { return; } - try (FileGroupWriter fileGroupWriter = createTaskWriter(4)) { + try (TaskWriter taskWriter = createTaskWriter(4)) { List rows = Lists.newArrayListWithCapacity(8000); List records = Lists.newArrayListWithCapacity(8000); for (int i = 0; i < 2000; i++) { @@ -197,10 +197,10 @@ public void testRollingWithTargetFileSize() throws IOException { } for (RowData row : rows) { - fileGroupWriter.write(row); + taskWriter.write(row); } - DataFile[] dataFiles = fileGroupWriter.complete().dataFiles(); + DataFile[] dataFiles = taskWriter.complete().dataFiles(); Assert.assertEquals(8, dataFiles.length); AppendFiles appendFiles = table.newAppend(); @@ -216,14 +216,14 @@ public void testRollingWithTargetFileSize() throws IOException { @Test public void testRandomData() throws IOException { - try (FileGroupWriter fileGroupWriter = createTaskWriter(TARGET_FILE_SIZE)) { + try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { Iterable rows = RandomRowData.generate(SimpleDataUtil.SCHEMA, 100, 1996); for (RowData row : rows) { - fileGroupWriter.write(row); + taskWriter.write(row); } - fileGroupWriter.close(); - DataFile[] dataFiles = fileGroupWriter.complete().dataFiles(); + taskWriter.close(); + DataFile[] dataFiles = taskWriter.complete().dataFiles(); AppendFiles appendFiles = table.newAppend(); for (DataFile dataFile : dataFiles) { appendFiles.appendFile(dataFile); @@ -235,7 +235,7 @@ public void testRandomData() throws IOException { } } - private FileGroupWriter createTaskWriter(long targetFileSize) { + 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(), 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 b7d56ccb8976..402d09823a8f 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDeleteWriters.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDeleteWriters.java @@ -139,7 +139,7 @@ public void testPositionDeleteWriter() throws IOException { } } - DeleteFile metadata = deleteWriter.toContentFile(); + DeleteFile metadata = deleteWriter.toDeleteFile(); 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.toContentFile(); + DeleteFile metadata = deleteWriter.toDeleteFile(); 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 d8050ceb356c..8d3e46f9622c 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 @@ -37,7 +37,7 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.io.OutputFileFactory; -import org.apache.iceberg.io.FileGroupWriter; +import org.apache.iceberg.io.TaskWriter; import org.apache.iceberg.io.UnpartitionedWriter; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.SparkSchemaUtil; @@ -103,7 +103,7 @@ private List rewriteDataForTask(CombinedScanTask task) throws Exceptio OutputFileFactory fileFactory = new OutputFileFactory( spec, format, locations, io.value(), encryptionManager.value(), partitionId, taskId); - FileGroupWriter writer; + TaskWriter writer; if (spec.fields().isEmpty()) { writer = new UnpartitionedWriter<>(format, fileFactory, io.value(), Long.MAX_VALUE, new DataFileWriterFactory<>(appenderFactory, spec)); From c0f498372c8b03ac9b0c252f20ecd92fb05dc404 Mon Sep 17 00:00:00 2001 From: huzheng Date: Thu, 5 Nov 2020 14:55:36 +0800 Subject: [PATCH 13/41] Revert "Add contentFiles in TaskWriterResult." This reverts commit 4650859a7628b5f2cc0851d5a5e7a297b9832632. --- .../org/apache/iceberg/io/BaseTaskWriter.java | 7 ++++- .../apache/iceberg/io/TaskWriterResult.java | 27 ------------------- 2 files changed, 6 insertions(+), 28 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java b/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java index 5aaa2923e263..ff385047754d 100644 --- a/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java @@ -55,7 +55,12 @@ public void abort() throws IOException { // clean up files created by this writer TaskWriterResult result = builder.build(); - Tasks.foreach(result.contentFiles()) + Tasks.foreach(result.dataFiles()) + .throwFailureWhenFinished() + .noRetry() + .run(file -> io.deleteFile(file.path().toString())); + + Tasks.foreach(result.deleteFiles()) .throwFailureWhenFinished() .noRetry() .run(file -> io.deleteFile(file.path().toString())); diff --git a/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java b/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java index 3afa23435428..6917e562c9b0 100644 --- a/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java +++ b/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java @@ -19,9 +19,7 @@ package org.apache.iceberg.io; -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; @@ -45,31 +43,6 @@ 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 Builder builder() { return new Builder(); } From ddb326a406ccd364e7bdf2cad4da776078da1ee4 Mon Sep 17 00:00:00 2001 From: huzheng Date: Thu, 5 Nov 2020 14:55:43 +0800 Subject: [PATCH 14/41] Revert "Minor fixes" This reverts commit 5ca464b6a836eee74536c490d8b4ae7533b8f109. --- .../org/apache/iceberg/io/BaseTaskWriter.java | 7 +- .../io/EqualityDeleteWriterFactory.java | 33 +++++++++ .../org/apache/iceberg/io/TaskWriter.java | 2 +- .../apache/iceberg/io/TaskWriterResult.java | 40 +++++++---- .../flink/sink/PartitionedFanoutWriter.java | 2 +- .../flink/sink/RowDataTaskWriterFactory.java | 72 +++++-------------- 6 files changed, 85 insertions(+), 71 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/io/EqualityDeleteWriterFactory.java diff --git a/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java b/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java index ff385047754d..6194d16b69c8 100644 --- a/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java @@ -86,6 +86,10 @@ public RollingFileWriter(PartitionKey partitionKey) { openCurrent(); } + protected PartitionKey partitionKey() { + return partitionKey; + } + public void add(T record) throws IOException { this.currentFileWriter.write(record); this.currentRows++; @@ -114,6 +118,7 @@ private boolean shouldRollToNewFile() { currentRows % ROWS_DIVISOR == 0 && currentFileWriter.length() >= targetFileSize; } + @SuppressWarnings("unchecked") private void closeCurrent() throws IOException { if (currentFileWriter != null) { currentFileWriter.close(); @@ -124,7 +129,7 @@ private void closeCurrent() throws IOException { if (metrics.recordCount() == 0L) { io.deleteFile(currentFile.encryptingOutputFile()); } else if (contentFile instanceof ContentFile) { - builder.add((ContentFile) contentFile); + builder.add((ContentFile) contentFile); } else { throw new RuntimeException(String.format( "The newly generated content file must be DataFile or DeleteFile: %s", contentFile)); diff --git a/core/src/main/java/org/apache/iceberg/io/EqualityDeleteWriterFactory.java b/core/src/main/java/org/apache/iceberg/io/EqualityDeleteWriterFactory.java new file mode 100644 index 000000000000..5c1b52d2bbfb --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/io/EqualityDeleteWriterFactory.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.FileFormat; +import org.apache.iceberg.deletes.EqualityDeleteWriter; + +/** + * Factory to create a new {@link EqualityDeleteWriter} to write equality deletions in row-level. + * + * @param data type of the rows to delete. + */ +public interface EqualityDeleteWriterFactory { + + EqualityDeleteWriter newEqualityDeleteWriter(OutputFile outputFile, FileFormat format); +} 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 36a699fe8803..8eb4b7f95e78 100644 --- a/core/src/main/java/org/apache/iceberg/io/TaskWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/TaskWriter.java @@ -42,7 +42,7 @@ public interface TaskWriter extends Closeable { void abort() throws IOException; /** - * Close the writer and get the completed data/delete files. + * Close the writer and get the completed data files. * * @return the completed data files of this task writer. */ diff --git a/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java b/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java index 6917e562c9b0..4c48e584b920 100644 --- a/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java +++ b/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java @@ -24,17 +24,29 @@ 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.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; public class TaskWriterResult { + private static final TaskWriterResult EMPTY = new TaskWriterResult(ImmutableList.of(), ImmutableList.of()); + private DataFile[] dataFiles; private DeleteFile[] deleteFiles; - private TaskWriterResult(List dataFiles, List deleteFiles) { + TaskWriterResult(DataFile[] dataFiles, DeleteFile[] deleteFiles) { + this.dataFiles = dataFiles; + this.deleteFiles = deleteFiles; + } + + TaskWriterResult(List dataFiles, List deleteFiles) { this.dataFiles = dataFiles.toArray(new DataFile[0]); this.deleteFiles = deleteFiles.toArray(new DeleteFile[0]); } + static TaskWriterResult empty() { + return EMPTY; + } + public DataFile[] dataFiles() { return dataFiles; } @@ -49,11 +61,21 @@ public static Builder builder() { public static TaskWriterResult concat(TaskWriterResult result0, TaskWriterResult result1) { Builder builder = new Builder(); + for (DataFile dataFile : result0.dataFiles) { + builder.add(dataFile); + } + + for (DataFile dataFile : result1.dataFiles) { + builder.add(dataFile); + } + + for (DeleteFile deleteFile : result0.deleteFiles) { + builder.add(deleteFile); + } - builder.addAll(result0.dataFiles); - builder.addAll(result0.deleteFiles); - builder.addAll(result1.dataFiles); - builder.addAll(result1.deleteFiles); + for (DeleteFile deleteFile : result1.deleteFiles) { + builder.add(deleteFile); + } return builder.build(); } @@ -67,12 +89,6 @@ private Builder() { this.deleteFiles = Lists.newArrayList(); } - public void addAll(ContentFile... files) { - for (ContentFile file : files) { - add(file); - } - } - public void add(ContentFile contentFile) { Preconditions.checkNotNull(contentFile, "Content file shouldn't be null."); switch (contentFile.content()) { @@ -86,7 +102,7 @@ public void add(ContentFile contentFile) { break; default: - throw new UnsupportedOperationException("Unknown file content: " + contentFile.content()); + throw new UnsupportedOperationException("Unknown file: " + contentFile.content()); } } 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 index 74c3bd570a91..ebdf4314b11c 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedFanoutWriter.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedFanoutWriter.java @@ -54,7 +54,7 @@ public void write(T row) throws IOException { 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); + writer = new RollingFileWriter(partitionKey); writers.put(copiedKey, writer); } 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 aee79a96695e..08d5006538fb 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 @@ -109,30 +109,21 @@ public TaskWriter create() { private abstract static class BaseMixedTaskWriter implements TaskWriter { - abstract boolean supportDeletion(); + protected abstract TaskWriter dataTaskWriter(); - abstract TaskWriter dataTaskWriter(); - - abstract TaskWriter deleteTaskWriter(); + protected abstract TaskWriter deleteTaskWriter(); @Override public void write(RowData row) throws IOException { switch (row.getRowKind()) { - case INSERT: case UPDATE_AFTER: dataTaskWriter().write(row); break; - case UPDATE_BEFORE: case DELETE: - if (supportDeletion()) { - deleteTaskWriter().write(row); - } else { - throw new UnsupportedOperationException("Couldn't accept deletion in task writer."); - } + deleteTaskWriter().write(row); break; - default: throw new UnsupportedOperationException("Unrecognized row kind: " + row.getRowKind()); } @@ -141,80 +132,57 @@ public void write(RowData row) throws IOException { @Override public void abort() throws IOException { dataTaskWriter().abort(); - if (supportDeletion()) { - deleteTaskWriter().abort(); - } + deleteTaskWriter().abort(); } @Override public TaskWriterResult complete() throws IOException { - if (supportDeletion()) { - return TaskWriterResult.concat(dataTaskWriter().complete(), deleteTaskWriter().complete()); - } else { - return dataTaskWriter().complete(); - } + return TaskWriterResult.concat(dataTaskWriter().complete(), deleteTaskWriter().complete()); } @Override public void close() throws IOException { dataTaskWriter().close(); - if (supportDeletion()) { - deleteTaskWriter().close(); - } + deleteTaskWriter().close(); } } private static class MixedUnpartitionedTaskWriter extends BaseMixedTaskWriter { - private final FileFormat format; private final TaskWriter dataTaskWriter; private final TaskWriter deleteTaskWriter; MixedUnpartitionedTaskWriter(FileFormat format, FileAppenderFactory appenderFactory, OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, RowType flinkSchema, Map tableProperties) { - this.format = format; - this.dataTaskWriter = new UnpartitionedWriter<>(format, fileFactory, io, targetFileSize, new DataFileWriterFactory<>(appenderFactory, PartitionSpec.unpartitioned())); // TODO: set the correct equality field ids. List equalityIds = ImmutableList.of(); - if (supportDeletion()) { - this.deleteTaskWriter = new UnpartitionedWriter<>(format, fileFactory, io, targetFileSize, - new FlinkEqualityDeleterFactory(schema, flinkSchema, PartitionSpec.unpartitioned(), equalityIds, - tableProperties)); - } else { - this.deleteTaskWriter = null; - } + this.deleteTaskWriter = new UnpartitionedWriter<>(format, fileFactory, io, targetFileSize, + new FlinkEqualityDeleterFactory(schema, flinkSchema, PartitionSpec.unpartitioned(), equalityIds, + tableProperties)); } @Override - boolean supportDeletion() { - return FileFormat.PARQUET.equals(format) || FileFormat.AVRO.equals(format); - } - - @Override - TaskWriter dataTaskWriter() { + protected TaskWriter dataTaskWriter() { return dataTaskWriter; } @Override - TaskWriter deleteTaskWriter() { + protected TaskWriter deleteTaskWriter() { return deleteTaskWriter; } } private static class MixedPartitionedTaskWriter extends BaseMixedTaskWriter { - private final FileFormat format; private final TaskWriter dataTaskWriter; private final TaskWriter deleteTaskWriter; MixedPartitionedTaskWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, RowType flinkSchema, Map tableProperties) { - this.format = format; - this.dataTaskWriter = new RowDataPartitionedFanoutWriter<>(spec, format, fileFactory, io, targetFileSize, schema, flinkSchema, new DataFileWriterFactory<>(appenderFactory, spec)); @@ -222,27 +190,19 @@ private static class MixedPartitionedTaskWriter extends BaseMixedTaskWriter { // TODO: set the correct equality field ids. List equalityIds = ImmutableList.of(); - if (supportDeletion()) { - this.deleteTaskWriter = - new RowDataPartitionedFanoutWriter<>(spec, format, fileFactory, io, targetFileSize, schema, - flinkSchema, new FlinkEqualityDeleterFactory(schema, flinkSchema, spec, equalityIds, tableProperties)); - } else { - this.deleteTaskWriter = null; - } - } + this.deleteTaskWriter = + new RowDataPartitionedFanoutWriter<>(spec, format, fileFactory, io, targetFileSize, schema, + flinkSchema, new FlinkEqualityDeleterFactory(schema, flinkSchema, spec, equalityIds, tableProperties)); - @Override - boolean supportDeletion() { - return FileFormat.PARQUET.equals(format) || FileFormat.AVRO.equals(format); } @Override - TaskWriter dataTaskWriter() { + protected TaskWriter dataTaskWriter() { return dataTaskWriter; } @Override - TaskWriter deleteTaskWriter() { + protected TaskWriter deleteTaskWriter() { return deleteTaskWriter; } } From f8c072c11a3b85a41114fa86847302c04a31f4da Mon Sep 17 00:00:00 2001 From: huzheng Date: Thu, 5 Nov 2020 14:55:50 +0800 Subject: [PATCH 15/41] Revert "Introduce the MixedTaskWriter to accept both INSERT and DELETE records." This reverts commit 5f56677186cfb24d18c9d188ed718a99257ef5bf. --- .../apache/iceberg/io/TaskWriterResult.java | 21 --- .../flink/sink/RowDataTaskWriterFactory.java | 126 ++---------------- 2 files changed, 10 insertions(+), 137 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java b/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java index 4c48e584b920..0200cd95a51c 100644 --- a/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java +++ b/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java @@ -59,27 +59,6 @@ public static Builder builder() { return new Builder(); } - public static TaskWriterResult concat(TaskWriterResult result0, TaskWriterResult result1) { - Builder builder = new Builder(); - for (DataFile dataFile : result0.dataFiles) { - builder.add(dataFile); - } - - for (DataFile dataFile : result1.dataFiles) { - builder.add(dataFile); - } - - for (DeleteFile deleteFile : result0.deleteFiles) { - builder.add(deleteFile); - } - - for (DeleteFile deleteFile : result1.deleteFiles) { - builder.add(deleteFile); - } - - return builder.build(); - } - public static class Builder { private final List dataFiles; private final List deleteFiles; 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 08d5006538fb..303742b4e86c 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 @@ -22,11 +22,10 @@ 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.ContentFileWriterFactory; +import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFileWriterFactory; import org.apache.iceberg.FileFormat; import org.apache.iceberg.MetricsConfig; @@ -46,12 +45,10 @@ import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.io.OutputFileFactory; import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.TaskWriterResult; 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; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; public class RowDataTaskWriterFactory implements TaskWriterFactory { private final Schema schema; @@ -62,7 +59,6 @@ public class RowDataTaskWriterFactory implements TaskWriterFactory { private final EncryptionManager encryptionManager; private final long targetFileSizeBytes; private final FileFormat format; - private final Map tableProperties; private final FileAppenderFactory appenderFactory; private transient OutputFileFactory outputFileFactory; @@ -84,7 +80,6 @@ public RowDataTaskWriterFactory(Schema schema, this.encryptionManager = encryptionManager; this.targetFileSizeBytes = targetFileSizeBytes; this.format = format; - this.tableProperties = tableProperties; this.appenderFactory = new FlinkFileAppenderFactory(schema, flinkSchema, tableProperties); } @@ -99,124 +94,23 @@ public TaskWriter create() { "The outputFileFactory shouldn't be null if we have invoked the initialize()."); if (spec.fields().isEmpty()) { - return new MixedUnpartitionedTaskWriter(format, appenderFactory, outputFileFactory, io, targetFileSizeBytes, - schema, flinkSchema, tableProperties); + return new UnpartitionedWriter<>(format, outputFileFactory, io, + targetFileSizeBytes, new DataFileWriterFactory<>(appenderFactory, spec)); } else { - return new MixedPartitionedTaskWriter(spec, format, appenderFactory, outputFileFactory, - io, targetFileSizeBytes, schema, flinkSchema, tableProperties); + return new RowDataPartitionedFanoutWriter(spec, format, appenderFactory, outputFileFactory, + io, targetFileSizeBytes, schema, flinkSchema); } } - private abstract static class BaseMixedTaskWriter implements TaskWriter { - - protected abstract TaskWriter dataTaskWriter(); - - protected abstract TaskWriter deleteTaskWriter(); - - @Override - public void write(RowData row) throws IOException { - switch (row.getRowKind()) { - case INSERT: - case UPDATE_AFTER: - dataTaskWriter().write(row); - break; - case UPDATE_BEFORE: - case DELETE: - deleteTaskWriter().write(row); - break; - default: - throw new UnsupportedOperationException("Unrecognized row kind: " + row.getRowKind()); - } - } - - @Override - public void abort() throws IOException { - dataTaskWriter().abort(); - deleteTaskWriter().abort(); - } - - @Override - public TaskWriterResult complete() throws IOException { - return TaskWriterResult.concat(dataTaskWriter().complete(), deleteTaskWriter().complete()); - } - - @Override - public void close() throws IOException { - dataTaskWriter().close(); - deleteTaskWriter().close(); - } - } - - private static class MixedUnpartitionedTaskWriter extends BaseMixedTaskWriter { - private final TaskWriter dataTaskWriter; - private final TaskWriter deleteTaskWriter; - - MixedUnpartitionedTaskWriter(FileFormat format, FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, - RowType flinkSchema, Map tableProperties) { - this.dataTaskWriter = new UnpartitionedWriter<>(format, fileFactory, io, - targetFileSize, new DataFileWriterFactory<>(appenderFactory, PartitionSpec.unpartitioned())); - - // TODO: set the correct equality field ids. - List equalityIds = ImmutableList.of(); - - this.deleteTaskWriter = new UnpartitionedWriter<>(format, fileFactory, io, targetFileSize, - new FlinkEqualityDeleterFactory(schema, flinkSchema, PartitionSpec.unpartitioned(), equalityIds, - tableProperties)); - } - - @Override - protected TaskWriter dataTaskWriter() { - return dataTaskWriter; - } - - @Override - protected TaskWriter deleteTaskWriter() { - return deleteTaskWriter; - } - } - - private static class MixedPartitionedTaskWriter extends BaseMixedTaskWriter { - private final TaskWriter dataTaskWriter; - private final TaskWriter deleteTaskWriter; - - MixedPartitionedTaskWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, - RowType flinkSchema, Map tableProperties) { - this.dataTaskWriter = - new RowDataPartitionedFanoutWriter<>(spec, format, fileFactory, io, targetFileSize, schema, - flinkSchema, new DataFileWriterFactory<>(appenderFactory, spec)); - - // TODO: set the correct equality field ids. - List equalityIds = ImmutableList.of(); - - this.deleteTaskWriter = - new RowDataPartitionedFanoutWriter<>(spec, format, fileFactory, io, targetFileSize, schema, - flinkSchema, new FlinkEqualityDeleterFactory(schema, flinkSchema, spec, equalityIds, tableProperties)); - - } - - @Override - protected TaskWriter dataTaskWriter() { - return dataTaskWriter; - } - - @Override - protected TaskWriter deleteTaskWriter() { - return deleteTaskWriter; - } - } - - private static class RowDataPartitionedFanoutWriter - extends PartitionedFanoutWriter { + private static class RowDataPartitionedFanoutWriter extends PartitionedFanoutWriter { private final PartitionKey partitionKey; private final RowDataWrapper rowDataWrapper; - RowDataPartitionedFanoutWriter(PartitionSpec spec, FileFormat format, OutputFileFactory fileFactory, FileIO io, - long targetFileSize, Schema schema, RowType flinkSchema, - ContentFileWriterFactory contentFileWriter) { - super(format, fileFactory, io, targetFileSize, contentFileWriter); + RowDataPartitionedFanoutWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema, + RowType flinkSchema) { + super(format, fileFactory, io, targetFileSize, new DataFileWriterFactory<>(appenderFactory, spec)); this.partitionKey = new PartitionKey(spec, schema); this.rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); } From 1cb68ad1252fa679317bb91465a3c476f480f505 Mon Sep 17 00:00:00 2001 From: huzheng Date: Thu, 5 Nov 2020 15:07:17 +0800 Subject: [PATCH 16/41] Make PositionDeleteWriter to be a ContentFileWriter. --- .../iceberg/deletes/PositionDeleteWriter.java | 23 ++++++++++++++++--- .../iceberg/avro/TestAvroDeleteWriters.java | 4 ++-- .../org/apache/iceberg/data/FileHelpers.java | 2 +- .../parquet/TestParquetDeleteWriters.java | 4 ++-- 4 files changed, 25 insertions(+), 8 deletions(-) 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..7727235d627e 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteWriter.java +++ b/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteWriter.java @@ -19,13 +19,14 @@ 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; +import org.apache.iceberg.Metrics; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.StructLike; import org.apache.iceberg.encryption.EncryptionKeyMetadata; @@ -33,7 +34,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 +86,23 @@ public Set referencedDataFiles() { return pathSet; } - public DeleteFile toDeleteFile() { + @Override + public void write(PositionDelete record) { + delete(record.path(), record.pos(), record.row()); + } + + @Override + public Metrics metrics() { + return appender.metrics(); + } + + @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/test/java/org/apache/iceberg/avro/TestAvroDeleteWriters.java b/core/src/test/java/org/apache/iceberg/avro/TestAvroDeleteWriters.java index 1f8bb66c0cdd..d6184ee05317 100644 --- a/core/src/test/java/org/apache/iceberg/avro/TestAvroDeleteWriters.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestAvroDeleteWriters.java @@ -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/test/java/org/apache/iceberg/data/FileHelpers.java b/data/src/test/java/org/apache/iceberg/data/FileHelpers.java index 0755ffb8ca18..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) 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 402d09823a8f..b7d56ccb8976 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDeleteWriters.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetDeleteWriters.java @@ -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()); From e41fb8edd6b7f074e761e7ee0d2e172b595ec6f5 Mon Sep 17 00:00:00 2001 From: huzheng Date: Thu, 5 Nov 2020 16:06:33 +0800 Subject: [PATCH 17/41] Introduce a separate RollingContentFileWriter to decouple with TaskWriter. --- .../org/apache/iceberg/io/BaseTaskWriter.java | 148 ------------------ .../apache/iceberg/io/PartitionedWriter.java | 48 +++++- .../iceberg/io/RollingContentFileWriter.java | 143 +++++++++++++++++ .../apache/iceberg/io/TaskWriterResult.java | 39 ++++- .../iceberg/io/UnpartitionedWriter.java | 18 ++- .../flink/sink/PartitionedFanoutWriter.java | 53 ++++++- 6 files changed, 285 insertions(+), 164 deletions(-) delete mode 100644 core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java create mode 100644 core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java 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 6194d16b69c8..000000000000 --- a/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java +++ /dev/null @@ -1,148 +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 org.apache.iceberg.ContentFile; -import org.apache.iceberg.ContentFileWriter; -import org.apache.iceberg.ContentFileWriterFactory; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Metrics; -import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.encryption.EncryptedOutputFile; -import org.apache.iceberg.util.Tasks; - -public abstract class BaseTaskWriter implements TaskWriter { - private final TaskWriterResult.Builder builder; - private final FileFormat format; - private final OutputFileFactory fileFactory; - private final FileIO io; - private final long targetFileSize; - private final ContentFileWriterFactory writerFactory; - - protected BaseTaskWriter(FileFormat format, OutputFileFactory fileFactory, FileIO io, long targetFileSize, - ContentFileWriterFactory writerFactory) { - this.builder = TaskWriterResult.builder(); - this.format = format; - this.fileFactory = fileFactory; - this.io = io; - this.targetFileSize = targetFileSize; - this.writerFactory = writerFactory; - } - - @Override - public void abort() throws IOException { - close(); - - // clean up files created by this writer - TaskWriterResult result = builder.build(); - - Tasks.foreach(result.dataFiles()) - .throwFailureWhenFinished() - .noRetry() - .run(file -> io.deleteFile(file.path().toString())); - - Tasks.foreach(result.deleteFiles()) - .throwFailureWhenFinished() - .noRetry() - .run(file -> io.deleteFile(file.path().toString())); - } - - @Override - public TaskWriterResult complete() throws IOException { - close(); - - return builder.build(); - } - - protected class RollingFileWriter implements Closeable { - private static final int ROWS_DIVISOR = 1000; - private final PartitionKey partitionKey; - - private EncryptedOutputFile currentFile = null; - private ContentFileWriter currentFileWriter = null; - private long currentRows = 0; - - public RollingFileWriter(PartitionKey partitionKey) { - this.partitionKey = partitionKey; - openCurrent(); - } - - protected PartitionKey partitionKey() { - return partitionKey; - } - - public void add(T record) throws IOException { - this.currentFileWriter.write(record); - this.currentRows++; - - if (shouldRollToNewFile()) { - closeCurrent(); - openCurrent(); - } - } - - 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; - } - - @SuppressWarnings("unchecked") - private void closeCurrent() throws IOException { - if (currentFileWriter != null) { - currentFileWriter.close(); - ContentFileT contentFile = currentFileWriter.toContentFile(); - Metrics metrics = currentFileWriter.metrics(); - this.currentFileWriter = null; - - if (metrics.recordCount() == 0L) { - io.deleteFile(currentFile.encryptingOutputFile()); - } else if (contentFile instanceof ContentFile) { - builder.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/PartitionedWriter.java b/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java index ef786121c4ae..f11bab4e1f9f 100644 --- a/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java @@ -26,20 +26,33 @@ import org.apache.iceberg.PartitionKey; 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 TaskWriterResult.Builder resultBuilder; + private final Set completedPartitions = Sets.newHashSet(); private PartitionKey currentKey = null; - private RollingFileWriter currentWriter = null; + private RollingContentFileWriter currentWriter = null; public PartitionedWriter(FileFormat format, OutputFileFactory fileFactory, FileIO io, long targetFileSize, ContentFileWriterFactory writerFactory) { - super(format, fileFactory, io, targetFileSize, writerFactory); + this.format = format; + this.fileFactory = fileFactory; + this.io = io; + this.targetFileSize = targetFileSize; + this.writerFactory = writerFactory; + this.resultBuilder = TaskWriterResult.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,7 +83,8 @@ 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); @@ -82,4 +96,28 @@ public void close() throws IOException { currentWriter.close(); } } + + @Override + public void abort() throws IOException { + if (currentWriter != null) { + // Called complete() rather abort() to get all the completed files. + resultBuilder.add(currentWriter.complete()); + currentWriter = null; + } + + Tasks.foreach(resultBuilder.build().contentFiles()) + .throwFailureWhenFinished() + .noRetry() + .run(file -> io.deleteFile(file.path().toString())); + } + + @Override + public TaskWriterResult complete() throws IOException { + 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..0281d9750362 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java @@ -0,0 +1,143 @@ +/* + * 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 org.apache.iceberg.ContentFile; +import org.apache.iceberg.ContentFileWriter; +import org.apache.iceberg.ContentFileWriterFactory; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Metrics; +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 TaskWriterResult.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 = TaskWriterResult.builder(); + + openCurrent(); + } + + public void add(T record) throws IOException { + this.currentFileWriter.write(record); + this.currentRows++; + + if (shouldRollToNewFile()) { + closeCurrent(); + openCurrent(); + } + } + + public void abort() throws IOException { + close(); + + TaskWriterResult result = resultBuilder.build(); + + Tasks.foreach(result.contentFiles()) + .throwFailureWhenFinished() + .noRetry() + .run(file -> io.deleteFile(file.path().toString())); + } + + public TaskWriterResult complete() throws IOException { + close(); + + 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; + } + + private void closeCurrent() throws IOException { + if (currentFileWriter != null) { + currentFileWriter.close(); + ContentFileT contentFile = currentFileWriter.toContentFile(); + Metrics metrics = currentFileWriter.metrics(); + this.currentFileWriter = null; + + if (metrics.recordCount() == 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/TaskWriterResult.java b/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java index 0200cd95a51c..1a131261d555 100644 --- a/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java +++ b/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java @@ -19,7 +19,9 @@ package org.apache.iceberg.io; +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; @@ -55,6 +57,31 @@ 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 Builder builder() { return new Builder(); } @@ -68,7 +95,17 @@ private Builder() { this.deleteFiles = Lists.newArrayList(); } - public void add(ContentFile contentFile) { + public void add(TaskWriterResult 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: 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 994fe259ef18..583742fa5b6f 100644 --- a/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java @@ -23,14 +23,14 @@ import org.apache.iceberg.ContentFileWriterFactory; import org.apache.iceberg.FileFormat; -public class UnpartitionedWriter extends BaseTaskWriter { +public class UnpartitionedWriter implements TaskWriter { - private final RollingFileWriter currentWriter; + private final RollingContentFileWriter currentWriter; public UnpartitionedWriter(FileFormat format, OutputFileFactory fileFactory, FileIO io, long targetFileSize, ContentFileWriterFactory writerFactory) { - super(format, fileFactory, io, targetFileSize, writerFactory); - currentWriter = new RollingFileWriter(null); + currentWriter = new RollingContentFileWriter<>(null, format, fileFactory, io, + targetFileSize, writerFactory); } @Override @@ -38,6 +38,16 @@ public void write(T record) throws IOException { currentWriter.add(record); } + @Override + public void abort() throws IOException { + currentWriter.abort(); + } + + @Override + public TaskWriterResult complete() throws IOException { + return currentWriter.complete(); + } + @Override public void close() throws IOException { currentWriter.close(); 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 index ebdf4314b11c..8e29719d8aef 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedFanoutWriter.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedFanoutWriter.java @@ -24,17 +24,34 @@ import org.apache.iceberg.ContentFileWriterFactory; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.io.BaseTaskWriter; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.io.RollingContentFileWriter; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.TaskWriterResult; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -abstract class PartitionedFanoutWriter extends BaseTaskWriter { - private final Map writers = Maps.newHashMap(); +abstract class PartitionedFanoutWriter implements TaskWriter { + + private static final Logger LOG = LoggerFactory.getLogger(PartitionedFanoutWriter.class); + + private final FileFormat format; + private final OutputFileFactory fileFactory; + private final FileIO io; + private final long targetFileSize; + private final ContentFileWriterFactory writerFactory; + + private final Map> writers = Maps.newHashMap(); PartitionedFanoutWriter(FileFormat format, OutputFileFactory fileFactory, FileIO io, long targetFileSize, ContentFileWriterFactory writerFactory) { - super(format, fileFactory, io, targetFileSize, writerFactory); + this.format = format; + this.fileFactory = fileFactory; + this.io = io; + this.targetFileSize = targetFileSize; + this.writerFactory = writerFactory; } /** @@ -50,11 +67,11 @@ abstract class PartitionedFanoutWriter extends BaseTaskWriter 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(partitionKey); + writer = new RollingContentFileWriter<>(copiedKey, format, fileFactory, io, targetFileSize, writerFactory); writers.put(copiedKey, writer); } @@ -70,4 +87,28 @@ public void close() throws IOException { writers.clear(); } } + + @Override + public void abort() { + for (RollingContentFileWriter writer : writers.values()) { + try { + writer.abort(); + } catch (IOException e) { + LOG.warn("Failed to abort the writer {} because: ", writer, e); + } + } + writers.clear(); + } + + @Override + public TaskWriterResult complete() throws IOException { + TaskWriterResult.Builder builder = TaskWriterResult.builder(); + + for (RollingContentFileWriter writer : writers.values()) { + builder.add(writer.complete()); + } + writers.clear(); + + return builder.build(); + } } From b8c47952361de3090c573a5dac5028beb04a963a Mon Sep 17 00:00:00 2001 From: huzheng Date: Thu, 5 Nov 2020 18:08:43 +0800 Subject: [PATCH 18/41] Introdue DeltaWriter to write insert and delete rows. --- .../apache/iceberg/io/BaseDeltaWriter.java | 230 ++++++++++++++++++ .../org/apache/iceberg/io/DeltaWriter.java | 62 +++++ .../apache/iceberg/io/PartitionedWriter.java | 8 +- .../iceberg/io/RollingContentFileWriter.java | 18 +- .../org/apache/iceberg/io/TaskWriter.java | 2 +- .../iceberg/io/UnpartitionedWriter.java | 4 +- ...askWriterResult.java => WriterResult.java} | 16 +- .../apache/iceberg/util/StructLikeMap.java | 127 ++++++++++ .../apache/iceberg/io/TestDeltaWriter.java | 30 +++ .../flink/sink/PartitionedFanoutWriter.java | 8 +- 10 files changed, 481 insertions(+), 24 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java create mode 100644 core/src/main/java/org/apache/iceberg/io/DeltaWriter.java rename core/src/main/java/org/apache/iceberg/io/{TaskWriterResult.java => WriterResult.java} (88%) create mode 100644 core/src/main/java/org/apache/iceberg/util/StructLikeMap.java create mode 100644 core/src/test/java/org/apache/iceberg/io/TestDeltaWriter.java 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..cd95159c68b2 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java @@ -0,0 +1,230 @@ +/* + * 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 java.util.function.Function; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +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.Sets; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.StructLikeMap; +import org.apache.iceberg.util.StructProjection; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public abstract class BaseDeltaWriter implements DeltaWriter { + private static final Logger LOG = LoggerFactory.getLogger(BaseDeltaWriter.class); + + private final RollingContentFileWriter dataWriter; + private final RollingContentFileWriter equalityDeleteWriter; + private final RollingContentFileWriter> posDeleteWriter; + + private final PositionDelete positionDelete = new PositionDelete<>(); + private final StructProjection projectionRow; + private final StructLikeMap insertedRowMap; + + // Function to convert the generic data to a StructLike. + private final Function structLikeFun; + + BaseDeltaWriter(RollingContentFileWriter dataWriter) { + this(dataWriter, null); + } + + BaseDeltaWriter(RollingContentFileWriter dataWriter, + RollingContentFileWriter> posDeleteWriter) { + this(dataWriter, posDeleteWriter, null, null, null, null); + } + + BaseDeltaWriter(RollingContentFileWriter dataWriter, + RollingContentFileWriter> posDeleteWriter, + RollingContentFileWriter equalityDeleteWriter, + Schema tableSchema, + List equalityFieldIds, + Function structLikeFun) { + + Preconditions.checkNotNull(dataWriter, "Data writer should always not be null."); + + if (posDeleteWriter == null) { + // Only accept INSERT records. + Preconditions.checkArgument(equalityDeleteWriter == null); + } + + if (posDeleteWriter != null && equalityDeleteWriter == null) { + // Only accept INSERT records and position deletion. + Preconditions.checkArgument(tableSchema == null); + Preconditions.checkArgument(equalityFieldIds == null); + } + + if (equalityDeleteWriter != null) { + // Accept insert records, position deletion, equality deletions. + 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"); + Preconditions.checkNotNull(structLikeFun, "StructLike function shouldn't be null"); + + Schema deleteSchema = TypeUtil.select(tableSchema, Sets.newHashSet(equalityFieldIds)); + this.projectionRow = StructProjection.create(tableSchema, deleteSchema); + this.insertedRowMap = StructLikeMap.create(deleteSchema.asStruct()); + this.structLikeFun = structLikeFun; + } else { + this.projectionRow = null; + this.insertedRowMap = null; + this.structLikeFun = null; + } + + this.dataWriter = dataWriter; + this.equalityDeleteWriter = equalityDeleteWriter; + this.posDeleteWriter = posDeleteWriter; + } + + @Override + public void writeRow(T row) throws IOException { + if (enableEqualityDelete()) { + FilePos filePos = FilePos.create(dataWriter.currentPath(), dataWriter.currentPos()); + insertedRowMap.put(projectionRow.wrap(structLikeFun.apply(row)), filePos); + } + + dataWriter.write(row); + } + + @Override + public void writeEqualityDelete(T equalityDelete) throws IOException { + if (!enableEqualityDelete()) { + throw new UnsupportedOperationException("Could not accept equality deletion."); + } + + FilePos existing = insertedRowMap.get(projectionRow.wrap(structLikeFun.apply(equalityDelete))); + + if (existing == null) { + // Delete the row which did not written by this delta writer. + equalityDeleteWriter.write(equalityDelete); + } else { + // Delete the row which was written in current delta writer. + posDeleteWriter.write(positionDelete.set(existing.path(), existing.pos(), equalityDelete)); + } + } + + @Override + public void writePosDelete(CharSequence path, long offset, T row) throws IOException { + if (!enablePosDelete()) { + throw new UnsupportedOperationException("Could not accept position deletion."); + } + + posDeleteWriter.write(positionDelete.set(path, offset, row)); + } + + @Override + public void abort() throws IOException { + if (dataWriter != null) { + try { + dataWriter.abort(); + } catch (IOException e) { + LOG.warn("Failed to abort the data writer {} because: ", dataWriter, e); + } + } + + if (equalityDeleteWriter != null) { + try { + equalityDeleteWriter.abort(); + } catch (IOException e) { + LOG.warn("Failed to abort the equality-delete writer {} because: ", equalityDeleteWriter, e); + } + } + + if (posDeleteWriter != null) { + try { + posDeleteWriter.abort(); + } catch (IOException e) { + LOG.warn("Failed to abort the pos-delete writer {} because: ", posDeleteWriter, e); + } + } + } + + @Override + public WriterResult complete() throws IOException { + WriterResult.Builder builder = WriterResult.builder(); + + if (dataWriter != null) { + builder.add(dataWriter.complete()); + } + + if (equalityDeleteWriter != null) { + builder.add(equalityDeleteWriter.complete()); + } + + if (posDeleteWriter != null) { + builder.add(posDeleteWriter.complete()); + } + + return builder.build(); + } + + @Override + public void close() throws IOException { + if (dataWriter != null) { + dataWriter.close(); + } + + if (equalityDeleteWriter != null) { + equalityDeleteWriter.close(); + } + + if (posDeleteWriter != null) { + posDeleteWriter.close(); + } + } + + private boolean enableEqualityDelete() { + return equalityDeleteWriter != null && posDeleteWriter != null; + } + + private boolean enablePosDelete() { + return posDeleteWriter != null; + } + + private static class FilePos { + private final CharSequence path; + private final long pos; + + private FilePos(CharSequence path, long pos) { + this.path = path; + this.pos = pos; + } + + private static FilePos create(CharSequence path, long pos) { + return new FilePos(path, pos); + } + + private CharSequence path() { + return path; + } + + private long pos() { + return pos; + } + } +} 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..d27d37bd6a8a --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/io/DeltaWriter.java @@ -0,0 +1,62 @@ +/* + * 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; + +/** + * 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) throws IOException; + + /** + * Write the equality delete record. + */ + void writeEqualityDelete(T equalityDelete) throws IOException; + + /** + * Write the deletion with file path and position into underlying system. + */ + default void writePosDelete(CharSequence path, long offset) throws IOException { + 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) throws IOException; + + /** + * Abort the writer to clean all generated files. + */ + void abort() throws IOException; + + /** + * Close the writer and get all the completed data files and delete files. + */ + WriterResult complete() throws IOException; +} 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 f11bab4e1f9f..91e59d7b9044 100644 --- a/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java @@ -38,7 +38,7 @@ public abstract class PartitionedWriter implements TaskWriter writerFactory; - private final TaskWriterResult.Builder resultBuilder; + private final WriterResult.Builder resultBuilder; private final Set completedPartitions = Sets.newHashSet(); @@ -52,7 +52,7 @@ public PartitionedWriter(FileFormat format, OutputFileFactory fileFactory, FileI this.io = io; this.targetFileSize = targetFileSize; this.writerFactory = writerFactory; - this.resultBuilder = TaskWriterResult.builder(); + this.resultBuilder = WriterResult.builder(); } /** @@ -87,7 +87,7 @@ public void write(T row) throws IOException { fileFactory, io, targetFileSize, writerFactory); } - currentWriter.add(row); + currentWriter.write(row); } @Override @@ -112,7 +112,7 @@ public void abort() throws IOException { } @Override - public TaskWriterResult complete() throws IOException { + public WriterResult complete() throws IOException { if (currentWriter != null) { resultBuilder.add(currentWriter.complete()); currentWriter = null; diff --git a/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java b/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java index 0281d9750362..8f04ef6758a4 100644 --- a/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java @@ -41,7 +41,7 @@ public class RollingContentFileWriter implements Closeable { private final long targetFileSize; private final ContentFileWriterFactory writerFactory; - private final TaskWriterResult.Builder resultBuilder; + private final WriterResult.Builder resultBuilder; private EncryptedOutputFile currentFile = null; private ContentFileWriter currentFileWriter = null; @@ -57,12 +57,20 @@ public RollingContentFileWriter(PartitionKey partitionKey, FileFormat format, this.targetFileSize = targetFileSize; this.writerFactory = writerFactory; - this.resultBuilder = TaskWriterResult.builder(); + this.resultBuilder = WriterResult.builder(); openCurrent(); } - public void add(T record) throws IOException { + public CharSequence currentPath() { + return currentFile.encryptingOutputFile().location(); + } + + public long currentPos() { + return currentRows; + } + + public void write(T record) throws IOException { this.currentFileWriter.write(record); this.currentRows++; @@ -75,7 +83,7 @@ public void add(T record) throws IOException { public void abort() throws IOException { close(); - TaskWriterResult result = resultBuilder.build(); + WriterResult result = resultBuilder.build(); Tasks.foreach(result.contentFiles()) .throwFailureWhenFinished() @@ -83,7 +91,7 @@ public void abort() throws IOException { .run(file -> io.deleteFile(file.path().toString())); } - public TaskWriterResult complete() throws IOException { + public WriterResult complete() throws IOException { close(); return resultBuilder.build(); 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 8eb4b7f95e78..98fd75e96cd2 100644 --- a/core/src/main/java/org/apache/iceberg/io/TaskWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/TaskWriter.java @@ -46,5 +46,5 @@ public interface TaskWriter extends Closeable { * * @return the completed data files of this task writer. */ - TaskWriterResult 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 583742fa5b6f..38651984d4c4 100644 --- a/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java @@ -35,7 +35,7 @@ public UnpartitionedWriter(FileFormat format, OutputFileFactory fileFactory, Fil @Override public void write(T record) throws IOException { - currentWriter.add(record); + currentWriter.write(record); } @Override @@ -44,7 +44,7 @@ public void abort() throws IOException { } @Override - public TaskWriterResult complete() throws IOException { + public WriterResult complete() throws IOException { return currentWriter.complete(); } diff --git a/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java b/core/src/main/java/org/apache/iceberg/io/WriterResult.java similarity index 88% rename from core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java rename to core/src/main/java/org/apache/iceberg/io/WriterResult.java index 1a131261d555..288cf8d51908 100644 --- a/core/src/main/java/org/apache/iceberg/io/TaskWriterResult.java +++ b/core/src/main/java/org/apache/iceberg/io/WriterResult.java @@ -29,23 +29,23 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -public class TaskWriterResult { - private static final TaskWriterResult EMPTY = new TaskWriterResult(ImmutableList.of(), ImmutableList.of()); +public class WriterResult { + private static final WriterResult EMPTY = new WriterResult(ImmutableList.of(), ImmutableList.of()); private DataFile[] dataFiles; private DeleteFile[] deleteFiles; - TaskWriterResult(DataFile[] dataFiles, DeleteFile[] deleteFiles) { + WriterResult(DataFile[] dataFiles, DeleteFile[] deleteFiles) { this.dataFiles = dataFiles; this.deleteFiles = deleteFiles; } - TaskWriterResult(List dataFiles, List deleteFiles) { + WriterResult(List dataFiles, List deleteFiles) { this.dataFiles = dataFiles.toArray(new DataFile[0]); this.deleteFiles = deleteFiles.toArray(new DeleteFile[0]); } - static TaskWriterResult empty() { + static WriterResult empty() { return EMPTY; } @@ -95,7 +95,7 @@ private Builder() { this.deleteFiles = Lists.newArrayList(); } - public void add(TaskWriterResult result) { + public void add(WriterResult result) { addAll(result.contentFiles()); } @@ -122,8 +122,8 @@ public void add(ContentFile contentFile) { } } - public TaskWriterResult build() { - return new TaskWriterResult(dataFiles, deleteFiles); + 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..f45b85a2a3c6 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/util/StructLikeMap.java @@ -0,0 +1,127 @@ +/* + * 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.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 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/test/java/org/apache/iceberg/io/TestDeltaWriter.java b/core/src/test/java/org/apache/iceberg/io/TestDeltaWriter.java new file mode 100644 index 000000000000..ec97c899a32b --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/io/TestDeltaWriter.java @@ -0,0 +1,30 @@ +/* + * 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.junit.Test; + +public class TestDeltaWriter { + + @Test + public void test() { + + } +} 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 index 8e29719d8aef..6fcc1e2c39d3 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedFanoutWriter.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedFanoutWriter.java @@ -28,7 +28,7 @@ import org.apache.iceberg.io.OutputFileFactory; import org.apache.iceberg.io.RollingContentFileWriter; import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.TaskWriterResult; +import org.apache.iceberg.io.WriterResult; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -75,7 +75,7 @@ public void write(T row) throws IOException { writers.put(copiedKey, writer); } - writer.add(row); + writer.write(row); } @Override @@ -101,8 +101,8 @@ public void abort() { } @Override - public TaskWriterResult complete() throws IOException { - TaskWriterResult.Builder builder = TaskWriterResult.builder(); + public WriterResult complete() throws IOException { + WriterResult.Builder builder = WriterResult.builder(); for (RollingContentFileWriter writer : writers.values()) { builder.add(writer.complete()); From 54673c123beabf6127e36a4f76529e177703db03 Mon Sep 17 00:00:00 2001 From: huzheng Date: Fri, 6 Nov 2020 14:25:41 +0800 Subject: [PATCH 19/41] Implement GenericDeltaWriter. --- .../apache/iceberg/io/BaseDeltaWriter.java | 45 ++-- .../{ => io}/DataFileWriterFactory.java | 14 +- .../apache/iceberg/io/DeltaWriterFactory.java | 94 ++++++++ .../io/EqualityDeleteWriterFactory.java | 33 --- .../apache/iceberg/util/StructLikeMap.java | 3 +- .../apache/iceberg/util/StructLikeSet.java | 5 +- .../org/apache/iceberg/TableTestBase.java | 2 +- .../org/apache/iceberg/TestOverwrite.java | 2 +- .../apache/iceberg/io/TestDeltaWriter.java | 30 --- .../data/GenericDeltaWriterFactory.java | 210 ++++++++++++++++++ .../iceberg/data/TestGenericDeltaWriter.java | 191 ++++++++++++++++ .../flink/sink/RowDataTaskWriterFactory.java | 2 +- .../iceberg/spark/source/RowDataRewriter.java | 2 +- .../spark/source/SparkPartitionedWriter.java | 2 +- .../apache/iceberg/spark/source/Writer.java | 2 +- .../iceberg/spark/source/SparkWrite.java | 2 +- 16 files changed, 549 insertions(+), 90 deletions(-) rename core/src/main/java/org/apache/iceberg/{ => io}/DataFileWriterFactory.java (79%) create mode 100644 core/src/main/java/org/apache/iceberg/io/DeltaWriterFactory.java delete mode 100644 core/src/main/java/org/apache/iceberg/io/EqualityDeleteWriterFactory.java delete mode 100644 core/src/test/java/org/apache/iceberg/io/TestDeltaWriter.java create mode 100644 data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java create mode 100644 data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java diff --git a/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java b/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java index cd95159c68b2..899f43067c50 100644 --- a/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java @@ -27,6 +27,7 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.StructLike; import org.apache.iceberg.deletes.PositionDelete; +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; @@ -35,7 +36,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public abstract class BaseDeltaWriter implements DeltaWriter { +public class BaseDeltaWriter implements DeltaWriter { private static final Logger LOG = LoggerFactory.getLogger(BaseDeltaWriter.class); private final RollingContentFileWriter dataWriter; @@ -49,21 +50,21 @@ public abstract class BaseDeltaWriter implements DeltaWriter { // Function to convert the generic data to a StructLike. private final Function structLikeFun; - BaseDeltaWriter(RollingContentFileWriter dataWriter) { + public BaseDeltaWriter(RollingContentFileWriter dataWriter) { this(dataWriter, null); } - BaseDeltaWriter(RollingContentFileWriter dataWriter, - RollingContentFileWriter> posDeleteWriter) { + public BaseDeltaWriter(RollingContentFileWriter dataWriter, + RollingContentFileWriter> posDeleteWriter) { this(dataWriter, posDeleteWriter, null, null, null, null); } - BaseDeltaWriter(RollingContentFileWriter dataWriter, - RollingContentFileWriter> posDeleteWriter, - RollingContentFileWriter equalityDeleteWriter, - Schema tableSchema, - List equalityFieldIds, - Function structLikeFun) { + public BaseDeltaWriter(RollingContentFileWriter dataWriter, + RollingContentFileWriter> posDeleteWriter, + RollingContentFileWriter equalityDeleteWriter, + Schema tableSchema, + List equalityFieldIds, + Function structLikeFun) { Preconditions.checkNotNull(dataWriter, "Data writer should always not be null."); @@ -105,7 +106,10 @@ public abstract class BaseDeltaWriter implements DeltaWriter { public void writeRow(T row) throws IOException { if (enableEqualityDelete()) { FilePos filePos = FilePos.create(dataWriter.currentPath(), dataWriter.currentPos()); - insertedRowMap.put(projectionRow.wrap(structLikeFun.apply(row)), filePos); + + LOG.info("writeRow: {} -- filePos: {}", row, filePos); + + insertedRowMap.put(structLikeFun.apply(row), filePos); } dataWriter.write(row); @@ -117,10 +121,12 @@ public void writeEqualityDelete(T equalityDelete) throws IOException { throw new UnsupportedOperationException("Could not accept equality deletion."); } - FilePos existing = insertedRowMap.get(projectionRow.wrap(structLikeFun.apply(equalityDelete))); + FilePos existing = insertedRowMap.get(structLikeFun.apply(equalityDelete)); + + LOG.info("writeEqualityDelete: {}, existing: {}", equalityDelete, existing); if (existing == null) { - // Delete the row which did not written by this delta writer. + // Delete the row which have been written by other completed delta writer. equalityDeleteWriter.write(equalityDelete); } else { // Delete the row which was written in current delta writer. @@ -138,7 +144,7 @@ public void writePosDelete(CharSequence path, long offset, T row) throws IOExcep } @Override - public void abort() throws IOException { + public void abort() { if (dataWriter != null) { try { dataWriter.abort(); @@ -153,6 +159,7 @@ public void abort() throws IOException { } catch (IOException e) { LOG.warn("Failed to abort the equality-delete writer {} because: ", equalityDeleteWriter, e); } + insertedRowMap.clear(); } if (posDeleteWriter != null) { @@ -174,6 +181,7 @@ public WriterResult complete() throws IOException { if (equalityDeleteWriter != null) { builder.add(equalityDeleteWriter.complete()); + insertedRowMap.clear(); } if (posDeleteWriter != null) { @@ -191,6 +199,7 @@ public void close() throws IOException { if (equalityDeleteWriter != null) { equalityDeleteWriter.close(); + insertedRowMap.clear(); } if (posDeleteWriter != null) { @@ -226,5 +235,13 @@ private CharSequence path() { private long pos() { return pos; } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("path", path) + .add("pos", pos) + .toString(); + } } } diff --git a/core/src/main/java/org/apache/iceberg/DataFileWriterFactory.java b/core/src/main/java/org/apache/iceberg/io/DataFileWriterFactory.java similarity index 79% rename from core/src/main/java/org/apache/iceberg/DataFileWriterFactory.java rename to core/src/main/java/org/apache/iceberg/io/DataFileWriterFactory.java index 366970593823..a57bf477e15d 100644 --- a/core/src/main/java/org/apache/iceberg/DataFileWriterFactory.java +++ b/core/src/main/java/org/apache/iceberg/io/DataFileWriterFactory.java @@ -17,12 +17,20 @@ * under the License. */ -package org.apache.iceberg; +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; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.io.FileAppenderFactory; +/** + * TODO: This factory would be removed once we've replaced by using {@link DeltaWriterFactory} in the upper layer. + */ public class DataFileWriterFactory implements ContentFileWriterFactory { private final FileAppenderFactory appenderFactory; private final PartitionSpec spec; 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..54e3687ba6a4 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/io/DeltaWriterFactory.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.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.deletes.PositionDelete; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; + +public interface DeltaWriterFactory { + + DeltaWriter createDeltaWriter(PartitionKey partitionKey, Context context); + + FileAppenderFactory createFileAppenderFactory(); + + ContentFileWriterFactory createDataFileWriterFactory(); + + ContentFileWriterFactory createEqualityDeleteWriterFactory(List equalityFieldIds); + + ContentFileWriterFactory> createPosDeleteWriterFactory(); + + class Context { + private final boolean allowPosDelete; + private final boolean allowEqualityDelete; + private final List equalityFieldIds; + + private Context(boolean allowPosDelete, boolean allowEqualityDelete, List equalityFieldIds) { + this.allowPosDelete = allowPosDelete; + this.allowEqualityDelete = allowEqualityDelete; + this.equalityFieldIds = equalityFieldIds; + } + + public boolean allowPosDelete() { + return allowPosDelete; + } + + public boolean allowEqualityDelete() { + return allowEqualityDelete; + } + + public List equalityFieldIds() { + return equalityFieldIds; + } + + public static Builder builder() { + return new Builder(); + } + + public static class Builder { + private boolean allowPosDelete = false; + private boolean allowEqualityDelete = false; + private List equalityFieldIds = ImmutableList.of(); + + 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 Context build() { + return new Context(allowPosDelete, allowEqualityDelete, equalityFieldIds); + } + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/io/EqualityDeleteWriterFactory.java b/core/src/main/java/org/apache/iceberg/io/EqualityDeleteWriterFactory.java deleted file mode 100644 index 5c1b52d2bbfb..000000000000 --- a/core/src/main/java/org/apache/iceberg/io/EqualityDeleteWriterFactory.java +++ /dev/null @@ -1,33 +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 org.apache.iceberg.FileFormat; -import org.apache.iceberg.deletes.EqualityDeleteWriter; - -/** - * Factory to create a new {@link EqualityDeleteWriter} to write equality deletions in row-level. - * - * @param data type of the rows to delete. - */ -public interface EqualityDeleteWriterFactory { - - EqualityDeleteWriter newEqualityDeleteWriter(OutputFile outputFile, FileFormat format); -} diff --git a/core/src/main/java/org/apache/iceberg/util/StructLikeMap.java b/core/src/main/java/org/apache/iceberg/util/StructLikeMap.java index f45b85a2a3c6..5b045e66d0f6 100644 --- a/core/src/main/java/org/apache/iceberg/util/StructLikeMap.java +++ b/core/src/main/java/org/apache/iceberg/util/StructLikeMap.java @@ -19,6 +19,7 @@ package org.apache.iceberg.util; +import java.util.AbstractMap; import java.util.Collection; import java.util.Map; import java.util.Set; @@ -27,7 +28,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; -public class StructLikeMap implements Map { +public class StructLikeMap extends AbstractMap implements Map { public static StructLikeMap create(Types.StructType type) { return new StructLikeMap<>(type); 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/TestOverwrite.java b/core/src/test/java/org/apache/iceberg/TestOverwrite.java index 6512ea114f23..8c8cca1930c0 100644 --- a/core/src/test/java/org/apache/iceberg/TestOverwrite.java +++ b/core/src/test/java/org/apache/iceberg/TestOverwrite.java @@ -94,7 +94,7 @@ public class TestOverwrite extends TableTestBase { @Parameterized.Parameters(name = "formatVersion = {0}") public static Object[] parameters() { - return new Object[] { 1, 2 }; + return new Object[] {1, 2}; } public TestOverwrite(int formatVersion) { diff --git a/core/src/test/java/org/apache/iceberg/io/TestDeltaWriter.java b/core/src/test/java/org/apache/iceberg/io/TestDeltaWriter.java deleted file mode 100644 index ec97c899a32b..000000000000 --- a/core/src/test/java/org/apache/iceberg/io/TestDeltaWriter.java +++ /dev/null @@ -1,30 +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 org.junit.Test; - -public class TestDeltaWriter { - - @Test - public void test() { - - } -} 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..0cfd3014e4ac --- /dev/null +++ b/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java @@ -0,0 +1,210 @@ +/* + * 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.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.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()) { + return new BaseDeltaWriter<>(dataWriter); + } + + RollingContentFileWriter> posDeleteWriter = + new RollingContentFileWriter<>(partitionKey, + format, fileFactory, io, targetFileSize, createPosDeleteWriterFactory()); + + if (ctxt.allowPosDelete() && !ctxt.allowEqualityDelete()) { + return new BaseDeltaWriter<>(dataWriter, posDeleteWriter); + } + + 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."); + + RollingContentFileWriter eqDeleteWriter = new RollingContentFileWriter<>(partitionKey, + format, fileFactory, io, targetFileSize, createEqualityDeleteWriterFactory(ctxt.equalityFieldIds())); + + + return new BaseDeltaWriter<>(dataWriter, posDeleteWriter, eqDeleteWriter, schema, ctxt.equalityFieldIds(), + t -> t); + } + + @Override + public FileAppenderFactory createFileAppenderFactory() { + return new GenericAppenderFactory(schema); + } + + @Override + public ContentFileWriterFactory createDataFileWriterFactory() { + return (partitionKey, outputFile, fileFormat) -> { + FileAppender appender = appenderFactory.newAppender(outputFile.encryptingOutputFile(), format); + + return new DataFileWriter<>(appender, + format, + outputFile.encryptingOutputFile().location(), + partitionKey, + spec, outputFile.keyMetadata()); + }; + } + + @Override + public ContentFileWriterFactory createEqualityDeleteWriterFactory( + List equalityFieldIds) { + 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(schema) + .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(schema) + .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() { + 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(schema) // TODO: it should be optional. + .withSpec(spec) + .withKeyMetadata(outputFile.keyMetadata()) + .buildPositionWriter(); + + case PARQUET: + return Parquet.writeDeletes(outputFile.encryptingOutputFile()) + .createWriterFunc(GenericParquetWriter::buildWriter) + .withPartition(partitionKey) + .overwrite() + .setAll(tableProperties) + .metricsConfig(metricsConfig) + .rowSchema(schema) + .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); + } + }; + } +} 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..e18796b694ed --- /dev/null +++ b/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java @@ -0,0 +1,191 @@ +/* + * 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.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.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 String TABLE_NAME = "delta_table"; + + @Rule + public final TemporaryFolder tempFolder = new TemporaryFolder(); + + @Parameterized.Parameters(name = "formatVersion = {0}, format = {1}, partitioned = {2}") + public static Object[][] parameters() { + return new Object[][] { + {2, "avro", false}, + {2, "parquet", false}, + }; + } + + private final FileFormat format; + private final boolean partitioned; + private Table table; + + public TestGenericDeltaWriter(int formatVersion, String format, boolean partitioned) { + super(formatVersion); + 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() throws IOException { + 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); + for (Record record : expected) { + 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(expected), Sets.newHashSet(IcebergGenerics.read(table).build())); + } + + @Test + public void testWriteEqualityDelete() throws IOException { + DeltaWriterFactory writerFactory = createDeltaWriterFactory(); + + DeltaWriterFactory.Context ctxt = DeltaWriterFactory.Context.builder() + .allowEqualityDelete(true) + .equalityFieldIds(ImmutableList.of(3)) /* Field id for 'id' is 3 */ + .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()); + } + + 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/sink/RowDataTaskWriterFactory.java b/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java index 303742b4e86c..b2e3cf51a80a 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 @@ -26,7 +26,6 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFileWriterFactory; import org.apache.iceberg.FileFormat; import org.apache.iceberg.MetricsConfig; import org.apache.iceberg.PartitionKey; @@ -38,6 +37,7 @@ 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.DataFileWriterFactory; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.FileAppenderFactory; import org.apache.iceberg.io.FileIO; 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 8d3e46f9622c..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 @@ -27,13 +27,13 @@ import java.util.stream.Collectors; import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFileWriterFactory; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; 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; 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 7e23bd4bd37e..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 @@ -20,11 +20,11 @@ package org.apache.iceberg.spark.source; import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFileWriterFactory; 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; 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 41c08fddabf7..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 @@ -26,7 +26,6 @@ import java.util.Optional; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFileWriterFactory; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.ReplacePartitions; @@ -36,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; 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 7c4b127e4c16..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 @@ -26,7 +26,6 @@ import java.util.Optional; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFileWriterFactory; import org.apache.iceberg.FileFormat; import org.apache.iceberg.OverwriteFiles; import org.apache.iceberg.PartitionSpec; @@ -40,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; From 360bc4b190e81b9f561af4c21045b4de98b3738a Mon Sep 17 00:00:00 2001 From: huzheng Date: Fri, 6 Nov 2020 18:29:24 +0800 Subject: [PATCH 20/41] Fix the broken unit tests. --- .../java/org/apache/iceberg/data/TestGenericDeltaWriter.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java b/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java index e18796b694ed..5c682ecaaab3 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java +++ b/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java @@ -117,9 +117,10 @@ public void testWritePureInsert() throws IOException { public void testWriteEqualityDelete() throws IOException { DeltaWriterFactory writerFactory = createDeltaWriterFactory(); + List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); DeltaWriterFactory.Context ctxt = DeltaWriterFactory.Context.builder() .allowEqualityDelete(true) - .equalityFieldIds(ImmutableList.of(3)) /* Field id for 'id' is 3 */ + .equalityFieldIds(equalityFieldIds) .build(); // TODO More unit tests to test the partitioned case. From af757855cfeb8651d7881a419219cc22aee78bbc Mon Sep 17 00:00:00 2001 From: huzheng Date: Fri, 6 Nov 2020 19:17:58 +0800 Subject: [PATCH 21/41] Rebase to master. --- .../java/org/apache/iceberg/flink/source/RowDataRewriter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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..3c8d62146148 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 @@ -131,7 +131,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); From b83ac21cd94e8fd2b6a00ed0f794b25e6b2b88d6 Mon Sep 17 00:00:00 2001 From: huzheng Date: Fri, 6 Nov 2020 19:54:49 +0800 Subject: [PATCH 22/41] Add more unit tests for GenericDeltaWriter. --- .../apache/iceberg/io/BaseDeltaWriter.java | 29 ++++----- .../apache/iceberg/io/DeltaWriterFactory.java | 21 ++++++- .../data/GenericDeltaWriterFactory.java | 13 ++-- .../iceberg/data/TestGenericDeltaWriter.java | 59 +++++++++++++++++++ 4 files changed, 99 insertions(+), 23 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java b/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java index 899f43067c50..9b596502ca29 100644 --- a/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java @@ -29,10 +29,10 @@ import org.apache.iceberg.deletes.PositionDelete; 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.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.StructLikeMap; -import org.apache.iceberg.util.StructProjection; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,8 +44,7 @@ public class BaseDeltaWriter implements DeltaWriter { private final RollingContentFileWriter> posDeleteWriter; private final PositionDelete positionDelete = new PositionDelete<>(); - private final StructProjection projectionRow; - private final StructLikeMap insertedRowMap; + private final StructLikeMap> insertedRowMap; // Function to convert the generic data to a StructLike. private final Function structLikeFun; @@ -88,11 +87,9 @@ public BaseDeltaWriter(RollingContentFileWriter dataWriter, Preconditions.checkNotNull(structLikeFun, "StructLike function shouldn't be null"); Schema deleteSchema = TypeUtil.select(tableSchema, Sets.newHashSet(equalityFieldIds)); - this.projectionRow = StructProjection.create(tableSchema, deleteSchema); this.insertedRowMap = StructLikeMap.create(deleteSchema.asStruct()); this.structLikeFun = structLikeFun; } else { - this.projectionRow = null; this.insertedRowMap = null; this.structLikeFun = null; } @@ -106,10 +103,14 @@ public BaseDeltaWriter(RollingContentFileWriter dataWriter, public void writeRow(T row) throws IOException { if (enableEqualityDelete()) { FilePos filePos = FilePos.create(dataWriter.currentPath(), dataWriter.currentPos()); - - LOG.info("writeRow: {} -- filePos: {}", row, filePos); - - insertedRowMap.put(structLikeFun.apply(row), filePos); + insertedRowMap.compute(structLikeFun.apply(row), (k, v) -> { + if (v == null) { + return Lists.newArrayList(filePos); + } else { + v.add(filePos); + return v; + } + }); } dataWriter.write(row); @@ -121,16 +122,16 @@ public void writeEqualityDelete(T equalityDelete) throws IOException { throw new UnsupportedOperationException("Could not accept equality deletion."); } - FilePos existing = insertedRowMap.get(structLikeFun.apply(equalityDelete)); - - LOG.info("writeEqualityDelete: {}, existing: {}", equalityDelete, existing); + List existing = insertedRowMap.get(structLikeFun.apply(equalityDelete)); if (existing == null) { // Delete the row which have been written by other completed delta writer. equalityDeleteWriter.write(equalityDelete); } else { - // Delete the row which was written in current delta writer. - posDeleteWriter.write(positionDelete.set(existing.path(), existing.pos(), equalityDelete)); + // Delete the rows which was written in current delta writer. + for (FilePos filePos : existing) { + posDeleteWriter.write(positionDelete.set(filePos.path(), filePos.pos(), null)); + } } } diff --git a/core/src/main/java/org/apache/iceberg/io/DeltaWriterFactory.java b/core/src/main/java/org/apache/iceberg/io/DeltaWriterFactory.java index 54e3687ba6a4..dab732232c56 100644 --- a/core/src/main/java/org/apache/iceberg/io/DeltaWriterFactory.java +++ b/core/src/main/java/org/apache/iceberg/io/DeltaWriterFactory.java @@ -24,6 +24,7 @@ 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.collect.ImmutableList; @@ -35,7 +36,8 @@ public interface DeltaWriterFactory { ContentFileWriterFactory createDataFileWriterFactory(); - ContentFileWriterFactory createEqualityDeleteWriterFactory(List equalityFieldIds); + ContentFileWriterFactory createEqualityDeleteWriterFactory(List equalityFieldIds, + Schema rowSchema); ContentFileWriterFactory> createPosDeleteWriterFactory(); @@ -43,11 +45,14 @@ class Context { private final boolean allowPosDelete; private final boolean allowEqualityDelete; private final List equalityFieldIds; + private final Schema rowSchema; - private Context(boolean allowPosDelete, boolean allowEqualityDelete, List equalityFieldIds) { + private Context(boolean allowPosDelete, boolean allowEqualityDelete, List equalityFieldIds, + Schema rowSchema) { this.allowPosDelete = allowPosDelete; this.allowEqualityDelete = allowEqualityDelete; this.equalityFieldIds = equalityFieldIds; + this.rowSchema = rowSchema; } public boolean allowPosDelete() { @@ -62,6 +67,10 @@ public List equalityFieldIds() { return equalityFieldIds; } + public Schema rowSchema() { + return rowSchema; + } + public static Builder builder() { return new Builder(); } @@ -70,6 +79,7 @@ public static class Builder { private boolean allowPosDelete = false; private boolean allowEqualityDelete = false; private List equalityFieldIds = ImmutableList.of(); + private Schema rowSchema; public Builder allowPosDelete(boolean enable) { this.allowPosDelete = enable; @@ -86,8 +96,13 @@ public Builder equalityFieldIds(List newEqualityFieldIds) { return this; } + public Builder rowSchema(Schema newRowSchema) { + this.rowSchema = newRowSchema; + return this; + } + public Context build() { - return new Context(allowPosDelete, allowEqualityDelete, equalityFieldIds); + return new Context(allowPosDelete, allowEqualityDelete, equalityFieldIds, rowSchema); } } } diff --git a/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java b/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java index 0cfd3014e4ac..ab7ae612d157 100644 --- a/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java +++ b/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java @@ -97,7 +97,8 @@ public DeltaWriter createDeltaWriter(PartitionKey partitionKey, Context "Equality field id list shouldn't be null or emtpy."); RollingContentFileWriter eqDeleteWriter = new RollingContentFileWriter<>(partitionKey, - format, fileFactory, io, targetFileSize, createEqualityDeleteWriterFactory(ctxt.equalityFieldIds())); + format, fileFactory, io, targetFileSize, + createEqualityDeleteWriterFactory(ctxt.equalityFieldIds(), ctxt.rowSchema())); return new BaseDeltaWriter<>(dataWriter, posDeleteWriter, eqDeleteWriter, schema, ctxt.equalityFieldIds(), @@ -124,7 +125,7 @@ public ContentFileWriterFactory createDataFileWriterFactory() @Override public ContentFileWriterFactory createEqualityDeleteWriterFactory( - List equalityFieldIds) { + List equalityFieldIds, Schema rowSchema) { return (partitionKey, outputFile, fileFormat) -> { MetricsConfig metricsConfig = MetricsConfig.fromProperties(tableProperties); @@ -136,7 +137,7 @@ public ContentFileWriterFactory createEqualityDeleteWriterFa .withPartition(partitionKey) .overwrite() .setAll(tableProperties) - .rowSchema(schema) + .rowSchema(rowSchema) .withSpec(spec) .withKeyMetadata(outputFile.keyMetadata()) .equalityFieldIds(equalityFieldIds) @@ -149,7 +150,7 @@ public ContentFileWriterFactory createEqualityDeleteWriterFa .overwrite() .setAll(tableProperties) .metricsConfig(metricsConfig) - .rowSchema(schema) + .rowSchema(rowSchema) .withSpec(spec) .withKeyMetadata(outputFile.keyMetadata()) .equalityFieldIds(equalityFieldIds) @@ -179,7 +180,7 @@ public ContentFileWriterFactory> createPosDel .withPartition(partitionKey) .overwrite() .setAll(tableProperties) - .rowSchema(schema) // TODO: it should be optional. + .rowSchema(schema) // it's a nullable field. .withSpec(spec) .withKeyMetadata(outputFile.keyMetadata()) .buildPositionWriter(); @@ -191,7 +192,7 @@ public ContentFileWriterFactory> createPosDel .overwrite() .setAll(tableProperties) .metricsConfig(metricsConfig) - .rowSchema(schema) + .rowSchema(schema) // it's a nullable field. .withSpec(spec) .withKeyMetadata(outputFile.keyMetadata()) .buildPositionWriter(); diff --git a/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java b/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java index 5c682ecaaab3..13e556686425 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java +++ b/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java @@ -157,6 +157,65 @@ public void testWriteEqualityDelete() throws IOException { assertTableRecords(ImmutableSet.of()); } + @Test + public void testUpsertSameRow() throws IOException { + DeltaWriterFactory writerFactory = createDeltaWriterFactory(); + List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); + DeltaWriterFactory.Context ctxt = DeltaWriterFactory.Context.builder() + .allowEqualityDelete(true) + .equalityFieldIds(equalityFieldIds) + .rowSchema(SCHEMA) + .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"); + 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"); + + deltaWriter.writeRow(record1); + deltaWriter.writeRow(record2); + + // Commit the transaction. + WriterResult result = deltaWriter.complete(); + Assert.assertEquals(result.dataFiles().length, 1); + Assert.assertEquals(result.deleteFiles().length, 0); + commitTransaction(result); + + assertTableRecords(ImmutableSet.of(record1, record2)); + + deltaWriter = writerFactory.createDeltaWriter(null, ctxt); + + // UPSERT (1, "ccc") + deltaWriter.writeEqualityDelete(record3); + deltaWriter.writeRow(record3); + + // INSERT (1, "ddd") + // INSERT (1, "eee") + deltaWriter.writeRow(record4); + deltaWriter.writeRow(record5); + + // UPSERT (1, "fff") + deltaWriter.writeEqualityDelete(record6); + deltaWriter.writeRow(record6); + + // INSERT (1, "ggg") + deltaWriter.writeRow(record7); + + // Commit the transaction. + result = deltaWriter.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, record7)); + } + private void assertTableRecords(Set expectedRecords) { StructLikeSet expectedSet = StructLikeSet.create(SCHEMA.asStruct()); expectedSet.addAll(expectedRecords); From a374fd67e742466b4490c0a0004dc9c1773286ba Mon Sep 17 00:00:00 2001 From: huzheng Date: Mon, 9 Nov 2020 09:37:04 +0800 Subject: [PATCH 23/41] Fix the broken unit tests. --- .../org/apache/iceberg/io/DeltaWriterFactory.java | 11 +++++++++++ .../apache/iceberg/data/TestGenericDeltaWriter.java | 3 ++- .../main/java/org/apache/iceberg/parquet/Parquet.java | 2 +- 3 files changed, 14 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/io/DeltaWriterFactory.java b/core/src/main/java/org/apache/iceberg/io/DeltaWriterFactory.java index dab732232c56..7b485df7092d 100644 --- a/core/src/main/java/org/apache/iceberg/io/DeltaWriterFactory.java +++ b/core/src/main/java/org/apache/iceberg/io/DeltaWriterFactory.java @@ -26,6 +26,7 @@ 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; public interface DeltaWriterFactory { @@ -102,6 +103,16 @@ public Builder rowSchema(Schema newRowSchema) { } public Context build() { + if (allowEqualityDelete) { + Preconditions.checkNotNull(equalityFieldIds, "Equality field ids shouldn't be null for equality deletes"); + Preconditions.checkNotNull(rowSchema, "Row schema shouldn't be null for equality deletes"); + + for (Integer fieldId : equalityFieldIds) { + Preconditions.checkNotNull(rowSchema.findField(fieldId), + "Missing field id %s in provided row schema: %s", fieldId, rowSchema); + } + } + return new Context(allowPosDelete, allowEqualityDelete, equalityFieldIds, rowSchema); } } diff --git a/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java b/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java index 13e556686425..fbecdab14923 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java +++ b/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java @@ -121,6 +121,7 @@ public void testWriteEqualityDelete() throws IOException { DeltaWriterFactory.Context ctxt = DeltaWriterFactory.Context.builder() .allowEqualityDelete(true) .equalityFieldIds(equalityFieldIds) + .rowSchema(table.schema()) .build(); // TODO More unit tests to test the partitioned case. @@ -164,7 +165,7 @@ public void testUpsertSameRow() throws IOException { DeltaWriterFactory.Context ctxt = DeltaWriterFactory.Context.builder() .allowEqualityDelete(true) .equalityFieldIds(equalityFieldIds) - .rowSchema(SCHEMA) + .rowSchema(table.schema().select("id")) .build(); DeltaWriter deltaWriter = writerFactory.createDeltaWriter(null, ctxt); 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..cc14b9d8551d 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -361,7 +361,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"); From 11e61deef078158eae440608635ca8b47dd2d6cb Mon Sep 17 00:00:00 2001 From: huzheng Date: Mon, 9 Nov 2020 10:51:33 +0800 Subject: [PATCH 24/41] Add more unit tests. --- .../iceberg/data/TestGenericDeltaWriter.java | 60 +++++++++++++++++-- 1 file changed, 55 insertions(+), 5 deletions(-) diff --git a/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java b/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java index fbecdab14923..c9cda56cf955 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java +++ b/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java @@ -52,16 +52,17 @@ @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 = "formatVersion = {0}, format = {1}, partitioned = {2}") + @Parameterized.Parameters(name = "format = {0}, partitioned = {1}") public static Object[][] parameters() { return new Object[][] { - {2, "avro", false}, - {2, "parquet", false}, + {"avro", false}, + {"parquet", false}, }; } @@ -69,8 +70,8 @@ public static Object[][] parameters() { private final boolean partitioned; private Table table; - public TestGenericDeltaWriter(int formatVersion, String format, boolean partitioned) { - super(formatVersion); + public TestGenericDeltaWriter(String format, boolean partitioned) { + super(FORMAT_VERSION_V2); this.format = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH)); this.partitioned = partitioned; } @@ -158,6 +159,55 @@ public void testWriteEqualityDelete() throws IOException { assertTableRecords(ImmutableSet.of()); } + @Test + public void testPositionDelete() throws IOException { + 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() throws IOException { DeltaWriterFactory writerFactory = createDeltaWriterFactory(); From 9deb59d49fcc7e48459fd31f78a70860ee85c928 Mon Sep 17 00:00:00 2001 From: huzheng Date: Mon, 9 Nov 2020 11:49:22 +0800 Subject: [PATCH 25/41] Add flink delta writers. --- .../org/apache/iceberg/TableMetadata.java | 12 +- .../java/org/apache/iceberg/avro/Avro.java | 18 +- .../apache/iceberg/deletes/DeletesUtil.java | 46 +++ .../apache/iceberg/hadoop/HadoopTables.java | 3 +- .../apache/iceberg/io/DeltaWriterFactory.java | 2 +- .../data/GenericDeltaWriterFactory.java | 8 +- .../flink/sink/FlinkDeltaWriterFactory.java | 284 ++++++++++++++++ .../flink/sink/TestFlinkDeltaWriter.java | 306 ++++++++++++++++++ .../org/apache/iceberg/parquet/Parquet.java | 51 +-- .../iceberg/parquet/ParquetValueWriters.java | 12 +- 10 files changed, 693 insertions(+), 49 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/deletes/DeletesUtil.java create mode 100644 flink/src/main/java/org/apache/iceberg/flink/sink/FlinkDeltaWriterFactory.java create mode 100644 flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkDeltaWriter.java diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index d00e2a8f37a1..fd63feb918ec 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -82,12 +82,12 @@ public static TableMetadata newTableMetadata(Schema schema, return newTableMetadata(schema, spec, SortOrder.unsorted(), location, properties, DEFAULT_TABLE_FORMAT_VERSION); } - static TableMetadata newTableMetadata(Schema schema, - PartitionSpec spec, - SortOrder sortOrder, - String location, - Map properties, - int formatVersion) { + public static TableMetadata newTableMetadata(Schema schema, + PartitionSpec spec, + SortOrder sortOrder, + String location, + Map properties, + int formatVersion) { // reassign all column ids to ensure consistency AtomicInteger lastColumnId = new AtomicInteger(0); Schema freshSchema = TypeUtil.assignFreshIds(schema, lastColumnId::incrementAndGet); 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..1f8cca4ef61f 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; @@ -300,23 +299,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.posDeleteSchema(rowSchema)); + if (createWriterFunc != null) { 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.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..83c07f00aa05 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/deletes/DeletesUtil.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.deletes; + +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.Types; + +public class DeletesUtil { + + private DeletesUtil() { + } + + public static Schema posDeleteSchema(Schema rowSchema) { + if (rowSchema == null) { + return new Schema( + MetadataColumns.DELETE_FILE_PATH, + MetadataColumns.DELETE_FILE_POS); + } else { + // the appender uses the row schema wrapped with position fields + return new Schema( + MetadataColumns.DELETE_FILE_PATH, + MetadataColumns.DELETE_FILE_POS, + Types.NestedField.optional( + MetadataColumns.DELETE_FILE_ROW_FIELD_ID, "row", rowSchema.asStruct(), + MetadataColumns.DELETE_FILE_ROW_DOC)); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/hadoop/HadoopTables.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopTables.java index 645efc964778..1e43b4b013c2 100644 --- a/core/src/main/java/org/apache/iceberg/hadoop/HadoopTables.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopTables.java @@ -141,7 +141,8 @@ public Table create(Schema schema, PartitionSpec spec, SortOrder order, Map tableProps = properties == null ? ImmutableMap.of() : properties; PartitionSpec partitionSpec = spec == null ? PartitionSpec.unpartitioned() : spec; SortOrder sortOrder = order == null ? SortOrder.unsorted() : order; - TableMetadata metadata = TableMetadata.newTableMetadata(schema, partitionSpec, sortOrder, location, tableProps); + // TODO: add API to support write format v2. + TableMetadata metadata = TableMetadata.newTableMetadata(schema, partitionSpec, sortOrder, location, tableProps, 2); ops.commit(null, metadata); return new BaseTable(ops, location); diff --git a/core/src/main/java/org/apache/iceberg/io/DeltaWriterFactory.java b/core/src/main/java/org/apache/iceberg/io/DeltaWriterFactory.java index 7b485df7092d..0a986fbb9d50 100644 --- a/core/src/main/java/org/apache/iceberg/io/DeltaWriterFactory.java +++ b/core/src/main/java/org/apache/iceberg/io/DeltaWriterFactory.java @@ -40,7 +40,7 @@ public interface DeltaWriterFactory { ContentFileWriterFactory createEqualityDeleteWriterFactory(List equalityFieldIds, Schema rowSchema); - ContentFileWriterFactory> createPosDeleteWriterFactory(); + ContentFileWriterFactory> createPosDeleteWriterFactory(Schema rowSchema); class Context { private final boolean allowPosDelete; diff --git a/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java b/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java index ab7ae612d157..2c54f5eb41a9 100644 --- a/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java +++ b/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java @@ -86,7 +86,7 @@ public DeltaWriter createDeltaWriter(PartitionKey partitionKey, Context RollingContentFileWriter> posDeleteWriter = new RollingContentFileWriter<>(partitionKey, - format, fileFactory, io, targetFileSize, createPosDeleteWriterFactory()); + format, fileFactory, io, targetFileSize, createPosDeleteWriterFactory(ctxt.rowSchema())); if (ctxt.allowPosDelete() && !ctxt.allowEqualityDelete()) { return new BaseDeltaWriter<>(dataWriter, posDeleteWriter); @@ -169,7 +169,7 @@ public ContentFileWriterFactory createEqualityDeleteWriterFa } @Override - public ContentFileWriterFactory> createPosDeleteWriterFactory() { + public ContentFileWriterFactory> createPosDeleteWriterFactory(Schema rowSchema) { return (partitionKey, outputFile, fileFormat) -> { MetricsConfig metricsConfig = MetricsConfig.fromProperties(tableProperties); try { @@ -180,7 +180,7 @@ public ContentFileWriterFactory> createPosDel .withPartition(partitionKey) .overwrite() .setAll(tableProperties) - .rowSchema(schema) // it's a nullable field. + .rowSchema(rowSchema) // it's a nullable field. .withSpec(spec) .withKeyMetadata(outputFile.keyMetadata()) .buildPositionWriter(); @@ -192,7 +192,7 @@ public ContentFileWriterFactory> createPosDel .overwrite() .setAll(tableProperties) .metricsConfig(metricsConfig) - .rowSchema(schema) // it's a nullable field. + .rowSchema(rowSchema) // it's a nullable field. .withSpec(spec) .withKeyMetadata(outputFile.keyMetadata()) .buildPositionWriter(); 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..e54b58cc958c --- /dev/null +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkDeltaWriterFactory.java @@ -0,0 +1,284 @@ +/* + * 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.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.FlinkOrcWriter; +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.orc.ORC; +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()) { + return new BaseDeltaWriter<>(dataWriter); + } + + RollingContentFileWriter> posDeleteWriter = + new RollingContentFileWriter<>(partitionKey, + format, fileFactory, io, targetFileSize, createPosDeleteWriterFactory(ctxt.rowSchema())); + + if (ctxt.allowPosDelete() && !ctxt.allowEqualityDelete()) { + return new BaseDeltaWriter<>(dataWriter, posDeleteWriter); + } + + 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."); + + RollingContentFileWriter eqDeleteWriter = new RollingContentFileWriter<>(partitionKey, + format, fileFactory, io, targetFileSize, + createEqualityDeleteWriterFactory(ctxt.equalityFieldIds(), ctxt.rowSchema())); + + // Define flink's as struct like function. + RowDataWrapper asStructLike = new RowDataWrapper(flinkSchema, schema.asStruct()); + + return new BaseDeltaWriter<>(dataWriter, posDeleteWriter, eqDeleteWriter, schema, ctxt.equalityFieldIds(), + asStructLike::wrap); + } + + @Override + public FileAppenderFactory createFileAppenderFactory() { + return (outputFile, fileFormat) -> { + MetricsConfig metricsConfig = MetricsConfig.fromProperties(tableProperties); + try { + switch (format) { + case AVRO: + return Avro.write(outputFile) + .createWriterFunc(ignore -> new FlinkAvroWriter(flinkSchema)) + .setAll(tableProperties) + .schema(schema) + .overwrite() + .build(); + + case ORC: + return ORC.write(outputFile) + .createWriterFunc((iSchema, typDesc) -> FlinkOrcWriter.buildWriter(flinkSchema, iSchema)) + .setAll(tableProperties) + .schema(schema) + .overwrite() + .build(); + + case PARQUET: + return Parquet.write(outputFile) + .createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(flinkSchema, msgType)) + .setAll(tableProperties) + .metricsConfig(metricsConfig) + .schema(schema) + .overwrite() + .build(); + + default: + throw new UnsupportedOperationException("Cannot write unknown file format: " + format); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + }; + } + + @Override + public ContentFileWriterFactory createDataFileWriterFactory() { + // TODO Move this to its default function ??? + return (partitionKey, outputFile, fileFormat) -> { + FileAppender appender = appenderFactory.newAppender(outputFile.encryptingOutputFile(), format); + + return new DataFileWriter<>(appender, + format, + 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 static class RowDataPositionAccessor implements Parquet.PositionAccessor { + private static final RowDataPositionAccessor INSTANCE = new RowDataPositionAccessor(); + + @Override + public StringData accessFilePath(CharSequence path) { + return StringData.fromString(path.toString()); + } + + @Override + public Long accessPos(long pos) { + return pos; + } + } +} 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..fb12cf8f8564 --- /dev/null +++ b/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkDeltaWriter.java @@ -0,0 +1,306 @@ +/* + * 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.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.SimpleDataUtil; +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.ImmutableMap; +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.SCHEMA; +import static org.apache.iceberg.flink.SimpleDataUtil.createRowData; + +@RunWith(Parameterized.class) +public class TestFlinkDeltaWriter { + + @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) { + 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()); + + this.table = SimpleDataUtil.createTable(tableDir.getAbsolutePath(), ImmutableMap.of(), partitioned); + } + + @Test + public void testWritePureInsert() throws IOException { + 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() throws IOException { + DeltaWriterFactory writerFactory = createDeltaWriterFactory(); + + List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); + DeltaWriterFactory.Context ctxt = DeltaWriterFactory.Context.builder() + .allowEqualityDelete(true) + .equalityFieldIds(equalityFieldIds) + .rowSchema(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 testPositionDelete() throws IOException { + 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() throws IOException { + DeltaWriterFactory writerFactory = createDeltaWriterFactory(); + List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); + DeltaWriterFactory.Context ctxt = DeltaWriterFactory.Context.builder() + .allowEqualityDelete(true) + .equalityFieldIds(equalityFieldIds) + .rowSchema(table.schema().select("id")) + .build(); + DeltaWriter deltaWriter = 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"); + + deltaWriter.writeRow(record1); + deltaWriter.writeRow(record2); + + // Commit the transaction. + WriterResult result = deltaWriter.complete(); + Assert.assertEquals(result.dataFiles().length, 1); + Assert.assertEquals(result.deleteFiles().length, 0); + commitTransaction(result); + + assertTableRecords(ImmutableSet.of(record1, record2)); + + deltaWriter = writerFactory.createDeltaWriter(null, ctxt); + + // UPSERT (1, "ccc") + deltaWriter.writeEqualityDelete(record3); + deltaWriter.writeRow(record3); + + // INSERT (1, "ddd") + // INSERT (1, "eee") + deltaWriter.writeRow(record4); + deltaWriter.writeRow(record5); + + // UPSERT (1, "fff") + deltaWriter.writeEqualityDelete(record6); + deltaWriter.writeRow(record6); + + // INSERT (1, "ggg") + deltaWriter.writeRow(record7); + + // Commit the transaction. + result = deltaWriter.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, record7)); + } + + 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/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java index cc14b9d8551d..fb6561b8757e 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 accessFilePath(CharSequence path); + + POS accessPos(long pos); + } + + private static class IdentityPositionAccessor implements Parquet.PositionAccessor { + + static final IdentityPositionAccessor INSTANCE = new IdentityPositionAccessor(); + + @Override + public CharSequence accessFilePath(CharSequence path) { + return path; + } + + @Override + public Long accessPos(long pos) { + return pos; + } + } + public static class DeleteWriteBuilder { private final WriteBuilder appenderBuilder; private final String location; @@ -379,42 +400,36 @@ 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))); + appenderBuilder.schema(DeletesUtil.posDeleteSchema(rowSchema)); + if (createWriterFunc != null) { 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.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..0ddb5fb63b9e 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.accessFilePath(delete.path()); case 1: - return delete.pos(); + return positionAccessor.accessPos(delete.pos()); case 2: return delete.row(); } From 01951459497663bf79edc1870ee0046853ccfdd8 Mon Sep 17 00:00:00 2001 From: huzheng Date: Mon, 9 Nov 2020 17:59:06 +0800 Subject: [PATCH 26/41] Remove the useless FlinkEqualityDeleterFactory. --- .../sink/FlinkEqualityDeleterFactory.java | 100 ------------------ 1 file changed, 100 deletions(-) delete mode 100644 flink/src/main/java/org/apache/iceberg/flink/sink/FlinkEqualityDeleterFactory.java diff --git a/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkEqualityDeleterFactory.java b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkEqualityDeleterFactory.java deleted file mode 100644 index b0a8e9c5d139..000000000000 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkEqualityDeleterFactory.java +++ /dev/null @@ -1,100 +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.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.ContentFileWriter; -import org.apache.iceberg.ContentFileWriterFactory; -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.avro.Avro; -import org.apache.iceberg.encryption.EncryptedOutputFile; -import org.apache.iceberg.flink.data.FlinkAvroWriter; -import org.apache.iceberg.flink.data.FlinkParquetWriters; -import org.apache.iceberg.parquet.Parquet; - -public class FlinkEqualityDeleterFactory implements ContentFileWriterFactory, Serializable { - private final Schema schema; - private final RowType flinkSchema; - private final PartitionSpec spec; - private final List equalityFieldIds; - private final Map props; - - public FlinkEqualityDeleterFactory(Schema schema, - RowType flinkSchema, - PartitionSpec spec, - List equalityFieldIds, - Map props) { - this.schema = schema; - this.flinkSchema = flinkSchema; - this.spec = spec; - this.equalityFieldIds = equalityFieldIds; - this.props = props; - } - - @Override - public ContentFileWriter createWriter(PartitionKey partitionKey, - EncryptedOutputFile outputFile, - FileFormat fileFormat) { - MetricsConfig metricsConfig = MetricsConfig.fromProperties(props); - try { - switch (fileFormat) { - case AVRO: - return Avro.writeDeletes(outputFile.encryptingOutputFile()) - .createWriterFunc(ignore -> new FlinkAvroWriter(flinkSchema)) - .withPartition(partitionKey) - .overwrite() - .setAll(props) - .rowSchema(schema) - .withSpec(spec) - .equalityFieldIds(equalityFieldIds) - .buildEqualityWriter(); - - case PARQUET: - return Parquet.writeDeletes(outputFile.encryptingOutputFile()) - .createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(flinkSchema, msgType)) - .withPartition(partitionKey) - .overwrite() - .setAll(props) - .metricsConfig(metricsConfig) - .rowSchema(schema) - .withSpec(spec) - .equalityFieldIds(equalityFieldIds) - .buildEqualityWriter(); - - case ORC: - default: - throw new UnsupportedOperationException("Cannot write unknown file format: " + fileFormat); - } - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } -} From 3449d87b013f10f16119536f78d96010dfcdf295 Mon Sep 17 00:00:00 2001 From: huzheng Date: Mon, 9 Nov 2020 19:23:27 +0800 Subject: [PATCH 27/41] Fix the broken TestTaskWriters. --- .../apache/iceberg/io/PartitionedWriter.java | 12 ++++++++-- .../iceberg/io/RollingContentFileWriter.java | 1 + .../flink/sink/PartitionedFanoutWriter.java | 24 +++++++------------ 3 files changed, 19 insertions(+), 18 deletions(-) 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 91e59d7b9044..feda2358c237 100644 --- a/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java @@ -99,9 +99,15 @@ public void close() throws IOException { @Override public void abort() throws IOException { + close(); + if (currentWriter != null) { - // Called complete() rather abort() to get all the completed files. - resultBuilder.add(currentWriter.complete()); + try { + currentWriter.abort(); + } catch (IOException e) { + LOG.warn("Failed to abort the current writer: {} because: ", currentWriter, e); + } + currentWriter = null; } @@ -113,6 +119,8 @@ public void abort() throws IOException { @Override public WriterResult complete() throws IOException { + close(); + if (currentWriter != null) { resultBuilder.add(currentWriter.complete()); currentWriter = null; diff --git a/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java b/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java index 8f04ef6758a4..febad42802ab 100644 --- a/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java @@ -126,6 +126,7 @@ private boolean shouldRollToNewFile() { private void closeCurrent() throws IOException { if (currentFileWriter != null) { currentFileWriter.close(); + ContentFileT contentFile = currentFileWriter.toContentFile(); Metrics metrics = currentFileWriter.metrics(); this.currentFileWriter = null; 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 index 6fcc1e2c39d3..d451d3fcbcdf 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedFanoutWriter.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedFanoutWriter.java @@ -30,13 +30,9 @@ import org.apache.iceberg.io.TaskWriter; import org.apache.iceberg.io.WriterResult; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; abstract class PartitionedFanoutWriter implements TaskWriter { - private static final Logger LOG = LoggerFactory.getLogger(PartitionedFanoutWriter.class); - private final FileFormat format; private final OutputFileFactory fileFactory; private final FileIO io; @@ -81,33 +77,29 @@ public void write(T row) throws IOException { @Override public void close() throws IOException { if (!writers.isEmpty()) { - for (PartitionKey key : writers.keySet()) { - writers.get(key).close(); + for (RollingContentFileWriter writer : writers.values()) { + writer.close(); } - writers.clear(); } } @Override - public void abort() { + public void abort() throws IOException { + close(); + for (RollingContentFileWriter writer : writers.values()) { - try { - writer.abort(); - } catch (IOException e) { - LOG.warn("Failed to abort the writer {} because: ", writer, e); - } + writer.abort(); } - writers.clear(); } @Override public WriterResult complete() throws IOException { - WriterResult.Builder builder = WriterResult.builder(); + close(); + WriterResult.Builder builder = WriterResult.builder(); for (RollingContentFileWriter writer : writers.values()) { builder.add(writer.complete()); } - writers.clear(); return builder.build(); } From 69b2eb2e0cbf100e08725b2de816422d8d6e3676 Mon Sep 17 00:00:00 2001 From: huzheng Date: Mon, 9 Nov 2020 20:24:35 +0800 Subject: [PATCH 28/41] Integrate the flink sink with delta writer. --- .../java/org/apache/iceberg/avro/Avro.java | 2 +- .../iceberg/flink/sink/RowDataTaskWriter.java | 162 ++++++++++++++++++ .../flink/sink/RowDataTaskWriterFactory.java | 39 +---- .../flink/sink/TestRowDataTaskWriter.java | 29 ++++ 4 files changed, 197 insertions(+), 35 deletions(-) create mode 100644 flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriter.java create mode 100644 flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataTaskWriter.java 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 1f8cca4ef61f..e5d05cf63922 100644 --- a/core/src/main/java/org/apache/iceberg/avro/Avro.java +++ b/core/src/main/java/org/apache/iceberg/avro/Avro.java @@ -301,7 +301,7 @@ public PositionDeleteWriter buildPositionWriter() throws IOException { appenderBuilder.schema(DeletesUtil.posDeleteSchema(rowSchema)); - if (createWriterFunc != null) { + if (rowSchema != null && createWriterFunc != null) { appenderBuilder.createWriterFunc( avroSchema -> new PositionAndRowDatumWriter<>(createWriterFunc.apply(avroSchema))); } else { 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..1ca2d2eef5c3 --- /dev/null +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriter.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.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.Lists; +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; + +public 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) { + this.spec = spec; + this.deltaWriterFactory = new FlinkDeltaWriterFactory(schema, flinkSchema, spec, + format, fileFactory, io, targetFileSize, tableProperties); + this.partitionKey = new PartitionKey(spec, schema); + this.rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); + + this.deltaWriterMap = Maps.newHashMap(); + + // TODO make it to be a valuable equality field ids. + List equalityFieldIds = Lists.newArrayList(); + this.ctxt = DeltaWriterFactory.Context.builder() + //.allowEqualityDelete(true) TODO enable this switch??? + //.equalityFieldIds(equalityFieldIds) + .rowSchema(TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds))) + .build(); + } + + @Override + public void write(RowData row) throws IOException { + 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 -> { + try { + deltaWriter.abort(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + }); + } + + @Override + public WriterResult complete() throws IOException { + 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 b2e3cf51a80a..8d859ae5452c 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 @@ -25,19 +25,15 @@ import java.util.Map; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.DataFile; 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.DataFileWriterFactory; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.FileAppenderFactory; import org.apache.iceberg.io.FileIO; @@ -45,7 +41,6 @@ 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; @@ -59,7 +54,7 @@ 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 transient OutputFileFactory outputFileFactory; @@ -80,7 +75,7 @@ public RowDataTaskWriterFactory(Schema schema, this.encryptionManager = encryptionManager; this.targetFileSizeBytes = targetFileSizeBytes; this.format = format; - this.appenderFactory = new FlinkFileAppenderFactory(schema, flinkSchema, tableProperties); + this.tableProperties = tableProperties; } @Override @@ -93,35 +88,11 @@ 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<>(format, outputFileFactory, io, - targetFileSizeBytes, new DataFileWriterFactory<>(appenderFactory, spec)); - } 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(format, fileFactory, io, targetFileSize, new DataFileWriterFactory<>(appenderFactory, spec)); - 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; - } + return new RowDataTaskWriter(schema, flinkSchema, spec, format, outputFileFactory, io, targetFileSizeBytes, + tableProperties); } + // TODO we should clear this class once we changed to use the FlinkDeltaWriterFactory#createFileAppenderFactory(). public static class FlinkFileAppenderFactory implements FileAppenderFactory, Serializable { private final Schema schema; private final RowType flinkSchema; 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() { + } +} From ee87c08b3bc9ab2804f383f145dbf7ac2d870e7d Mon Sep 17 00:00:00 2001 From: huzheng Date: Tue, 10 Nov 2020 10:04:25 +0800 Subject: [PATCH 29/41] Fix the broken TestIcebergSourceHadoopTables3 tests --- build.gradle | 1 + .../java/org/apache/iceberg/TableMetadata.java | 12 ++++++------ .../org/apache/iceberg/hadoop/HadoopTables.java | 3 +-- .../flink/sink/TestFlinkDeltaWriter.java | 17 ++++++++++++----- 4 files changed, 20 insertions(+), 13 deletions(-) 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/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index fd63feb918ec..d00e2a8f37a1 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -82,12 +82,12 @@ public static TableMetadata newTableMetadata(Schema schema, return newTableMetadata(schema, spec, SortOrder.unsorted(), location, properties, DEFAULT_TABLE_FORMAT_VERSION); } - public static TableMetadata newTableMetadata(Schema schema, - PartitionSpec spec, - SortOrder sortOrder, - String location, - Map properties, - int formatVersion) { + static TableMetadata newTableMetadata(Schema schema, + PartitionSpec spec, + SortOrder sortOrder, + String location, + Map properties, + int formatVersion) { // reassign all column ids to ensure consistency AtomicInteger lastColumnId = new AtomicInteger(0); Schema freshSchema = TypeUtil.assignFreshIds(schema, lastColumnId::incrementAndGet); diff --git a/core/src/main/java/org/apache/iceberg/hadoop/HadoopTables.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopTables.java index 1e43b4b013c2..645efc964778 100644 --- a/core/src/main/java/org/apache/iceberg/hadoop/HadoopTables.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopTables.java @@ -141,8 +141,7 @@ public Table create(Schema schema, PartitionSpec spec, SortOrder order, Map tableProps = properties == null ? ImmutableMap.of() : properties; PartitionSpec partitionSpec = spec == null ? PartitionSpec.unpartitioned() : spec; SortOrder sortOrder = order == null ? SortOrder.unsorted() : order; - // TODO: add API to support write format v2. - TableMetadata metadata = TableMetadata.newTableMetadata(schema, partitionSpec, sortOrder, location, tableProps, 2); + TableMetadata metadata = TableMetadata.newTableMetadata(schema, partitionSpec, sortOrder, location, tableProps); ops.commit(null, metadata); return new BaseTable(ops, location); 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 index fb12cf8f8564..5c1fa54eec35 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkDeltaWriter.java +++ b/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkDeltaWriter.java @@ -29,21 +29,22 @@ 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.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.SimpleDataUtil; 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.ImmutableMap; 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; @@ -57,11 +58,12 @@ import org.junit.runners.Parameterized; import static org.apache.iceberg.flink.SimpleDataUtil.ROW_TYPE; -import static org.apache.iceberg.flink.SimpleDataUtil.SCHEMA; import static org.apache.iceberg.flink.SimpleDataUtil.createRowData; @RunWith(Parameterized.class) -public class TestFlinkDeltaWriter { +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(); @@ -79,6 +81,7 @@ public static Object[][] parameters() { private Table table; public TestFlinkDeltaWriter(String format, boolean partitioned) { + super(FORMAT_V2); this.format = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH)); this.partitioned = partitioned; } @@ -88,7 +91,11 @@ public void before() throws IOException { File tableDir = tempFolder.newFolder(); Assert.assertTrue(tableDir.delete()); - this.table = SimpleDataUtil.createTable(tableDir.getAbsolutePath(), ImmutableMap.of(), partitioned); + 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 From fb2e6f8fa8f103b49fac311f01981b6995b6dc7b Mon Sep 17 00:00:00 2001 From: huzheng Date: Tue, 10 Nov 2020 10:12:28 +0800 Subject: [PATCH 30/41] Unfied the flink file appender factory. --- .../flink/sink/FlinkDeltaWriterFactory.java | 39 +------- .../flink/sink/FlinkFileAppenderFactory.java | 90 +++++++++++++++++++ .../iceberg/flink/sink/RowDataTaskWriter.java | 5 +- .../flink/sink/RowDataTaskWriterFactory.java | 64 ------------- .../apache/iceberg/flink/SimpleDataUtil.java | 5 +- 5 files changed, 96 insertions(+), 107 deletions(-) create mode 100644 flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileAppenderFactory.java 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 index e54b58cc958c..35b5c80f94ca 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkDeltaWriterFactory.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkDeltaWriterFactory.java @@ -43,7 +43,6 @@ 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.FlinkOrcWriter; import org.apache.iceberg.flink.data.FlinkParquetWriters; import org.apache.iceberg.io.BaseDeltaWriter; import org.apache.iceberg.io.DeltaWriter; @@ -53,7 +52,6 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.OutputFileFactory; import org.apache.iceberg.io.RollingContentFileWriter; -import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -122,42 +120,7 @@ public DeltaWriter createDeltaWriter(PartitionKey partitionKey, Context @Override public FileAppenderFactory createFileAppenderFactory() { - return (outputFile, fileFormat) -> { - MetricsConfig metricsConfig = MetricsConfig.fromProperties(tableProperties); - try { - switch (format) { - case AVRO: - return Avro.write(outputFile) - .createWriterFunc(ignore -> new FlinkAvroWriter(flinkSchema)) - .setAll(tableProperties) - .schema(schema) - .overwrite() - .build(); - - case ORC: - return ORC.write(outputFile) - .createWriterFunc((iSchema, typDesc) -> FlinkOrcWriter.buildWriter(flinkSchema, iSchema)) - .setAll(tableProperties) - .schema(schema) - .overwrite() - .build(); - - case PARQUET: - return Parquet.write(outputFile) - .createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(flinkSchema, msgType)) - .setAll(tableProperties) - .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 FlinkFileAppenderFactory(schema, flinkSchema, tableProperties); } @Override 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/RowDataTaskWriter.java b/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriter.java index 1ca2d2eef5c3..5074cf496e66 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriter.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriter.java @@ -42,7 +42,7 @@ import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.Tasks; -public class RowDataTaskWriter implements TaskWriter { +class RowDataTaskWriter implements TaskWriter { private final PartitionSpec spec; private final FlinkDeltaWriterFactory deltaWriterFactory; @@ -60,9 +60,10 @@ public class RowDataTaskWriter implements TaskWriter { FileIO io, long targetFileSize, Map tableProperties) { - this.spec = spec; 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()); 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 8d859ae5452c..3f3c30a358ba 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,30 +19,17 @@ 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.PartitionSpec; import org.apache.iceberg.Schema; -import org.apache.iceberg.avro.Avro; import org.apache.iceberg.encryption.EncryptionManager; -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.orc.ORC; -import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; public class RowDataTaskWriterFactory implements TaskWriterFactory { @@ -91,55 +78,4 @@ public TaskWriter create() { return new RowDataTaskWriter(schema, flinkSchema, spec, format, outputFileFactory, io, targetFileSizeBytes, tableProperties); } - - // TODO we should clear this class once we changed to use the FlinkDeltaWriterFactory#createFileAppenderFactory(). - 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); - } - } - } } 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..a8b076aa218c 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) { From 18a6808a0ccaa07dbf40b222924d6008c1fe4962 Mon Sep 17 00:00:00 2001 From: huzheng Date: Tue, 10 Nov 2020 11:37:32 +0800 Subject: [PATCH 31/41] Add equality field ids to the FlinkSink builder. --- .../apache/iceberg/flink/sink/FlinkSink.java | 25 ++++++++++++++++--- .../iceberg/flink/sink/RowDataTaskWriter.java | 20 ++++++++------- .../flink/sink/RowDataTaskWriterFactory.java | 8 ++++-- .../iceberg/flink/source/RowDataRewriter.java | 3 ++- .../flink/sink/TestIcebergStreamWriter.java | 2 +- .../iceberg/flink/sink/TestTaskWriters.java | 2 +- 6 files changed, 43 insertions(+), 17 deletions(-) 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..a075cd0bf78a 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; @@ -43,6 +44,7 @@ import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.TableLoader; 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 +114,7 @@ public static class Builder { private TableSchema tableSchema; private boolean overwrite = false; private Integer writeParallelism = null; + private List equalityFieldNames = null; private Builder() { } @@ -168,6 +171,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,7 +191,16 @@ 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; @@ -201,7 +218,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 +242,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/RowDataTaskWriter.java b/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriter.java index 5074cf496e66..c38cdd72731d 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriter.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriter.java @@ -36,7 +36,6 @@ 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.Lists; 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; @@ -59,7 +58,8 @@ class RowDataTaskWriter implements TaskWriter { OutputFileFactory fileFactory, FileIO io, long targetFileSize, - Map tableProperties) { + Map tableProperties, + List equalityFieldIds) { this.deltaWriterFactory = new FlinkDeltaWriterFactory(schema, flinkSchema, spec, format, fileFactory, io, targetFileSize, tableProperties); @@ -69,13 +69,15 @@ class RowDataTaskWriter implements TaskWriter { this.deltaWriterMap = Maps.newHashMap(); - // TODO make it to be a valuable equality field ids. - List equalityFieldIds = Lists.newArrayList(); - this.ctxt = DeltaWriterFactory.Context.builder() - //.allowEqualityDelete(true) TODO enable this switch??? - //.equalityFieldIds(equalityFieldIds) - .rowSchema(TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds))) - .build(); + if (equalityFieldIds != null && equalityFieldIds.size() > 0) { + this.ctxt = DeltaWriterFactory.Context.builder() + .allowEqualityDelete(true) + .equalityFieldIds(equalityFieldIds) + .rowSchema(TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds))) + .build(); + } else { + this.ctxt = DeltaWriterFactory.Context.builder().build(); + } } @Override 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 3f3c30a358ba..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,6 +19,7 @@ package org.apache.iceberg.flink.sink; +import java.util.List; import java.util.Map; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; @@ -42,6 +43,7 @@ public class RowDataTaskWriterFactory implements TaskWriterFactory { private final long targetFileSizeBytes; private final FileFormat format; private final Map tableProperties; + private final List equalityFieldIds; private transient OutputFileFactory outputFileFactory; @@ -53,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; @@ -63,6 +66,7 @@ public RowDataTaskWriterFactory(Schema schema, this.targetFileSizeBytes = targetFileSizeBytes; this.format = format; this.tableProperties = tableProperties; + this.equalityFieldIds = equalityFieldIds; } @Override @@ -76,6 +80,6 @@ public TaskWriter create() { "The outputFileFactory shouldn't be null if we have invoked the initialize()."); return new RowDataTaskWriter(schema, flinkSchema, spec, format, outputFileFactory, io, targetFileSizeBytes, - tableProperties); + 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 3c8d62146148..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) { 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..51e5e1cd1ab2 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 @@ -317,7 +317,7 @@ private OneInputStreamOperatorTestHarness createIcebergStream private OneInputStreamOperatorTestHarness createIcebergStreamWriter( Table icebergTable, TableSchema flinkSchema) throws Exception { - IcebergStreamWriter streamWriter = FlinkSink.createStreamWriter(icebergTable, flinkSchema); + IcebergStreamWriter streamWriter = FlinkSink.createStreamWriter(icebergTable, flinkSchema, null); OneInputStreamOperatorTestHarness harness = new OneInputStreamOperatorTestHarness<>( streamWriter, 1, 1, 0); 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 5505f9a1b7c8..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 @@ -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(); } From 0788b6347e086dd5678bafe1a42888f30ffff096 Mon Sep 17 00:00:00 2001 From: huzheng Date: Tue, 10 Nov 2020 11:45:55 +0800 Subject: [PATCH 32/41] Flink: commit transaction with both data files and delete files. --- .../org/apache/iceberg/io/WriterResult.java | 19 +-- ...estSerializer.java => DeltaManifests.java} | 44 ++++--- .../flink/sink/DeltaManifestsSerializer.java | 94 ++++++++++++++ .../iceberg/flink/sink/FlinkManifestUtil.java | 72 ++++++++++- .../apache/iceberg/flink/sink/FlinkSink.java | 3 +- .../flink/sink/IcebergFilesCommitter.java | 96 ++++++++------ .../flink/sink/IcebergStreamWriter.java | 18 ++- .../apache/iceberg/flink/SimpleDataUtil.java | 8 +- .../flink/sink/TestFlinkIcebergSink.java | 118 +++++++++++++++--- .../iceberg/flink/sink/TestFlinkManifest.java | 13 +- .../flink/sink/TestIcebergFilesCommitter.java | 63 +++++----- .../flink/sink/TestIcebergStreamWriter.java | 56 +++++---- 12 files changed, 448 insertions(+), 156 deletions(-) rename flink/src/main/java/org/apache/iceberg/flink/sink/{FlinkManifestSerializer.java => DeltaManifests.java} (50%) create mode 100644 flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java diff --git a/core/src/main/java/org/apache/iceberg/io/WriterResult.java b/core/src/main/java/org/apache/iceberg/io/WriterResult.java index 288cf8d51908..4c08b168aaa5 100644 --- a/core/src/main/java/org/apache/iceberg/io/WriterResult.java +++ b/core/src/main/java/org/apache/iceberg/io/WriterResult.java @@ -19,6 +19,7 @@ package org.apache.iceberg.io; +import java.io.Serializable; import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; @@ -26,11 +27,11 @@ 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.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -public class WriterResult { - private static final WriterResult EMPTY = new WriterResult(ImmutableList.of(), ImmutableList.of()); +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; @@ -45,10 +46,6 @@ public class WriterResult { this.deleteFiles = deleteFiles.toArray(new DeleteFile[0]); } - static WriterResult empty() { - return EMPTY; - } - public DataFile[] dataFiles() { return dataFiles; } @@ -82,6 +79,14 @@ public ContentFile next() { }; } + 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(); } 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/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 a075cd0bf78a..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 @@ -43,6 +43,7 @@ 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; @@ -206,7 +207,7 @@ public DataStreamSink build() { 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) 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 8e9861171534..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().dataFiles()) { - 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().dataFiles()) { - 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/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java index a8b076aa218c..fdf53b4bc6b9 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java +++ b/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java @@ -115,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/TestFlinkIcebergSink.java b/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java index 0022425d8447..c16d92e0842a 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,10 @@ import java.io.IOException; import java.util.List; import java.util.Locale; -import java.util.Map; 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 +34,52 @@ 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.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; 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; + @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 +107,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 +115,28 @@ 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()); - 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 +151,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 +165,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); + SimpleDataUtil.assertTableRows(table, expectedRows); } private void testWriteRow(TableSchema tableSchema) throws Exception { @@ -150,7 +175,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 +186,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); + SimpleDataUtil.assertTableRows(table, expectedRows); } @Test @@ -171,6 +196,61 @@ 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") + ); + List expectedResult = Lists.newArrayList(Iterables.concat(rowDataList, rowDataList)); + SimpleDataUtil.assertTableRows(table, expectedResult); + } + + 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 51e5e1cd1ab2..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, null); - OneInputStreamOperatorTestHarness harness = new OneInputStreamOperatorTestHarness<>( + OneInputStreamOperatorTestHarness harness = new OneInputStreamOperatorTestHarness<>( streamWriter, 1, 1, 0); harness.setup(); From 0b8ded656e5e3a79094ac66d044fd77206b0386d Mon Sep 17 00:00:00 2001 From: huzheng Date: Wed, 11 Nov 2020 11:23:04 +0800 Subject: [PATCH 33/41] Fix the broken unit tests. --- .../main/java/org/apache/iceberg/Files.java | 8 ++++-- .../flink/sink/TestFlinkIcebergSink.java | 28 ++++++++++++++++--- 2 files changed, 29 insertions(+), 7 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/Files.java b/api/src/main/java/org/apache/iceberg/Files.java index 705a3e3c93f3..1b348cd43c85 100644 --- a/api/src/main/java/org/apache/iceberg/Files.java +++ b/api/src/main/java/org/apache/iceberg/Files.java @@ -59,9 +59,11 @@ public PositionOutputStream create() { } if (!file.getParentFile().isDirectory() && !file.getParentFile().mkdirs()) { - throw new RuntimeIOException( - "Failed to create the file's directory at %s.", - file.getParentFile().getAbsolutePath()); + if (!new File(file.getParentFile().getAbsolutePath()).exists()) { + throw new RuntimeIOException( + "Failed to create the file's directory at %s.", + file.getParentFile().getAbsolutePath()); + } } try { 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 c16d92e0842a..33c704ece744 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,6 +23,7 @@ import java.io.IOException; import java.util.List; import java.util.Locale; +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; @@ -43,10 +44,14 @@ 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.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; @@ -59,6 +64,7 @@ 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 TableTestBase { @@ -165,7 +171,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(table, expectedRows); + assertTableRows(expectedRows); } private void testWriteRow(TableSchema tableSchema) throws Exception { @@ -186,7 +192,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(table, expectedRows); + assertTableRows(expectedRows); } @Test @@ -225,8 +231,22 @@ public void testUpsertRow() throws Exception { SimpleDataUtil.createRowData(2, "bbb"), SimpleDataUtil.createRowData(3, "ccc") ); - List expectedResult = Lists.newArrayList(Iterables.concat(rowDataList, rowDataList)); - SimpleDataUtil.assertTableRows(table, expectedResult); + 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 { From d432d3f3917ec5d29858cc83d1914d9f63c79613 Mon Sep 17 00:00:00 2001 From: huzheng Date: Wed, 11 Nov 2020 14:31:04 +0800 Subject: [PATCH 34/41] Minor changes. --- api/src/main/java/org/apache/iceberg/Files.java | 8 +++----- .../apache/iceberg/flink/sink/TestFlinkIcebergSink.java | 8 +++++--- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/Files.java b/api/src/main/java/org/apache/iceberg/Files.java index 1b348cd43c85..705a3e3c93f3 100644 --- a/api/src/main/java/org/apache/iceberg/Files.java +++ b/api/src/main/java/org/apache/iceberg/Files.java @@ -59,11 +59,9 @@ public PositionOutputStream create() { } if (!file.getParentFile().isDirectory() && !file.getParentFile().mkdirs()) { - if (!new File(file.getParentFile().getAbsolutePath()).exists()) { - throw new RuntimeIOException( - "Failed to create the file's directory at %s.", - file.getParentFile().getAbsolutePath()); - } + throw new RuntimeIOException( + "Failed to create the file's directory at %s.", + file.getParentFile().getAbsolutePath()); } try { 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 33c704ece744..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 @@ -124,10 +124,12 @@ public void before() throws IOException { 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()); + if (partitioned) { - PartitionSpec spec = PartitionSpec.builderFor(SimpleDataUtil.SCHEMA) - .bucket("data", 16) - .build(); + 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, From 65971bf9eed60ddeb041909b482a95a08fd4d649 Mon Sep 17 00:00:00 2001 From: huzheng Date: Wed, 11 Nov 2020 17:30:27 +0800 Subject: [PATCH 35/41] Minor fixes --- core/src/test/java/org/apache/iceberg/TestOverwrite.java | 2 +- .../org/apache/iceberg/data/GenericDeltaWriterFactory.java | 4 ++-- .../apache/iceberg/flink/sink/FlinkDeltaWriterFactory.java | 6 +++--- .../src/main/java/org/apache/iceberg/parquet/Parquet.java | 4 ++-- .../org/apache/iceberg/parquet/ParquetValueWriters.java | 2 +- 5 files changed, 9 insertions(+), 9 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestOverwrite.java b/core/src/test/java/org/apache/iceberg/TestOverwrite.java index 8c8cca1930c0..6512ea114f23 100644 --- a/core/src/test/java/org/apache/iceberg/TestOverwrite.java +++ b/core/src/test/java/org/apache/iceberg/TestOverwrite.java @@ -94,7 +94,7 @@ public class TestOverwrite extends TableTestBase { @Parameterized.Parameters(name = "formatVersion = {0}") public static Object[] parameters() { - return new Object[] {1, 2}; + return new Object[] { 1, 2 }; } public TestOverwrite(int formatVersion) { diff --git a/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java b/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java index 2c54f5eb41a9..903b299ecdd1 100644 --- a/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java +++ b/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java @@ -113,10 +113,10 @@ public FileAppenderFactory createFileAppenderFactory() { @Override public ContentFileWriterFactory createDataFileWriterFactory() { return (partitionKey, outputFile, fileFormat) -> { - FileAppender appender = appenderFactory.newAppender(outputFile.encryptingOutputFile(), format); + FileAppender appender = appenderFactory.newAppender(outputFile.encryptingOutputFile(), fileFormat); return new DataFileWriter<>(appender, - format, + fileFormat, outputFile.encryptingOutputFile().location(), partitionKey, spec, outputFile.keyMetadata()); 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 index 35b5c80f94ca..b966c308be72 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkDeltaWriterFactory.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkDeltaWriterFactory.java @@ -127,10 +127,10 @@ public FileAppenderFactory createFileAppenderFactory() { public ContentFileWriterFactory createDataFileWriterFactory() { // TODO Move this to its default function ??? return (partitionKey, outputFile, fileFormat) -> { - FileAppender appender = appenderFactory.newAppender(outputFile.encryptingOutputFile(), format); + FileAppender appender = appenderFactory.newAppender(outputFile.encryptingOutputFile(), fileFormat); return new DataFileWriter<>(appender, - format, + fileFormat, outputFile.encryptingOutputFile().location(), partitionKey, spec, outputFile.keyMetadata()); @@ -235,7 +235,7 @@ private static class RowDataPositionAccessor implements Parquet.PositionAccessor private static final RowDataPositionAccessor INSTANCE = new RowDataPositionAccessor(); @Override - public StringData accessFilePath(CharSequence path) { + public StringData accessPath(CharSequence path) { return StringData.fromString(path.toString()); } 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 fb6561b8757e..bbe0c04f36b1 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -273,7 +273,7 @@ public static DeleteWriteBuilder writeDeletes(OutputFile file) { public interface PositionAccessor { - FILE accessFilePath(CharSequence path); + FILE accessPath(CharSequence path); POS accessPos(long pos); } @@ -283,7 +283,7 @@ private static class IdentityPositionAccessor implements Parquet.PositionAccesso static final IdentityPositionAccessor INSTANCE = new IdentityPositionAccessor(); @Override - public CharSequence accessFilePath(CharSequence path) { + public CharSequence accessPath(CharSequence path) { return path; } 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 0ddb5fb63b9e..db96b65ad562 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java @@ -574,7 +574,7 @@ public PositionDeleteStructWriter(StructWriter replacedWriter, Parquet.Positi protected Object get(PositionDelete delete, int index) { switch (index) { case 0: - return positionAccessor.accessFilePath(delete.path()); + return positionAccessor.accessPath(delete.path()); case 1: return positionAccessor.accessPos(delete.pos()); case 2: From fb556e48d67145aecec236d758ac31e72bae287b Mon Sep 17 00:00:00 2001 From: huzheng Date: Thu, 19 Nov 2020 16:40:14 +0800 Subject: [PATCH 36/41] Addressing comments. --- .../java/org/apache/iceberg/avro/Avro.java | 6 +- .../apache/iceberg/deletes/DeletesUtil.java | 33 +++--- .../apache/iceberg/io/BaseDeltaWriter.java | 44 +++----- .../org/apache/iceberg/io/DeltaWriter.java | 8 +- .../iceberg/io/RollingContentFileWriter.java | 11 +- .../iceberg/data/TestGenericDeltaWriter.java | 36 +++--- .../flink/sink/PartitionedFanoutWriter.java | 106 ------------------ .../org/apache/iceberg/parquet/Parquet.java | 6 +- 8 files changed, 77 insertions(+), 173 deletions(-) delete mode 100644 flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedFanoutWriter.java 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 e5d05cf63922..198b479fa053 100644 --- a/core/src/main/java/org/apache/iceberg/avro/Avro.java +++ b/core/src/main/java/org/apache/iceberg/avro/Avro.java @@ -299,12 +299,14 @@ public PositionDeleteWriter buildPositionWriter() throws IOException { meta("delete-type", "position"); - appenderBuilder.schema(DeletesUtil.posDeleteSchema(rowSchema)); - if (rowSchema != null && createWriterFunc != null) { + appenderBuilder.schema(DeletesUtil.pathPosSchema(rowSchema)); + appenderBuilder.createWriterFunc( avroSchema -> new PositionAndRowDatumWriter<>(createWriterFunc.apply(avroSchema))); } else { + 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 index 83c07f00aa05..b1b69b5ff832 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/DeletesUtil.java +++ b/core/src/main/java/org/apache/iceberg/deletes/DeletesUtil.java @@ -21,6 +21,7 @@ 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 { @@ -28,19 +29,25 @@ 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) { - if (rowSchema == null) { - return new Schema( - MetadataColumns.DELETE_FILE_PATH, - MetadataColumns.DELETE_FILE_POS); - } else { - // the appender uses the row schema wrapped with position fields - return new Schema( - MetadataColumns.DELETE_FILE_PATH, - MetadataColumns.DELETE_FILE_POS, - Types.NestedField.optional( - MetadataColumns.DELETE_FILE_ROW_FIELD_ID, "row", rowSchema.asStruct(), - MetadataColumns.DELETE_FILE_ROW_DOC)); - } + return rowSchema == null ? pathPosSchema() : pathPosSchema(rowSchema); } } diff --git a/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java b/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java index 9b596502ca29..85f0f031cdf5 100644 --- a/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java @@ -27,9 +27,9 @@ 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.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.StructLikeMap; @@ -44,7 +44,7 @@ public class BaseDeltaWriter implements DeltaWriter { private final RollingContentFileWriter> posDeleteWriter; private final PositionDelete positionDelete = new PositionDelete<>(); - private final StructLikeMap> insertedRowMap; + private final StructLikeMap insertedRowMap; // Function to convert the generic data to a StructLike. private final Function structLikeFun; @@ -69,13 +69,15 @@ public BaseDeltaWriter(RollingContentFileWriter dataWriter, if (posDeleteWriter == null) { // Only accept INSERT records. - Preconditions.checkArgument(equalityDeleteWriter == null); + Preconditions.checkArgument(equalityDeleteWriter == null, + "Could not accept equality deletes when position delete writer is null."); } if (posDeleteWriter != null && equalityDeleteWriter == null) { // Only accept INSERT records and position deletion. - Preconditions.checkArgument(tableSchema == null); - Preconditions.checkArgument(equalityFieldIds == null); + 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 (equalityDeleteWriter != null) { @@ -100,43 +102,37 @@ public BaseDeltaWriter(RollingContentFileWriter dataWriter, } @Override - public void writeRow(T row) throws IOException { + public void writeRow(T row) { if (enableEqualityDelete()) { FilePos filePos = FilePos.create(dataWriter.currentPath(), dataWriter.currentPos()); - insertedRowMap.compute(structLikeFun.apply(row), (k, v) -> { - if (v == null) { - return Lists.newArrayList(filePos); - } else { - v.add(filePos); - return v; - } - }); + + StructLike key = structLikeFun.apply(row); + FilePos previous = insertedRowMap.putIfAbsent(key, filePos); + ValidationException.check(previous == null, "Detected duplicate insert for %s", key); } dataWriter.write(row); } @Override - public void writeEqualityDelete(T equalityDelete) throws IOException { + public void writeEqualityDelete(T equalityDelete) { if (!enableEqualityDelete()) { throw new UnsupportedOperationException("Could not accept equality deletion."); } - List existing = insertedRowMap.get(structLikeFun.apply(equalityDelete)); + FilePos existing = insertedRowMap.get(structLikeFun.apply(equalityDelete)); if (existing == null) { // Delete the row which have been written by other completed delta writer. equalityDeleteWriter.write(equalityDelete); } else { // Delete the rows which was written in current delta writer. - for (FilePos filePos : existing) { - posDeleteWriter.write(positionDelete.set(filePos.path(), filePos.pos(), null)); - } + posDeleteWriter.write(positionDelete.set(existing.path, existing.pos, null)); } } @Override - public void writePosDelete(CharSequence path, long offset, T row) throws IOException { + public void writePosDelete(CharSequence path, long offset, T row) { if (!enablePosDelete()) { throw new UnsupportedOperationException("Could not accept position deletion."); } @@ -229,14 +225,6 @@ private static FilePos create(CharSequence path, long pos) { return new FilePos(path, pos); } - private CharSequence path() { - return path; - } - - private long pos() { - return pos; - } - @Override public String toString() { return MoreObjects.toStringHelper(this) diff --git a/core/src/main/java/org/apache/iceberg/io/DeltaWriter.java b/core/src/main/java/org/apache/iceberg/io/DeltaWriter.java index d27d37bd6a8a..e90358166bc8 100644 --- a/core/src/main/java/org/apache/iceberg/io/DeltaWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/DeltaWriter.java @@ -31,24 +31,24 @@ public interface DeltaWriter extends Closeable { /** * Write the insert record. */ - void writeRow(T row) throws IOException; + void writeRow(T row); /** * Write the equality delete record. */ - void writeEqualityDelete(T equalityDelete) throws IOException; + void writeEqualityDelete(T equalityDelete); /** * Write the deletion with file path and position into underlying system. */ - default void writePosDelete(CharSequence path, long offset) throws IOException { + 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) throws IOException; + void writePosDelete(CharSequence path, long offset, T row); /** * Abort the writer to clean all generated files. diff --git a/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java b/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java index febad42802ab..5562a235af34 100644 --- a/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java @@ -21,6 +21,7 @@ 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; @@ -70,7 +71,7 @@ public long currentPos() { return currentRows; } - public void write(T record) throws IOException { + public void write(T record) { this.currentFileWriter.write(record); this.currentRows++; @@ -123,9 +124,13 @@ private boolean shouldRollToNewFile() { currentRows % ROWS_DIVISOR == 0 && currentFileWriter.length() >= targetFileSize; } - private void closeCurrent() throws IOException { + private void closeCurrent() { if (currentFileWriter != null) { - currentFileWriter.close(); + try { + currentFileWriter.close(); + } catch (IOException e) { + throw new UncheckedIOException("Failed to close the current file writer", e); + } ContentFileT contentFile = currentFileWriter.toContentFile(); Metrics metrics = currentFileWriter.metrics(); diff --git a/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java b/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java index c9cda56cf955..caab9af5611b 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java +++ b/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java @@ -24,6 +24,7 @@ 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; @@ -32,6 +33,7 @@ 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; @@ -217,7 +219,7 @@ public void testUpsertSameRow() throws IOException { .equalityFieldIds(equalityFieldIds) .rowSchema(table.schema().select("id")) .build(); - DeltaWriter deltaWriter = writerFactory.createDeltaWriter(null, ctxt); + final DeltaWriter deltaWriter1 = writerFactory.createDeltaWriter(null, ctxt); GenericRecord record = GenericRecord.create(SCHEMA); Record record1 = record.copy("id", 1, "data", "aaa"); @@ -228,43 +230,47 @@ public void testUpsertSameRow() throws IOException { Record record6 = record.copy("id", 1, "data", "fff"); Record record7 = record.copy("id", 1, "data", "ggg"); - deltaWriter.writeRow(record1); - deltaWriter.writeRow(record2); + deltaWriter1.writeRow(record1); + AssertHelpers.assertThrows("Detect duplicated keys", ValidationException.class, + () -> deltaWriter1.writeRow(record2)); // Commit the transaction. - WriterResult result = deltaWriter.complete(); + WriterResult result = deltaWriter1.complete(); Assert.assertEquals(result.dataFiles().length, 1); Assert.assertEquals(result.deleteFiles().length, 0); commitTransaction(result); - assertTableRecords(ImmutableSet.of(record1, record2)); + assertTableRecords(ImmutableSet.of(record1)); - deltaWriter = writerFactory.createDeltaWriter(null, ctxt); + final DeltaWriter deltaWriter2 = writerFactory.createDeltaWriter(null, ctxt); // UPSERT (1, "ccc") - deltaWriter.writeEqualityDelete(record3); - deltaWriter.writeRow(record3); + deltaWriter2.writeEqualityDelete(record3); + deltaWriter2.writeRow(record3); // INSERT (1, "ddd") // INSERT (1, "eee") - deltaWriter.writeRow(record4); - deltaWriter.writeRow(record5); + AssertHelpers.assertThrows("Detect duplicated keys", ValidationException.class, + () -> deltaWriter2.writeRow(record4)); + AssertHelpers.assertThrows("Detect duplicated keys", ValidationException.class, + () -> deltaWriter2.writeRow(record5)); // UPSERT (1, "fff") - deltaWriter.writeEqualityDelete(record6); - deltaWriter.writeRow(record6); + deltaWriter2.writeEqualityDelete(record6); + deltaWriter2.writeRow(record6); // INSERT (1, "ggg") - deltaWriter.writeRow(record7); + AssertHelpers.assertThrows("Detect duplicated keys", ValidationException.class, + () -> deltaWriter2.writeRow(record7)); // Commit the transaction. - result = deltaWriter.complete(); + 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, record7)); + assertTableRecords(ImmutableSet.of(record6)); } private void assertTableRecords(Set expectedRecords) { 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 d451d3fcbcdf..000000000000 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedFanoutWriter.java +++ /dev/null @@ -1,106 +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.ContentFileWriterFactory; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.OutputFileFactory; -import org.apache.iceberg.io.RollingContentFileWriter; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriterResult; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; - -abstract class PartitionedFanoutWriter implements TaskWriter { - - private final FileFormat format; - private final OutputFileFactory fileFactory; - private final FileIO io; - private final long targetFileSize; - private final ContentFileWriterFactory writerFactory; - - private final Map> writers = Maps.newHashMap(); - - PartitionedFanoutWriter(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; - } - - /** - * 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); - - RollingContentFileWriter 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 RollingContentFileWriter<>(copiedKey, format, fileFactory, io, targetFileSize, writerFactory); - writers.put(copiedKey, writer); - } - - writer.write(row); - } - - @Override - public void close() throws IOException { - if (!writers.isEmpty()) { - for (RollingContentFileWriter writer : writers.values()) { - writer.close(); - } - } - } - - @Override - public void abort() throws IOException { - close(); - - for (RollingContentFileWriter writer : writers.values()) { - writer.abort(); - } - } - - @Override - public WriterResult complete() throws IOException { - close(); - - WriterResult.Builder builder = WriterResult.builder(); - for (RollingContentFileWriter writer : writers.values()) { - builder.add(writer.complete()); - } - - return builder.build(); - } -} diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java index bbe0c04f36b1..8fe4c9619bb8 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -405,9 +405,9 @@ public PositionDeleteWriter buildPositionWriter(PositionAccessor po meta("delete-type", "position"); - appenderBuilder.schema(DeletesUtil.posDeleteSchema(rowSchema)); - if (createWriterFunc != null) { + appenderBuilder.schema(DeletesUtil.posDeleteSchema(rowSchema)); + appenderBuilder.createWriterFunc(parquetSchema -> { ParquetValueWriter writer = createWriterFunc.apply(parquetSchema); if (writer instanceof StructWriter) { @@ -418,6 +418,8 @@ public PositionDeleteWriter buildPositionWriter(PositionAccessor po }); } else { + appenderBuilder.schema(DeletesUtil.pathPosSchema()); + appenderBuilder.createWriterFunc(parquetSchema -> new PositionDeleteStructWriter((StructWriter) GenericParquetWriter.buildWriter(parquetSchema), positionAccessor)); From 7ce7bc5f462c2814a6884011a4df932c0348df7d Mon Sep 17 00:00:00 2001 From: huzheng Date: Thu, 19 Nov 2020 19:31:41 +0800 Subject: [PATCH 37/41] Separate the posDeleteRowSchema and eqDeleteRowSchema. --- .../org/apache/iceberg/ContentFileWriter.java | 5 -- .../org/apache/iceberg/DataFileWriter.java | 5 -- .../iceberg/deletes/EqualityDeleteWriter.java | 6 -- .../iceberg/deletes/PositionDeleteWriter.java | 6 -- .../apache/iceberg/io/BaseDeltaWriter.java | 30 ++++--- .../apache/iceberg/io/DeltaWriterFactory.java | 40 ++++++--- .../iceberg/io/RollingContentFileWriter.java | 6 +- .../data/GenericDeltaWriterFactory.java | 4 +- .../iceberg/data/TestGenericDeltaWriter.java | 49 ++++++++++- .../apache/iceberg/flink/RowDataWrapper.java | 5 ++ .../flink/sink/FlinkDeltaWriterFactory.java | 4 +- .../iceberg/flink/sink/RowDataTaskWriter.java | 2 +- .../flink/sink/TestFlinkDeltaWriter.java | 83 +++++++++++++++---- 13 files changed, 169 insertions(+), 76 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/ContentFileWriter.java b/core/src/main/java/org/apache/iceberg/ContentFileWriter.java index c1c0a546fa6f..c7b6838d2220 100644 --- a/core/src/main/java/org/apache/iceberg/ContentFileWriter.java +++ b/core/src/main/java/org/apache/iceberg/ContentFileWriter.java @@ -36,11 +36,6 @@ default void writeAll(Iterable values) { writeAll(values.iterator()); } - /** - * Returns {@link Metrics} for this file. Only valid after the file is closed. - */ - Metrics metrics(); - /** * Returns the length of this file. */ diff --git a/core/src/main/java/org/apache/iceberg/DataFileWriter.java b/core/src/main/java/org/apache/iceberg/DataFileWriter.java index 9cc54ffdad8d..4f2438482f54 100644 --- a/core/src/main/java/org/apache/iceberg/DataFileWriter.java +++ b/core/src/main/java/org/apache/iceberg/DataFileWriter.java @@ -50,11 +50,6 @@ public void write(T row) { appender.add(row); } - @Override - public Metrics metrics() { - return appender.metrics(); - } - @Override public long length() { return appender.length(); 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 75ea9fd91d69..b339881c83c9 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/EqualityDeleteWriter.java +++ b/core/src/main/java/org/apache/iceberg/deletes/EqualityDeleteWriter.java @@ -25,7 +25,6 @@ import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.FileMetadata; -import org.apache.iceberg.Metrics; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.StructLike; import org.apache.iceberg.encryption.EncryptionKeyMetadata; @@ -64,11 +63,6 @@ public void write(T row) { appender.add(row); } - @Override - public Metrics metrics() { - return appender.metrics(); - } - @Override public void close() throws IOException { if (deleteFile == null) { 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 7727235d627e..227d85f987c1 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteWriter.java +++ b/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteWriter.java @@ -26,7 +26,6 @@ import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.FileMetadata; -import org.apache.iceberg.Metrics; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.StructLike; import org.apache.iceberg.encryption.EncryptionKeyMetadata; @@ -91,11 +90,6 @@ public void write(PositionDelete record) { delete(record.path(), record.pos(), record.row()); } - @Override - public Metrics metrics() { - return appender.metrics(); - } - @Override public long length() { return appender.length(); diff --git a/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java b/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java index 85f0f031cdf5..1999fad00d42 100644 --- a/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java @@ -44,7 +44,7 @@ public class BaseDeltaWriter implements DeltaWriter { private final RollingContentFileWriter> posDeleteWriter; private final PositionDelete positionDelete = new PositionDelete<>(); - private final StructLikeMap insertedRowMap; + private final StructLikeMap insertedRowMap; // Function to convert the generic data to a StructLike. private final Function structLikeFun; @@ -104,10 +104,10 @@ public BaseDeltaWriter(RollingContentFileWriter dataWriter, @Override public void writeRow(T row) { if (enableEqualityDelete()) { - FilePos filePos = FilePos.create(dataWriter.currentPath(), dataWriter.currentPos()); + RowOffset rowOffset = RowOffset.create(dataWriter.currentPath(), dataWriter.currentRows()); StructLike key = structLikeFun.apply(row); - FilePos previous = insertedRowMap.putIfAbsent(key, filePos); + RowOffset previous = insertedRowMap.putIfAbsent(key, rowOffset); ValidationException.check(previous == null, "Detected duplicate insert for %s", key); } @@ -120,14 +120,18 @@ public void writeEqualityDelete(T equalityDelete) { throw new UnsupportedOperationException("Could not accept equality deletion."); } - FilePos existing = insertedRowMap.get(structLikeFun.apply(equalityDelete)); + StructLike key = structLikeFun.apply(equalityDelete); + RowOffset existing = insertedRowMap.get(key); if (existing == null) { // Delete the row which have been written by other completed delta writer. equalityDeleteWriter.write(equalityDelete); } else { - // Delete the rows which was written in current delta writer. - posDeleteWriter.write(positionDelete.set(existing.path, existing.pos, null)); + // 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); } } @@ -212,24 +216,24 @@ private boolean enablePosDelete() { return posDeleteWriter != null; } - private static class FilePos { + private static class RowOffset { private final CharSequence path; - private final long pos; + private final long rowId; - private FilePos(CharSequence path, long pos) { + private RowOffset(CharSequence path, long rowId) { this.path = path; - this.pos = pos; + this.rowId = rowId; } - private static FilePos create(CharSequence path, long pos) { - return new FilePos(path, pos); + 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("pos", pos) + .add("pos", rowId) .toString(); } } diff --git a/core/src/main/java/org/apache/iceberg/io/DeltaWriterFactory.java b/core/src/main/java/org/apache/iceberg/io/DeltaWriterFactory.java index 0a986fbb9d50..f56ccf1ce3c0 100644 --- a/core/src/main/java/org/apache/iceberg/io/DeltaWriterFactory.java +++ b/core/src/main/java/org/apache/iceberg/io/DeltaWriterFactory.java @@ -46,14 +46,17 @@ class Context { private final boolean allowPosDelete; private final boolean allowEqualityDelete; private final List equalityFieldIds; - private final Schema rowSchema; + private final Schema posDeleteRowSchema; + private final Schema eqDeleteRowSchema; - private Context(boolean allowPosDelete, boolean allowEqualityDelete, List equalityFieldIds, - Schema rowSchema) { + private Context(boolean allowPosDelete, boolean allowEqualityDelete, + List equalityFieldIds, Schema eqDeleteRowSchema, + Schema posDeleteRowSchema) { this.allowPosDelete = allowPosDelete; this.allowEqualityDelete = allowEqualityDelete; this.equalityFieldIds = equalityFieldIds; - this.rowSchema = rowSchema; + this.eqDeleteRowSchema = eqDeleteRowSchema; + this.posDeleteRowSchema = posDeleteRowSchema; } public boolean allowPosDelete() { @@ -68,8 +71,12 @@ public List equalityFieldIds() { return equalityFieldIds; } - public Schema rowSchema() { - return rowSchema; + public Schema posDeleteRowSchema() { + return posDeleteRowSchema; + } + + public Schema eqDeleteRowSchema() { + return eqDeleteRowSchema; } public static Builder builder() { @@ -80,7 +87,8 @@ public static class Builder { private boolean allowPosDelete = false; private boolean allowEqualityDelete = false; private List equalityFieldIds = ImmutableList.of(); - private Schema rowSchema; + private Schema eqDeleteRowSchema; + private Schema posDeleteRowSchema; public Builder allowPosDelete(boolean enable) { this.allowPosDelete = enable; @@ -97,23 +105,29 @@ public Builder equalityFieldIds(List newEqualityFieldIds) { return this; } - public Builder rowSchema(Schema newRowSchema) { - this.rowSchema = newRowSchema; + 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(rowSchema, "Row schema 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(rowSchema.findField(fieldId), - "Missing field id %s in provided row schema: %s", fieldId, rowSchema); + Preconditions.checkNotNull(eqDeleteRowSchema.findField(fieldId), + "Missing field id %s in provided row schema: %s", fieldId, eqDeleteRowSchema); } } - return new Context(allowPosDelete, allowEqualityDelete, equalityFieldIds, rowSchema); + return new Context(allowPosDelete, allowEqualityDelete, equalityFieldIds, eqDeleteRowSchema, + posDeleteRowSchema); } } } diff --git a/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java b/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java index 5562a235af34..5efe56419b8c 100644 --- a/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java @@ -26,7 +26,6 @@ import org.apache.iceberg.ContentFileWriter; import org.apache.iceberg.ContentFileWriterFactory; import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Metrics; import org.apache.iceberg.PartitionKey; import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; @@ -67,7 +66,7 @@ public CharSequence currentPath() { return currentFile.encryptingOutputFile().location(); } - public long currentPos() { + public long currentRows() { return currentRows; } @@ -133,10 +132,9 @@ private void closeCurrent() { } ContentFileT contentFile = currentFileWriter.toContentFile(); - Metrics metrics = currentFileWriter.metrics(); this.currentFileWriter = null; - if (metrics.recordCount() == 0L) { + if (currentRows == 0L) { io.deleteFile(currentFile.encryptingOutputFile()); } else if (contentFile instanceof ContentFile) { resultBuilder.add((ContentFile) contentFile); diff --git a/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java b/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java index 903b299ecdd1..1b82118352a1 100644 --- a/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java +++ b/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java @@ -86,7 +86,7 @@ public DeltaWriter createDeltaWriter(PartitionKey partitionKey, Context RollingContentFileWriter> posDeleteWriter = new RollingContentFileWriter<>(partitionKey, - format, fileFactory, io, targetFileSize, createPosDeleteWriterFactory(ctxt.rowSchema())); + format, fileFactory, io, targetFileSize, createPosDeleteWriterFactory(ctxt.posDeleteRowSchema())); if (ctxt.allowPosDelete() && !ctxt.allowEqualityDelete()) { return new BaseDeltaWriter<>(dataWriter, posDeleteWriter); @@ -98,7 +98,7 @@ public DeltaWriter createDeltaWriter(PartitionKey partitionKey, Context RollingContentFileWriter eqDeleteWriter = new RollingContentFileWriter<>(partitionKey, format, fileFactory, io, targetFileSize, - createEqualityDeleteWriterFactory(ctxt.equalityFieldIds(), ctxt.rowSchema())); + createEqualityDeleteWriterFactory(ctxt.equalityFieldIds(), ctxt.eqDeleteRowSchema())); return new BaseDeltaWriter<>(dataWriter, posDeleteWriter, eqDeleteWriter, schema, ctxt.equalityFieldIds(), diff --git a/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java b/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java index caab9af5611b..54e567880877 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java +++ b/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java @@ -124,7 +124,7 @@ public void testWriteEqualityDelete() throws IOException { DeltaWriterFactory.Context ctxt = DeltaWriterFactory.Context.builder() .allowEqualityDelete(true) .equalityFieldIds(equalityFieldIds) - .rowSchema(table.schema()) + .eqDeleteRowSchema(table.schema()) .build(); // TODO More unit tests to test the partitioned case. @@ -161,6 +161,50 @@ public void testWriteEqualityDelete() throws IOException { assertTableRecords(ImmutableSet.of()); } + @Test + public void testEqualityDeleteSameRow() throws IOException { + 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() throws IOException { DeltaWriterFactory writerFactory = createDeltaWriterFactory(); @@ -217,7 +261,8 @@ public void testUpsertSameRow() throws IOException { DeltaWriterFactory.Context ctxt = DeltaWriterFactory.Context.builder() .allowEqualityDelete(true) .equalityFieldIds(equalityFieldIds) - .rowSchema(table.schema().select("id")) + .eqDeleteRowSchema(table.schema().select("id")) + .posDeleteRowSchema(table.schema()) .build(); final DeltaWriter deltaWriter1 = writerFactory.createDeltaWriter(null, ctxt); 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..490faef3ff6c 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java +++ b/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java @@ -135,4 +135,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/FlinkDeltaWriterFactory.java b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkDeltaWriterFactory.java index b966c308be72..18bcfcfaa665 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkDeltaWriterFactory.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkDeltaWriterFactory.java @@ -97,7 +97,7 @@ public DeltaWriter createDeltaWriter(PartitionKey partitionKey, Context RollingContentFileWriter> posDeleteWriter = new RollingContentFileWriter<>(partitionKey, - format, fileFactory, io, targetFileSize, createPosDeleteWriterFactory(ctxt.rowSchema())); + format, fileFactory, io, targetFileSize, createPosDeleteWriterFactory(ctxt.posDeleteRowSchema())); if (ctxt.allowPosDelete() && !ctxt.allowEqualityDelete()) { return new BaseDeltaWriter<>(dataWriter, posDeleteWriter); @@ -109,7 +109,7 @@ public DeltaWriter createDeltaWriter(PartitionKey partitionKey, Context RollingContentFileWriter eqDeleteWriter = new RollingContentFileWriter<>(partitionKey, format, fileFactory, io, targetFileSize, - createEqualityDeleteWriterFactory(ctxt.equalityFieldIds(), ctxt.rowSchema())); + createEqualityDeleteWriterFactory(ctxt.equalityFieldIds(), ctxt.eqDeleteRowSchema())); // Define flink's as struct like function. RowDataWrapper asStructLike = new RowDataWrapper(flinkSchema, schema.asStruct()); 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 index c38cdd72731d..94dc8b68287d 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriter.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriter.java @@ -73,7 +73,7 @@ class RowDataTaskWriter implements TaskWriter { this.ctxt = DeltaWriterFactory.Context.builder() .allowEqualityDelete(true) .equalityFieldIds(equalityFieldIds) - .rowSchema(TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds))) + .eqDeleteRowSchema(TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds))) .build(); } else { this.ctxt = DeltaWriterFactory.Context.builder().build(); 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 index 5c1fa54eec35..d65a2d02c58c 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkDeltaWriter.java +++ b/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkDeltaWriter.java @@ -26,6 +26,7 @@ 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; @@ -37,6 +38,7 @@ 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; @@ -132,7 +134,7 @@ public void testWriteEqualityDelete() throws IOException { DeltaWriterFactory.Context ctxt = DeltaWriterFactory.Context.builder() .allowEqualityDelete(true) .equalityFieldIds(equalityFieldIds) - .rowSchema(table.schema().select("id")) + .eqDeleteRowSchema(table.schema().select("id")) .build(); // TODO More unit tests to test the partitioned case. @@ -168,6 +170,49 @@ public void testWriteEqualityDelete() throws IOException { assertTableRecords(ImmutableSet.of()); } + @Test + public void testEqualityDeleteSameRow() throws IOException { + 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() throws IOException { DeltaWriterFactory writerFactory = createDeltaWriterFactory(); @@ -223,9 +268,9 @@ public void testUpsertSameRow() throws IOException { DeltaWriterFactory.Context ctxt = DeltaWriterFactory.Context.builder() .allowEqualityDelete(true) .equalityFieldIds(equalityFieldIds) - .rowSchema(table.schema().select("id")) + .eqDeleteRowSchema(table.schema().select("id")) .build(); - DeltaWriter deltaWriter = writerFactory.createDeltaWriter(null, ctxt); + DeltaWriter deltaWriter1 = writerFactory.createDeltaWriter(null, ctxt); RowData record1 = createRowData(1, "aaa"); RowData record2 = createRowData(1, "bbb"); @@ -235,43 +280,47 @@ public void testUpsertSameRow() throws IOException { RowData record6 = createRowData(1, "fff"); RowData record7 = createRowData(1, "ggg"); - deltaWriter.writeRow(record1); - deltaWriter.writeRow(record2); + deltaWriter1.writeRow(record1); + AssertHelpers.assertThrows("Duplicated keys", ValidationException.class, + () -> deltaWriter1.writeRow(record2)); // Commit the transaction. - WriterResult result = deltaWriter.complete(); + WriterResult result = deltaWriter1.complete(); Assert.assertEquals(result.dataFiles().length, 1); Assert.assertEquals(result.deleteFiles().length, 0); commitTransaction(result); - assertTableRecords(ImmutableSet.of(record1, record2)); + assertTableRecords(ImmutableSet.of(record1)); - deltaWriter = writerFactory.createDeltaWriter(null, ctxt); + DeltaWriter deltaWriter2 = writerFactory.createDeltaWriter(null, ctxt); // UPSERT (1, "ccc") - deltaWriter.writeEqualityDelete(record3); - deltaWriter.writeRow(record3); + deltaWriter2.writeEqualityDelete(record3); + deltaWriter2.writeRow(record3); // INSERT (1, "ddd") // INSERT (1, "eee") - deltaWriter.writeRow(record4); - deltaWriter.writeRow(record5); + AssertHelpers.assertThrows("Duplicated keys", ValidationException.class, + () -> deltaWriter2.writeRow(record4)); + AssertHelpers.assertThrows("Duplicated keys", ValidationException.class, + () -> deltaWriter2.writeRow(record5)); // UPSERT (1, "fff") - deltaWriter.writeEqualityDelete(record6); - deltaWriter.writeRow(record6); + deltaWriter2.writeEqualityDelete(record6); + deltaWriter2.writeRow(record6); // INSERT (1, "ggg") - deltaWriter.writeRow(record7); + AssertHelpers.assertThrows("Duplicated keys", ValidationException.class, + () -> deltaWriter2.writeRow(record7)); // Commit the transaction. - result = deltaWriter.complete(); + 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, record7)); + assertTableRecords(ImmutableSet.of(record6)); } private void assertTableRecords(Set expectedRowDataSet) { From dd944663bd13dc5153617acb4b8e23d49cdbb027 Mon Sep 17 00:00:00 2001 From: huzheng Date: Thu, 19 Nov 2020 21:47:49 +0800 Subject: [PATCH 38/41] Fix the compile errors after rebase. --- .../iceberg/flink/source/TestFlinkMergingMetrics.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) 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); } From 3ba759bcb489b57ddfae1c686c3e03b3f5f821ac Mon Sep 17 00:00:00 2001 From: huzheng Date: Thu, 19 Nov 2020 22:28:17 +0800 Subject: [PATCH 39/41] Create the asCopiedKey and asKey in BaseDeltaWriter to avoid messing up keys in insertedRowMap. --- .../apache/iceberg/io/BaseDeltaWriter.java | 30 ++++--------- .../data/GenericDeltaWriterFactory.java | 37 ++++++++++++++-- .../apache/iceberg/flink/RowDataWrapper.java | 9 ++++ .../flink/sink/FlinkDeltaWriterFactory.java | 43 ++++++++++++++++--- 4 files changed, 87 insertions(+), 32 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java b/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java index 1999fad00d42..bd351a64f6f8 100644 --- a/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.util.List; -import java.util.function.Function; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.Schema; @@ -36,7 +35,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class BaseDeltaWriter implements DeltaWriter { +public abstract class BaseDeltaWriter implements DeltaWriter { private static final Logger LOG = LoggerFactory.getLogger(BaseDeltaWriter.class); private final RollingContentFileWriter dataWriter; @@ -46,24 +45,11 @@ public class BaseDeltaWriter implements DeltaWriter { private final PositionDelete positionDelete = new PositionDelete<>(); private final StructLikeMap insertedRowMap; - // Function to convert the generic data to a StructLike. - private final Function structLikeFun; - - public BaseDeltaWriter(RollingContentFileWriter dataWriter) { - this(dataWriter, null); - } - - public BaseDeltaWriter(RollingContentFileWriter dataWriter, - RollingContentFileWriter> posDeleteWriter) { - this(dataWriter, posDeleteWriter, null, null, null, null); - } - public BaseDeltaWriter(RollingContentFileWriter dataWriter, RollingContentFileWriter> posDeleteWriter, RollingContentFileWriter equalityDeleteWriter, Schema tableSchema, - List equalityFieldIds, - Function structLikeFun) { + List equalityFieldIds) { Preconditions.checkNotNull(dataWriter, "Data writer should always not be null."); @@ -86,14 +72,11 @@ public BaseDeltaWriter(RollingContentFileWriter dataWriter, "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"); - Preconditions.checkNotNull(structLikeFun, "StructLike function shouldn't be null"); Schema deleteSchema = TypeUtil.select(tableSchema, Sets.newHashSet(equalityFieldIds)); this.insertedRowMap = StructLikeMap.create(deleteSchema.asStruct()); - this.structLikeFun = structLikeFun; } else { this.insertedRowMap = null; - this.structLikeFun = null; } this.dataWriter = dataWriter; @@ -101,12 +84,17 @@ public BaseDeltaWriter(RollingContentFileWriter dataWriter, this.posDeleteWriter = posDeleteWriter; } + protected abstract StructLike asKey(T row); + + protected abstract StructLike asCopiedKey(T row); + @Override public void writeRow(T row) { if (enableEqualityDelete()) { RowOffset rowOffset = RowOffset.create(dataWriter.currentPath(), dataWriter.currentRows()); - StructLike key = structLikeFun.apply(row); + // Copy the key to avoid messing up keys in the insertedRowMap. + StructLike key = asCopiedKey(row); RowOffset previous = insertedRowMap.putIfAbsent(key, rowOffset); ValidationException.check(previous == null, "Detected duplicate insert for %s", key); } @@ -120,7 +108,7 @@ public void writeEqualityDelete(T equalityDelete) { throw new UnsupportedOperationException("Could not accept equality deletion."); } - StructLike key = structLikeFun.apply(equalityDelete); + StructLike key = asKey(equalityDelete); RowOffset existing = insertedRowMap.get(key); if (existing == null) { diff --git a/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java b/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java index 1b82118352a1..e89289786afb 100644 --- a/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java +++ b/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java @@ -32,6 +32,7 @@ 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; @@ -81,7 +82,7 @@ public DeltaWriter createDeltaWriter(PartitionKey partitionKey, Context format, fileFactory, io, targetFileSize, createDataFileWriterFactory()); if (!ctxt.allowPosDelete() && !ctxt.allowEqualityDelete()) { - return new BaseDeltaWriter<>(dataWriter); + return new GenericDeltaWriter(dataWriter); } RollingContentFileWriter> posDeleteWriter = @@ -89,7 +90,7 @@ public DeltaWriter createDeltaWriter(PartitionKey partitionKey, Context format, fileFactory, io, targetFileSize, createPosDeleteWriterFactory(ctxt.posDeleteRowSchema())); if (ctxt.allowPosDelete() && !ctxt.allowEqualityDelete()) { - return new BaseDeltaWriter<>(dataWriter, posDeleteWriter); + return new GenericDeltaWriter(dataWriter, posDeleteWriter); } Preconditions.checkState(ctxt.allowEqualityDelete(), "Should always allow equality-delete here."); @@ -101,8 +102,7 @@ public DeltaWriter createDeltaWriter(PartitionKey partitionKey, Context createEqualityDeleteWriterFactory(ctxt.equalityFieldIds(), ctxt.eqDeleteRowSchema())); - return new BaseDeltaWriter<>(dataWriter, posDeleteWriter, eqDeleteWriter, schema, ctxt.equalityFieldIds(), - t -> t); + return new GenericDeltaWriter(dataWriter, posDeleteWriter, eqDeleteWriter, schema, ctxt.equalityFieldIds()); } @Override @@ -208,4 +208,33 @@ public ContentFileWriterFactory> createPosDel } }; } + + private static class GenericDeltaWriter extends BaseDeltaWriter { + + GenericDeltaWriter(RollingContentFileWriter dataWriter) { + this(dataWriter, null); + } + + GenericDeltaWriter(RollingContentFileWriter dataWriter, + RollingContentFileWriter> posDeleteWriter) { + this(dataWriter, posDeleteWriter, null, null, null); + } + + GenericDeltaWriter(RollingContentFileWriter dataWriter, + RollingContentFileWriter> posDeleteWriter, + RollingContentFileWriter equalityDeleteWriter, Schema tableSchema, + List equalityFieldIds) { + super(dataWriter, posDeleteWriter, equalityDeleteWriter, tableSchema, equalityFieldIds); + } + + @Override + protected StructLike asKey(Record row) { + return row; + } + + @Override + protected StructLike asCopiedKey(Record row) { + return row.copy(); + } + } } 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 490faef3ff6c..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; 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 index 18bcfcfaa665..44b0db79c049 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkDeltaWriterFactory.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkDeltaWriterFactory.java @@ -37,6 +37,7 @@ 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; @@ -92,7 +93,7 @@ public DeltaWriter createDeltaWriter(PartitionKey partitionKey, Context format, fileFactory, io, targetFileSize, createDataFileWriterFactory()); if (!ctxt.allowPosDelete() && !ctxt.allowEqualityDelete()) { - return new BaseDeltaWriter<>(dataWriter); + return new RowDataDeltaWriter(dataWriter); } RollingContentFileWriter> posDeleteWriter = @@ -100,7 +101,7 @@ public DeltaWriter createDeltaWriter(PartitionKey partitionKey, Context format, fileFactory, io, targetFileSize, createPosDeleteWriterFactory(ctxt.posDeleteRowSchema())); if (ctxt.allowPosDelete() && !ctxt.allowEqualityDelete()) { - return new BaseDeltaWriter<>(dataWriter, posDeleteWriter); + return new RowDataDeltaWriter(dataWriter, posDeleteWriter); } Preconditions.checkState(ctxt.allowEqualityDelete(), "Should always allow equality-delete here."); @@ -111,11 +112,7 @@ public DeltaWriter createDeltaWriter(PartitionKey partitionKey, Context format, fileFactory, io, targetFileSize, createEqualityDeleteWriterFactory(ctxt.equalityFieldIds(), ctxt.eqDeleteRowSchema())); - // Define flink's as struct like function. - RowDataWrapper asStructLike = new RowDataWrapper(flinkSchema, schema.asStruct()); - - return new BaseDeltaWriter<>(dataWriter, posDeleteWriter, eqDeleteWriter, schema, ctxt.equalityFieldIds(), - asStructLike::wrap); + return new RowDataDeltaWriter(dataWriter, posDeleteWriter, eqDeleteWriter, schema, ctxt.equalityFieldIds()); } @Override @@ -231,6 +228,38 @@ public ContentFileWriterFactory> createPosDe }; } + private class RowDataDeltaWriter extends BaseDeltaWriter { + private final RowDataWrapper rowDataWrapper; + + RowDataDeltaWriter(RollingContentFileWriter dataWriter) { + this(dataWriter, null); + } + + RowDataDeltaWriter(RollingContentFileWriter dataWriter, + RollingContentFileWriter> posDeleteWriter) { + this(dataWriter, posDeleteWriter, null, null, null); + } + + RowDataDeltaWriter(RollingContentFileWriter dataWriter, + RollingContentFileWriter> posDeleteWriter, + RollingContentFileWriter 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(); From 957e3beb9530428f8a19ec36f598c0d0a631940e Mon Sep 17 00:00:00 2001 From: huzheng Date: Fri, 20 Nov 2020 00:03:52 +0800 Subject: [PATCH 40/41] Create a RollingPosDeleteWriter to simpify the position delete writers. --- .../apache/iceberg/io/BaseDeltaWriter.java | 4 +- .../apache/iceberg/io/DeltaWriterFactory.java | 18 +++++++ .../iceberg/io/RollingContentFileWriter.java | 7 +++ .../iceberg/io/RollingPosDeleteWriter.java | 52 +++++++++++++++++++ .../data/GenericDeltaWriterFactory.java | 11 ++-- .../flink/sink/FlinkDeltaWriterFactory.java | 10 ++-- 6 files changed, 89 insertions(+), 13 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/io/RollingPosDeleteWriter.java diff --git a/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java b/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java index bd351a64f6f8..2854acceb0a9 100644 --- a/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java @@ -40,13 +40,13 @@ public abstract class BaseDeltaWriter implements DeltaWriter { private final RollingContentFileWriter dataWriter; private final RollingContentFileWriter equalityDeleteWriter; - private final RollingContentFileWriter> posDeleteWriter; + private final RollingPosDeleteWriter posDeleteWriter; private final PositionDelete positionDelete = new PositionDelete<>(); private final StructLikeMap insertedRowMap; public BaseDeltaWriter(RollingContentFileWriter dataWriter, - RollingContentFileWriter> posDeleteWriter, + RollingPosDeleteWriter posDeleteWriter, RollingContentFileWriter equalityDeleteWriter, Schema tableSchema, List equalityFieldIds) { diff --git a/core/src/main/java/org/apache/iceberg/io/DeltaWriterFactory.java b/core/src/main/java/org/apache/iceberg/io/DeltaWriterFactory.java index f56ccf1ce3c0..ebf8450d1829 100644 --- a/core/src/main/java/org/apache/iceberg/io/DeltaWriterFactory.java +++ b/core/src/main/java/org/apache/iceberg/io/DeltaWriterFactory.java @@ -29,17 +29,35 @@ 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 { diff --git a/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java b/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java index 5efe56419b8c..a1d9bc533fb7 100644 --- a/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java @@ -123,8 +123,15 @@ private boolean shouldRollToNewFile() { 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) { 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/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java b/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java index e89289786afb..bbd2d312e5af 100644 --- a/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java +++ b/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java @@ -45,6 +45,7 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.OutputFileFactory; import org.apache.iceberg.io.RollingContentFileWriter; +import org.apache.iceberg.io.RollingPosDeleteWriter; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -85,9 +86,8 @@ public DeltaWriter createDeltaWriter(PartitionKey partitionKey, Context return new GenericDeltaWriter(dataWriter); } - RollingContentFileWriter> posDeleteWriter = - new RollingContentFileWriter<>(partitionKey, - format, fileFactory, io, targetFileSize, createPosDeleteWriterFactory(ctxt.posDeleteRowSchema())); + RollingPosDeleteWriter posDeleteWriter = new RollingPosDeleteWriter<>(partitionKey, + format, fileFactory, io, targetFileSize, createPosDeleteWriterFactory(ctxt.posDeleteRowSchema())); if (ctxt.allowPosDelete() && !ctxt.allowEqualityDelete()) { return new GenericDeltaWriter(dataWriter, posDeleteWriter); @@ -101,7 +101,6 @@ public DeltaWriter createDeltaWriter(PartitionKey partitionKey, Context format, fileFactory, io, targetFileSize, createEqualityDeleteWriterFactory(ctxt.equalityFieldIds(), ctxt.eqDeleteRowSchema())); - return new GenericDeltaWriter(dataWriter, posDeleteWriter, eqDeleteWriter, schema, ctxt.equalityFieldIds()); } @@ -216,12 +215,12 @@ private static class GenericDeltaWriter extends BaseDeltaWriter { } GenericDeltaWriter(RollingContentFileWriter dataWriter, - RollingContentFileWriter> posDeleteWriter) { + RollingPosDeleteWriter posDeleteWriter) { this(dataWriter, posDeleteWriter, null, null, null); } GenericDeltaWriter(RollingContentFileWriter dataWriter, - RollingContentFileWriter> posDeleteWriter, + RollingPosDeleteWriter posDeleteWriter, RollingContentFileWriter equalityDeleteWriter, Schema tableSchema, List equalityFieldIds) { super(dataWriter, posDeleteWriter, equalityDeleteWriter, tableSchema, equalityFieldIds); 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 index 44b0db79c049..e5daae313915 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkDeltaWriterFactory.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkDeltaWriterFactory.java @@ -53,6 +53,7 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.OutputFileFactory; import org.apache.iceberg.io.RollingContentFileWriter; +import org.apache.iceberg.io.RollingPosDeleteWriter; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -96,9 +97,8 @@ public DeltaWriter createDeltaWriter(PartitionKey partitionKey, Context return new RowDataDeltaWriter(dataWriter); } - RollingContentFileWriter> posDeleteWriter = - new RollingContentFileWriter<>(partitionKey, - format, fileFactory, io, targetFileSize, createPosDeleteWriterFactory(ctxt.posDeleteRowSchema())); + RollingPosDeleteWriter posDeleteWriter = new RollingPosDeleteWriter<>(partitionKey, + format, fileFactory, io, targetFileSize, createPosDeleteWriterFactory(ctxt.posDeleteRowSchema())); if (ctxt.allowPosDelete() && !ctxt.allowEqualityDelete()) { return new RowDataDeltaWriter(dataWriter, posDeleteWriter); @@ -236,12 +236,12 @@ private class RowDataDeltaWriter extends BaseDeltaWriter { } RowDataDeltaWriter(RollingContentFileWriter dataWriter, - RollingContentFileWriter> posDeleteWriter) { + RollingPosDeleteWriter posDeleteWriter) { this(dataWriter, posDeleteWriter, null, null, null); } RowDataDeltaWriter(RollingContentFileWriter dataWriter, - RollingContentFileWriter> posDeleteWriter, + RollingPosDeleteWriter posDeleteWriter, RollingContentFileWriter equalityDeleteWriter, Schema tableSchema, List equalityFieldIds) { super(dataWriter, posDeleteWriter, equalityDeleteWriter, tableSchema, equalityFieldIds); From 72940c331965a38353d15522b3b622f18f0efb2c Mon Sep 17 00:00:00 2001 From: huzheng Date: Fri, 20 Nov 2020 14:40:49 +0800 Subject: [PATCH 41/41] Minor fixes --- .../org/apache/iceberg/ContentFileWriter.java | 2 +- .../iceberg/ContentFileWriterFactory.java | 18 +++- .../apache/iceberg/io/BaseDeltaWriter.java | 85 +++++++------------ .../iceberg/io/DataFileWriterFactory.java | 2 +- .../org/apache/iceberg/io/DeltaWriter.java | 5 +- .../apache/iceberg/io/DeltaWriterFactory.java | 2 +- .../apache/iceberg/io/PartitionedWriter.java | 7 +- .../iceberg/io/RollingContentFileWriter.java | 18 ++-- .../iceberg/io/RollingEqDeleteWriter.java | 33 +++++++ .../iceberg/io/UnpartitionedWriter.java | 4 +- .../org/apache/iceberg/io/WriterResult.java | 6 +- .../data/GenericDeltaWriterFactory.java | 10 ++- .../iceberg/data/TestGenericDeltaWriter.java | 14 ++- .../flink/sink/FlinkDeltaWriterFactory.java | 8 +- .../iceberg/flink/sink/RowDataTaskWriter.java | 12 +-- .../flink/sink/TestFlinkDeltaWriter.java | 10 +-- 16 files changed, 128 insertions(+), 108 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/io/RollingEqDeleteWriter.java diff --git a/core/src/main/java/org/apache/iceberg/ContentFileWriter.java b/core/src/main/java/org/apache/iceberg/ContentFileWriter.java index c7b6838d2220..9cc8a1cab37a 100644 --- a/core/src/main/java/org/apache/iceberg/ContentFileWriter.java +++ b/core/src/main/java/org/apache/iceberg/ContentFileWriter.java @@ -42,7 +42,7 @@ default void writeAll(Iterable values) { long length(); /** - * Return a {@link DeleteFile} or {@link DataFile} + * 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 index eed0f3ddae89..f7528a0ebcf6 100644 --- a/core/src/main/java/org/apache/iceberg/ContentFileWriterFactory.java +++ b/core/src/main/java/org/apache/iceberg/ContentFileWriterFactory.java @@ -21,9 +21,23 @@ 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 { - ContentFileWriter createWriter(PartitionKey partitionKey, - EncryptedOutputFile outputFile, + /** + * 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/io/BaseDeltaWriter.java b/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java index 2854acceb0a9..129c8dcf9371 100644 --- a/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BaseDeltaWriter.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.util.List; import org.apache.iceberg.DataFile; -import org.apache.iceberg.DeleteFile; import org.apache.iceberg.Schema; import org.apache.iceberg.StructLike; import org.apache.iceberg.deletes.PositionDelete; @@ -32,42 +31,38 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.StructLikeMap; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public abstract class BaseDeltaWriter implements DeltaWriter { - private static final Logger LOG = LoggerFactory.getLogger(BaseDeltaWriter.class); - private final RollingContentFileWriter dataWriter; - private final RollingContentFileWriter equalityDeleteWriter; + private final RollingEqDeleteWriter eqDeleteWriter; private final RollingPosDeleteWriter posDeleteWriter; private final PositionDelete positionDelete = new PositionDelete<>(); private final StructLikeMap insertedRowMap; - public BaseDeltaWriter(RollingContentFileWriter dataWriter, - RollingPosDeleteWriter posDeleteWriter, - RollingContentFileWriter equalityDeleteWriter, - Schema tableSchema, - List equalityFieldIds) { + 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(equalityDeleteWriter == null, + Preconditions.checkArgument(eqDeleteWriter == null, "Could not accept equality deletes when position delete writer is null."); } - if (posDeleteWriter != null && equalityDeleteWriter == null) { - // Only accept INSERT records and position deletion. + 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 (equalityDeleteWriter != null) { - // Accept insert records, position deletion, equality deletions. + 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"); @@ -80,7 +75,7 @@ public BaseDeltaWriter(RollingContentFileWriter dataWriter, } this.dataWriter = dataWriter; - this.equalityDeleteWriter = equalityDeleteWriter; + this.eqDeleteWriter = eqDeleteWriter; this.posDeleteWriter = posDeleteWriter; } @@ -90,10 +85,10 @@ public BaseDeltaWriter(RollingContentFileWriter dataWriter, @Override public void writeRow(T row) { - if (enableEqualityDelete()) { + if (allowEqDelete()) { RowOffset rowOffset = RowOffset.create(dataWriter.currentPath(), dataWriter.currentRows()); - // Copy the key to avoid messing up keys in the insertedRowMap. + // 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); @@ -104,16 +99,14 @@ public void writeRow(T row) { @Override public void writeEqualityDelete(T equalityDelete) { - if (!enableEqualityDelete()) { - throw new UnsupportedOperationException("Could not accept equality deletion."); - } + 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. - equalityDeleteWriter.write(equalityDelete); + 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. @@ -125,9 +118,7 @@ public void writeEqualityDelete(T equalityDelete) { @Override public void writePosDelete(CharSequence path, long offset, T row) { - if (!enablePosDelete()) { - throw new UnsupportedOperationException("Could not accept position deletion."); - } + Preconditions.checkState(allowPosDelete(), "Could not accept position deletion."); posDeleteWriter.write(positionDelete.set(path, offset, row)); } @@ -135,41 +126,29 @@ public void writePosDelete(CharSequence path, long offset, T row) { @Override public void abort() { if (dataWriter != null) { - try { - dataWriter.abort(); - } catch (IOException e) { - LOG.warn("Failed to abort the data writer {} because: ", dataWriter, e); - } - } - - if (equalityDeleteWriter != null) { - try { - equalityDeleteWriter.abort(); - } catch (IOException e) { - LOG.warn("Failed to abort the equality-delete writer {} because: ", equalityDeleteWriter, e); - } + dataWriter.abort(); + } + + if (eqDeleteWriter != null) { + eqDeleteWriter.abort(); insertedRowMap.clear(); } if (posDeleteWriter != null) { - try { - posDeleteWriter.abort(); - } catch (IOException e) { - LOG.warn("Failed to abort the pos-delete writer {} because: ", posDeleteWriter, e); - } + posDeleteWriter.abort(); } } @Override - public WriterResult complete() throws IOException { + public WriterResult complete() { WriterResult.Builder builder = WriterResult.builder(); if (dataWriter != null) { builder.add(dataWriter.complete()); } - if (equalityDeleteWriter != null) { - builder.add(equalityDeleteWriter.complete()); + if (eqDeleteWriter != null) { + builder.add(eqDeleteWriter.complete()); insertedRowMap.clear(); } @@ -186,8 +165,8 @@ public void close() throws IOException { dataWriter.close(); } - if (equalityDeleteWriter != null) { - equalityDeleteWriter.close(); + if (eqDeleteWriter != null) { + eqDeleteWriter.close(); insertedRowMap.clear(); } @@ -196,11 +175,11 @@ public void close() throws IOException { } } - private boolean enableEqualityDelete() { - return equalityDeleteWriter != null && posDeleteWriter != null; + private boolean allowEqDelete() { + return eqDeleteWriter != null && posDeleteWriter != null; } - private boolean enablePosDelete() { + private boolean allowPosDelete() { return posDeleteWriter != null; } @@ -221,7 +200,7 @@ private static RowOffset create(CharSequence path, long pos) { public String toString() { return MoreObjects.toStringHelper(this) .add("path", path) - .add("pos", rowId) + .add("row_id", rowId) .toString(); } } diff --git a/core/src/main/java/org/apache/iceberg/io/DataFileWriterFactory.java b/core/src/main/java/org/apache/iceberg/io/DataFileWriterFactory.java index a57bf477e15d..8839eee52d10 100644 --- a/core/src/main/java/org/apache/iceberg/io/DataFileWriterFactory.java +++ b/core/src/main/java/org/apache/iceberg/io/DataFileWriterFactory.java @@ -29,7 +29,7 @@ import org.apache.iceberg.encryption.EncryptedOutputFile; /** - * TODO: This factory would be removed once we've replaced by using {@link DeltaWriterFactory} in the upper layer. + * 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; diff --git a/core/src/main/java/org/apache/iceberg/io/DeltaWriter.java b/core/src/main/java/org/apache/iceberg/io/DeltaWriter.java index e90358166bc8..57df75c53c47 100644 --- a/core/src/main/java/org/apache/iceberg/io/DeltaWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/DeltaWriter.java @@ -20,7 +20,6 @@ package org.apache.iceberg.io; import java.io.Closeable; -import java.io.IOException; /** * The write interface could accept INSERT, POS-DELETION, EQUALITY-DELETION. It usually write those operations @@ -53,10 +52,10 @@ default void writePosDelete(CharSequence path, long offset) { /** * Abort the writer to clean all generated files. */ - void abort() throws IOException; + void abort(); /** * Close the writer and get all the completed data files and delete files. */ - WriterResult complete() throws IOException; + 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 index ebf8450d1829..422ed368982f 100644 --- a/core/src/main/java/org/apache/iceberg/io/DeltaWriterFactory.java +++ b/core/src/main/java/org/apache/iceberg/io/DeltaWriterFactory.java @@ -140,7 +140,7 @@ public Context build() { for (Integer fieldId : equalityFieldIds) { Preconditions.checkNotNull(eqDeleteRowSchema.findField(fieldId), - "Missing field id %s in provided row schema: %s", fieldId, eqDeleteRowSchema); + "Unknown field with id %s in provided equality row schema: %s", fieldId, eqDeleteRowSchema); } } 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 feda2358c237..317de6559018 100644 --- a/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java @@ -102,12 +102,7 @@ public void abort() throws IOException { close(); if (currentWriter != null) { - try { - currentWriter.abort(); - } catch (IOException e) { - LOG.warn("Failed to abort the current writer: {} because: ", currentWriter, e); - } - + currentWriter.abort(); currentWriter = null; } diff --git a/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java b/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java index a1d9bc533fb7..09ef4ac3ab57 100644 --- a/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/RollingContentFileWriter.java @@ -48,8 +48,8 @@ public class RollingContentFileWriter implements Closeable { private long currentRows = 0; public RollingContentFileWriter(PartitionKey partitionKey, FileFormat format, - OutputFileFactory fileFactory, FileIO io, - long targetFileSize, ContentFileWriterFactory writerFactory) { + OutputFileFactory fileFactory, FileIO io, long targetFileSize, + ContentFileWriterFactory writerFactory) { this.partitionKey = partitionKey; this.format = format; this.fileFactory = fileFactory; @@ -62,11 +62,11 @@ public RollingContentFileWriter(PartitionKey partitionKey, FileFormat format, openCurrent(); } - public CharSequence currentPath() { + CharSequence currentPath() { return currentFile.encryptingOutputFile().location(); } - public long currentRows() { + long currentRows() { return currentRows; } @@ -80,8 +80,8 @@ public void write(T record) { } } - public void abort() throws IOException { - close(); + public void abort() { + closeCurrent(); WriterResult result = resultBuilder.build(); @@ -91,8 +91,8 @@ public void abort() throws IOException { .run(file -> io.deleteFile(file.path().toString())); } - public WriterResult complete() throws IOException { - close(); + public WriterResult complete() { + closeCurrent(); return resultBuilder.build(); } @@ -135,7 +135,7 @@ private void closeCurrent() { try { currentFileWriter.close(); } catch (IOException e) { - throw new UncheckedIOException("Failed to close the current file writer", e); + throw new UncheckedIOException("Failed to close the current file writer: ", e); } ContentFileT contentFile = currentFileWriter.toContentFile(); 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/UnpartitionedWriter.java b/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java index 38651984d4c4..3f81678e48bd 100644 --- a/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java @@ -39,12 +39,12 @@ public void write(T record) throws IOException { } @Override - public void abort() throws IOException { + public void abort() { currentWriter.abort(); } @Override - public WriterResult complete() throws IOException { + public WriterResult complete() { return currentWriter.complete(); } diff --git a/core/src/main/java/org/apache/iceberg/io/WriterResult.java b/core/src/main/java/org/apache/iceberg/io/WriterResult.java index 4c08b168aaa5..9beb2682c568 100644 --- a/core/src/main/java/org/apache/iceberg/io/WriterResult.java +++ b/core/src/main/java/org/apache/iceberg/io/WriterResult.java @@ -36,12 +36,12 @@ public class WriterResult implements Serializable { private DataFile[] dataFiles; private DeleteFile[] deleteFiles; - WriterResult(DataFile[] dataFiles, DeleteFile[] deleteFiles) { + private WriterResult(DataFile[] dataFiles, DeleteFile[] deleteFiles) { this.dataFiles = dataFiles; this.deleteFiles = deleteFiles; } - WriterResult(List dataFiles, List deleteFiles) { + private WriterResult(List dataFiles, List deleteFiles) { this.dataFiles = dataFiles.toArray(new DataFile[0]); this.deleteFiles = deleteFiles.toArray(new DeleteFile[0]); } @@ -123,7 +123,7 @@ public void add(ContentFile contentFile) { break; default: - throw new UnsupportedOperationException("Unknown file: " + contentFile.content()); + throw new UnsupportedOperationException("Unknown content file: " + contentFile.content()); } } diff --git a/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java b/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java index bbd2d312e5af..323920d77ad2 100644 --- a/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java +++ b/data/src/main/java/org/apache/iceberg/data/GenericDeltaWriterFactory.java @@ -45,6 +45,7 @@ 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; @@ -83,6 +84,7 @@ public DeltaWriter createDeltaWriter(PartitionKey partitionKey, Context format, fileFactory, io, targetFileSize, createDataFileWriterFactory()); if (!ctxt.allowPosDelete() && !ctxt.allowEqualityDelete()) { + // Only accept INSERT records. return new GenericDeltaWriter(dataWriter); } @@ -90,14 +92,16 @@ public DeltaWriter createDeltaWriter(PartitionKey partitionKey, Context 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."); - RollingContentFileWriter eqDeleteWriter = new RollingContentFileWriter<>(partitionKey, + RollingEqDeleteWriter eqDeleteWriter = new RollingEqDeleteWriter<>(partitionKey, format, fileFactory, io, targetFileSize, createEqualityDeleteWriterFactory(ctxt.equalityFieldIds(), ctxt.eqDeleteRowSchema())); @@ -221,9 +225,9 @@ private static class GenericDeltaWriter extends BaseDeltaWriter { GenericDeltaWriter(RollingContentFileWriter dataWriter, RollingPosDeleteWriter posDeleteWriter, - RollingContentFileWriter equalityDeleteWriter, Schema tableSchema, + RollingEqDeleteWriter eqDeleteWriter, Schema tableSchema, List equalityFieldIds) { - super(dataWriter, posDeleteWriter, equalityDeleteWriter, tableSchema, equalityFieldIds); + super(dataWriter, posDeleteWriter, eqDeleteWriter, tableSchema, equalityFieldIds); } @Override diff --git a/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java b/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java index 54e567880877..00553be09d0b 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java +++ b/data/src/test/java/org/apache/iceberg/data/TestGenericDeltaWriter.java @@ -91,7 +91,7 @@ public void before() throws IOException { } @Test - public void testWritePureInsert() throws IOException { + public void testWritePureInsert() { DeltaWriterFactory writerFactory = createDeltaWriterFactory(); DeltaWriterFactory.Context ctxt = DeltaWriterFactory.Context.builder() @@ -102,9 +102,7 @@ public void testWritePureInsert() throws IOException { DeltaWriter deltaWriter = writerFactory.createDeltaWriter(null, ctxt); List expected = RandomGenericData.generate(SCHEMA, 100, 22112234L); - for (Record record : expected) { - deltaWriter.writeRow(record); - } + expected.forEach(deltaWriter::writeRow); WriterResult result = deltaWriter.complete(); @@ -117,7 +115,7 @@ public void testWritePureInsert() throws IOException { } @Test - public void testWriteEqualityDelete() throws IOException { + public void testWriteEqualityDelete() { DeltaWriterFactory writerFactory = createDeltaWriterFactory(); List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); @@ -162,7 +160,7 @@ public void testWriteEqualityDelete() throws IOException { } @Test - public void testEqualityDeleteSameRow() throws IOException { + public void testEqualityDeleteSameRow() { DeltaWriterFactory writerFactory = createDeltaWriterFactory(); List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); @@ -206,7 +204,7 @@ public void testEqualityDeleteSameRow() throws IOException { } @Test - public void testPositionDelete() throws IOException { + public void testPositionDelete() { DeltaWriterFactory writerFactory = createDeltaWriterFactory(); DeltaWriterFactory.Context ctxt = DeltaWriterFactory.Context.builder() .allowPosDelete(true) @@ -255,7 +253,7 @@ public void testPositionDelete() throws IOException { } @Test - public void testUpsertSameRow() throws IOException { + public void testUpsertSameRow() { DeltaWriterFactory writerFactory = createDeltaWriterFactory(); List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); DeltaWriterFactory.Context ctxt = DeltaWriterFactory.Context.builder() 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 index e5daae313915..5f7b382102af 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkDeltaWriterFactory.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkDeltaWriterFactory.java @@ -53,6 +53,7 @@ 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; @@ -94,6 +95,7 @@ public DeltaWriter createDeltaWriter(PartitionKey partitionKey, Context format, fileFactory, io, targetFileSize, createDataFileWriterFactory()); if (!ctxt.allowPosDelete() && !ctxt.allowEqualityDelete()) { + // Only accept INSERT records. return new RowDataDeltaWriter(dataWriter); } @@ -101,14 +103,16 @@ public DeltaWriter createDeltaWriter(PartitionKey partitionKey, Context 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."); - RollingContentFileWriter eqDeleteWriter = new RollingContentFileWriter<>(partitionKey, + RollingEqDeleteWriter eqDeleteWriter = new RollingEqDeleteWriter<>(partitionKey, format, fileFactory, io, targetFileSize, createEqualityDeleteWriterFactory(ctxt.equalityFieldIds(), ctxt.eqDeleteRowSchema())); @@ -242,7 +246,7 @@ private class RowDataDeltaWriter extends BaseDeltaWriter { RowDataDeltaWriter(RollingContentFileWriter dataWriter, RollingPosDeleteWriter posDeleteWriter, - RollingContentFileWriter equalityDeleteWriter, Schema tableSchema, + RollingEqDeleteWriter equalityDeleteWriter, Schema tableSchema, List equalityFieldIds) { super(dataWriter, posDeleteWriter, equalityDeleteWriter, tableSchema, equalityFieldIds); 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 index 94dc8b68287d..a2bc91b62819 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriter.java +++ b/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriter.java @@ -81,7 +81,7 @@ class RowDataTaskWriter implements TaskWriter { } @Override - public void write(RowData row) throws IOException { + public void write(RowData row) { DeltaWriter deltaWriter; if (spec.fields().size() <= 0) { @@ -128,17 +128,11 @@ public void abort() { Tasks.foreach(deltaWriterMap.values()) .throwFailureWhenFinished() .noRetry() - .run(deltaWriter -> { - try { - deltaWriter.abort(); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - }); + .run(DeltaWriter::abort); } @Override - public WriterResult complete() throws IOException { + public WriterResult complete() { close(); WriterResult.Builder builder = WriterResult.builder(); 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 index d65a2d02c58c..83179e96c7a0 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkDeltaWriter.java +++ b/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkDeltaWriter.java @@ -101,7 +101,7 @@ public void before() throws IOException { } @Test - public void testWritePureInsert() throws IOException { + public void testWritePureInsert() { DeltaWriterFactory writerFactory = createDeltaWriterFactory(); DeltaWriterFactory.Context ctxt = DeltaWriterFactory.Context.builder() @@ -127,7 +127,7 @@ public void testWritePureInsert() throws IOException { } @Test - public void testWriteEqualityDelete() throws IOException { + public void testWriteEqualityDelete() { DeltaWriterFactory writerFactory = createDeltaWriterFactory(); List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); @@ -171,7 +171,7 @@ public void testWriteEqualityDelete() throws IOException { } @Test - public void testEqualityDeleteSameRow() throws IOException { + public void testEqualityDeleteSameRow() { DeltaWriterFactory writerFactory = createDeltaWriterFactory(); List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); @@ -214,7 +214,7 @@ public void testEqualityDeleteSameRow() throws IOException { } @Test - public void testPositionDelete() throws IOException { + public void testPositionDelete() { DeltaWriterFactory writerFactory = createDeltaWriterFactory(); DeltaWriterFactory.Context ctxt = DeltaWriterFactory.Context.builder() .allowPosDelete(true) @@ -262,7 +262,7 @@ public void testPositionDelete() throws IOException { } @Test - public void testUpsertSameRow() throws IOException { + public void testUpsertSameRow() { DeltaWriterFactory writerFactory = createDeltaWriterFactory(); List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); DeltaWriterFactory.Context ctxt = DeltaWriterFactory.Context.builder()