From e7bc4e6dd913cdcfdd43bcb445b0875509bc23a2 Mon Sep 17 00:00:00 2001 From: openinx Date: Thu, 16 Jul 2020 16:36:58 +0800 Subject: [PATCH 01/16] Abstract the generic task writers for sharing the common codes between spark and flink. --- .../apache/iceberg/taskio/BaseTaskWriter.java | 138 +++++++++++++++++ .../iceberg/taskio/FileAppenderFactory.java | 29 ++++ .../iceberg/taskio}/OutputFileFactory.java | 8 +- .../iceberg/taskio}/PartitionedWriter.java | 64 +++++--- .../org/apache/iceberg/taskio/TaskWriter.java | 34 +++++ .../iceberg/taskio/UnpartitionedWriter.java | 74 +++++++++ .../iceberg/spark/source/BaseWriter.java | 144 ------------------ .../iceberg/spark/source/RowDataRewriter.java | 13 +- .../spark/source/SparkAppenderFactory.java | 3 +- ...partitionedWriter.java => WriterUtil.java} | 30 ++-- .../apache/iceberg/spark/source/Writer.java | 32 +++- .../iceberg/spark/source/SparkBatchWrite.java | 32 +++- 12 files changed, 404 insertions(+), 197 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/taskio/BaseTaskWriter.java create mode 100644 core/src/main/java/org/apache/iceberg/taskio/FileAppenderFactory.java rename {spark/src/main/java/org/apache/iceberg/spark/source => core/src/main/java/org/apache/iceberg/taskio}/OutputFileFactory.java (91%) rename {spark/src/main/java/org/apache/iceberg/spark/source => core/src/main/java/org/apache/iceberg/taskio}/PartitionedWriter.java (54%) create mode 100644 core/src/main/java/org/apache/iceberg/taskio/TaskWriter.java create mode 100644 core/src/main/java/org/apache/iceberg/taskio/UnpartitionedWriter.java delete mode 100644 spark/src/main/java/org/apache/iceberg/spark/source/BaseWriter.java rename spark/src/main/java/org/apache/iceberg/spark/source/{UnpartitionedWriter.java => WriterUtil.java} (57%) diff --git a/core/src/main/java/org/apache/iceberg/taskio/BaseTaskWriter.java b/core/src/main/java/org/apache/iceberg/taskio/BaseTaskWriter.java new file mode 100644 index 000000000000..15ea4dd493e7 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/taskio/BaseTaskWriter.java @@ -0,0 +1,138 @@ +/* + * 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.taskio; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.function.Supplier; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +public abstract class BaseTaskWriter implements TaskWriter { + protected static final int ROWS_DIVISOR = 1000; + + private final List completedFiles = Lists.newArrayList(); + private final PartitionSpec spec; + private final FileFormat format; + private final FileAppenderFactory appenderFactory; + private final OutputFileFactory fileFactory; + private final FileIO io; + private final long targetFileSize; + + BaseTaskWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, FileIO io, long targetFileSize) { + this.spec = spec; + this.format = format; + this.appenderFactory = appenderFactory; + this.fileFactory = fileFactory; + this.io = io; + this.targetFileSize = targetFileSize; + } + + public List pollCompleteFiles() { + if (completedFiles.size() > 0) { + List dataFiles = ImmutableList.copyOf(completedFiles); + completedFiles.clear(); + return dataFiles; + } else { + return Collections.emptyList(); + } + } + + protected FileIO io() { + return this.io; + } + + protected OutputFileFactory outputFileFactory() { + return this.fileFactory; + } + + WrappedFileAppender createWrappedFileAppender(PartitionKey partitionKey, + Supplier outputFileSupplier) { + EncryptedOutputFile outputFile = outputFileSupplier.get(); + FileAppender appender = appenderFactory.newAppender(outputFile.encryptingOutputFile(), format); + return new WrappedFileAppender(partitionKey, outputFile, appender); + } + + void closeWrappedFileAppender(WrappedFileAppender appender) throws IOException { + DataFile dataFile = appender.closeAndBuildDataFile(); + completedFiles.add(dataFile); + } + + class WrappedFileAppender { + private final PartitionKey partitionKey; + private final EncryptedOutputFile encryptedOutputFile; + private final FileAppender appender; + private boolean closed = false; + private long currentRows = 0; + + WrappedFileAppender(PartitionKey partitionKey, EncryptedOutputFile encryptedOutputFile, FileAppender appender) { + this.partitionKey = partitionKey; + this.encryptedOutputFile = encryptedOutputFile; + this.appender = appender; + } + + void add(T record) { + this.appender.add(record); + this.currentRows++; + } + + boolean shouldRollToNewFile() { + //TODO: ORC file now not support target file size before closed + return !format.equals(FileFormat.ORC) && + currentRows % ROWS_DIVISOR == 0 && appender.length() >= targetFileSize; + } + + private void close() throws IOException { + if (!closed) { + appender.close(); + closed = true; + } + } + + private DataFile closeAndBuildDataFile() throws IOException { + // Close the file appender firstly. + this.close(); + + // metrics are only valid after the appender is closed. + Metrics metrics = appender.metrics(); + long fileSizeInBytes = appender.length(); + List splitOffsets = appender.splitOffsets(); + + return DataFiles.builder(spec) + .withEncryptedOutputFile(encryptedOutputFile) + .withFileSizeInBytes(fileSizeInBytes) + .withPartition(partitionKey) + .withMetrics(metrics) + .withSplitOffsets(splitOffsets) + .build(); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/taskio/FileAppenderFactory.java b/core/src/main/java/org/apache/iceberg/taskio/FileAppenderFactory.java new file mode 100644 index 000000000000..f72d757f3459 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/taskio/FileAppenderFactory.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.taskio; + +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.OutputFile; + +public interface FileAppenderFactory { + + FileAppender newAppender(OutputFile outputFile, FileFormat fileFormat); +} diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/OutputFileFactory.java b/core/src/main/java/org/apache/iceberg/taskio/OutputFileFactory.java similarity index 91% rename from spark/src/main/java/org/apache/iceberg/spark/source/OutputFileFactory.java rename to core/src/main/java/org/apache/iceberg/taskio/OutputFileFactory.java index 08e66df79362..8b09bbefa4c0 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/OutputFileFactory.java +++ b/core/src/main/java/org/apache/iceberg/taskio/OutputFileFactory.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iceberg.spark.source; +package org.apache.iceberg.taskio; import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; @@ -30,7 +30,7 @@ import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.io.OutputFile; -class OutputFileFactory { +public class OutputFileFactory { private final PartitionSpec spec; private final FileFormat format; private final LocationProvider locations; @@ -44,8 +44,8 @@ class OutputFileFactory { private final String uuid = UUID.randomUUID().toString(); private final AtomicInteger fileCount = new AtomicInteger(0); - OutputFileFactory(PartitionSpec spec, FileFormat format, LocationProvider locations, FileIO io, - EncryptionManager encryptionManager, int partitionId, long taskId) { + public OutputFileFactory(PartitionSpec spec, FileFormat format, LocationProvider locations, FileIO io, + EncryptionManager encryptionManager, int partitionId, long taskId) { this.spec = spec; this.format = format; this.locations = locations; diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/PartitionedWriter.java b/core/src/main/java/org/apache/iceberg/taskio/PartitionedWriter.java similarity index 54% rename from spark/src/main/java/org/apache/iceberg/spark/source/PartitionedWriter.java rename to core/src/main/java/org/apache/iceberg/taskio/PartitionedWriter.java index 0ead766f38c7..4a08034c8f23 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/PartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/taskio/PartitionedWriter.java @@ -17,43 +17,43 @@ * under the License. */ -package org.apache.iceberg.spark.source; +package org.apache.iceberg.taskio; import java.io.IOException; import java.util.Set; +import java.util.function.Function; 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.FileIO; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.iceberg.util.Tasks; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -class PartitionedWriter extends BaseWriter { +public class PartitionedWriter extends BaseTaskWriter { private static final Logger LOG = LoggerFactory.getLogger(PartitionedWriter.class); - private final PartitionKey key; - private final InternalRowWrapper wrapper; + private PartitionKey currentKey = null; + private WrappedFileAppender currentAppender = null; + private final Function partitionKeyGetter; private final Set completedPartitions = Sets.newHashSet(); - PartitionedWriter(PartitionSpec spec, FileFormat format, SparkAppenderFactory appenderFactory, - OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema writeSchema) { + + public PartitionedWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, FileIO io, long targetFileSize, + Function partitionKeyGetter) { super(spec, format, appenderFactory, fileFactory, io, targetFileSize); - this.key = new PartitionKey(spec, writeSchema); - this.wrapper = new InternalRowWrapper(SparkSchemaUtil.convert(writeSchema)); + this.partitionKeyGetter = partitionKeyGetter; } @Override - public void write(InternalRow row) throws IOException { - key.partition(wrapper.wrap(row)); + public void write(T row) throws IOException { + PartitionKey key = partitionKeyGetter.apply(row); - PartitionKey currentKey = getCurrentKey(); if (!key.equals(currentKey)) { - closeCurrent(); + closeCurrentWriter(); completedPartitions.add(currentKey); if (completedPartitions.contains(key)) { @@ -63,10 +63,38 @@ public void write(InternalRow row) throws IOException { throw new IllegalStateException("Already closed files for partition: " + key.toPath()); } - setCurrentKey(key.copy()); - openCurrent(); + currentKey = key.copy(); + + createWrappedFileAppender(currentKey, () -> outputFileFactory().newOutputFile(currentKey)); } - writeInternal(row); + currentAppender.add(row); + } + + @Override + public void abort() throws IOException { + closeCurrentWriter(); + + // clean up files created by this writer + Tasks.foreach(pollCompleteFiles()) + .throwFailureWhenFinished() + .noRetry() + .run(file -> io().deleteFile(file.path().toString())); + } + + @Override + public void close() throws IOException { + closeCurrentWriter(); + } + + private void closeCurrentWriter() throws IOException { + if (currentAppender != null) { + + // Close the current file appender and put the generated DataFile to completeDataFiles. + closeWrappedFileAppender(currentAppender); + + // Reset the current appender to be null. + currentAppender = null; + } } } diff --git a/core/src/main/java/org/apache/iceberg/taskio/TaskWriter.java b/core/src/main/java/org/apache/iceberg/taskio/TaskWriter.java new file mode 100644 index 000000000000..7b848253c442 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/taskio/TaskWriter.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.taskio; + +import java.io.Closeable; +import java.io.IOException; +import java.util.List; +import org.apache.iceberg.DataFile; + +public interface TaskWriter extends Closeable { + + void write(T row) throws IOException; + + void abort() throws IOException; + + List pollCompleteFiles(); +} diff --git a/core/src/main/java/org/apache/iceberg/taskio/UnpartitionedWriter.java b/core/src/main/java/org/apache/iceberg/taskio/UnpartitionedWriter.java new file mode 100644 index 000000000000..57549bdac9ad --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/taskio/UnpartitionedWriter.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.taskio; + +import java.io.IOException; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.util.Tasks; + +public class UnpartitionedWriter extends BaseTaskWriter { + + private WrappedFileAppender currentAppender = null; + + public UnpartitionedWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, FileIO io, long targetFileSize) { + super(spec, format, appenderFactory, fileFactory, io, targetFileSize); + } + + public void write(T record) throws IOException { + if (currentAppender == null) { + currentAppender = createWrappedFileAppender(null, outputFileFactory()::newOutputFile); + } + currentAppender.add(record); + + // Roll the writer if reach the target file size. + if (currentAppender.shouldRollToNewFile()) { + closeCurrentWriter(); + } + } + + @Override + public void abort() throws IOException { + closeCurrentWriter(); + + // clean up files created by this writer + Tasks.foreach(pollCompleteFiles()) + .throwFailureWhenFinished() + .noRetry() + .run(file -> io().deleteFile(file.path().toString())); + } + + public void close() throws IOException { + closeCurrentWriter(); + } + + private void closeCurrentWriter() throws IOException { + if (currentAppender != null) { + + // Close the current file appender and put the generated DataFile to completeDataFiles. + closeWrappedFileAppender(currentAppender); + + // Reset the current appender to be null. + currentAppender = null; + } + } +} diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/BaseWriter.java b/spark/src/main/java/org/apache/iceberg/spark/source/BaseWriter.java deleted file mode 100644 index 8c41e77d0f10..000000000000 --- a/spark/src/main/java/org/apache/iceberg/spark/source/BaseWriter.java +++ /dev/null @@ -1,144 +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.spark.source; - -import java.io.Closeable; -import java.io.IOException; -import java.util.List; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Metrics; -import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.encryption.EncryptedOutputFile; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.util.Tasks; -import org.apache.spark.sql.catalyst.InternalRow; - -abstract class BaseWriter implements Closeable { - protected static final int ROWS_DIVISOR = 1000; - - private final List completedFiles = Lists.newArrayList(); - private final PartitionSpec spec; - private final FileFormat format; - private final SparkAppenderFactory appenderFactory; - private final OutputFileFactory fileFactory; - private final FileIO io; - private final long targetFileSize; - private PartitionKey currentKey = null; - private FileAppender currentAppender = null; - private EncryptedOutputFile currentFile = null; - private long currentRows = 0; - - BaseWriter(PartitionSpec spec, FileFormat format, SparkAppenderFactory appenderFactory, - OutputFileFactory fileFactory, FileIO io, long targetFileSize) { - this.spec = spec; - this.format = format; - this.appenderFactory = appenderFactory; - this.fileFactory = fileFactory; - this.io = io; - this.targetFileSize = targetFileSize; - } - - public abstract void write(InternalRow row) throws IOException; - - public void writeInternal(InternalRow row) throws IOException { - //TODO: ORC file now not support target file size before closed - if (!format.equals(FileFormat.ORC) && - currentRows % ROWS_DIVISOR == 0 && currentAppender.length() >= targetFileSize) { - closeCurrent(); - openCurrent(); - } - - currentAppender.add(row); - currentRows++; - } - - public TaskResult complete() throws IOException { - closeCurrent(); - - return new TaskResult(completedFiles); - } - - public void abort() throws IOException { - closeCurrent(); - - // clean up files created by this writer - Tasks.foreach(completedFiles) - .throwFailureWhenFinished() - .noRetry() - .run(file -> io.deleteFile(file.path().toString())); - } - - @Override - public void close() throws IOException { - closeCurrent(); - } - - protected void openCurrent() { - if (spec.fields().size() == 0) { - // unpartitioned - currentFile = fileFactory.newOutputFile(); - } else { - // partitioned - currentFile = fileFactory.newOutputFile(currentKey); - } - currentAppender = appenderFactory.newAppender(currentFile.encryptingOutputFile(), format); - currentRows = 0; - } - - protected void closeCurrent() throws IOException { - if (currentAppender != null) { - currentAppender.close(); - // metrics are only valid after the appender is closed - Metrics metrics = currentAppender.metrics(); - long fileSizeInBytes = currentAppender.length(); - List splitOffsets = currentAppender.splitOffsets(); - this.currentAppender = null; - - if (metrics.recordCount() == 0L) { - io.deleteFile(currentFile.encryptingOutputFile()); - } else { - DataFile dataFile = DataFiles.builder(spec) - .withEncryptionKeyMetadata(currentFile.keyMetadata()) - .withPath(currentFile.encryptingOutputFile().location()) - .withFileSizeInBytes(fileSizeInBytes) - .withPartition(spec.fields().size() == 0 ? null : currentKey) // set null if unpartitioned - .withMetrics(metrics) - .withSplitOffsets(splitOffsets) - .build(); - completedFiles.add(dataFile); - } - - this.currentFile = null; - } - } - - protected PartitionKey getCurrentKey() { - return currentKey; - } - - protected void setCurrentKey(PartitionKey currentKey) { - this.currentKey = currentKey; - } -} 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 2837dbc2f59c..0591696fb0d2 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 @@ -36,6 +36,10 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.taskio.OutputFileFactory; +import org.apache.iceberg.taskio.PartitionedWriter; +import org.apache.iceberg.taskio.TaskWriter; +import org.apache.iceberg.taskio.UnpartitionedWriter; import org.apache.spark.TaskContext; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.broadcast.Broadcast; @@ -97,11 +101,12 @@ private TaskResult rewriteDataForTask(CombinedScanTask task) throws Exception { OutputFileFactory fileFactory = new OutputFileFactory( spec, format, locations, io.value(), encryptionManager.value(), partitionId, taskId); - BaseWriter writer; + TaskWriter writer; if (spec.fields().isEmpty()) { - writer = new UnpartitionedWriter(spec, format, appenderFactory, fileFactory, io.value(), Long.MAX_VALUE); + writer = new UnpartitionedWriter<>(spec, format, appenderFactory, fileFactory, io.value(), Long.MAX_VALUE); } else { - writer = new PartitionedWriter(spec, format, appenderFactory, fileFactory, io.value(), Long.MAX_VALUE, schema); + writer = new PartitionedWriter<>(spec, format, appenderFactory, fileFactory, io.value(), Long.MAX_VALUE, + WriterUtil.buildKeyGetter(spec, schema)); } try { @@ -112,7 +117,7 @@ private TaskResult rewriteDataForTask(CombinedScanTask task) throws Exception { dataReader.close(); dataReader = null; - return writer.complete(); + return WriterUtil.createTaskResult(writer.pollCompleteFiles()); } catch (Throwable originalThrowable) { try { diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java b/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java index d7b271e82396..9b7392bfb371 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java @@ -33,10 +33,11 @@ import org.apache.iceberg.spark.data.SparkAvroWriter; import org.apache.iceberg.spark.data.SparkOrcWriter; import org.apache.iceberg.spark.data.SparkParquetWriters; +import org.apache.iceberg.taskio.FileAppenderFactory; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.types.StructType; -class SparkAppenderFactory { +class SparkAppenderFactory implements FileAppenderFactory { private final Map properties; private final Schema writeSchema; private final StructType dsSchema; diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/UnpartitionedWriter.java b/spark/src/main/java/org/apache/iceberg/spark/source/WriterUtil.java similarity index 57% rename from spark/src/main/java/org/apache/iceberg/spark/source/UnpartitionedWriter.java rename to spark/src/main/java/org/apache/iceberg/spark/source/WriterUtil.java index 5692b6afe24d..e6cc53f40575 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/UnpartitionedWriter.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/WriterUtil.java @@ -19,22 +19,30 @@ package org.apache.iceberg.spark.source; -import java.io.IOException; -import org.apache.iceberg.FileFormat; +import java.util.List; +import java.util.function.Function; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.Schema; +import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.spark.sql.catalyst.InternalRow; -class UnpartitionedWriter extends BaseWriter { - UnpartitionedWriter(PartitionSpec spec, FileFormat format, SparkAppenderFactory appenderFactory, - OutputFileFactory fileFactory, FileIO io, long targetFileSize) { - super(spec, format, appenderFactory, fileFactory, io, targetFileSize); +class WriterUtil { + private WriterUtil() { + } + + static Function buildKeyGetter(PartitionSpec spec, Schema schema) { + PartitionKey key = new PartitionKey(spec, schema); + InternalRowWrapper wrapper = new InternalRowWrapper(SparkSchemaUtil.convert(schema)); - openCurrent(); + return row -> { + key.partition(wrapper.wrap(row)); + return key; + }; } - @Override - public void write(InternalRow row) throws IOException { - writeInternal(row); + static TaskResult createTaskResult(List dataFiles) { + return new TaskResult(dataFiles); } } 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 9ce590c4720b..256f64055c98 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 @@ -21,12 +21,15 @@ import java.io.IOException; import java.util.Arrays; +import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Optional; +import java.util.function.Function; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.ReplacePartitions; import org.apache.iceberg.Schema; @@ -40,6 +43,9 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.taskio.OutputFileFactory; +import org.apache.iceberg.taskio.PartitionedWriter; +import org.apache.iceberg.taskio.UnpartitionedWriter; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.Tasks; import org.apache.spark.broadcast.Broadcast; @@ -263,13 +269,14 @@ public DataWriter createDataWriter(int partitionId, long taskId, lo if (spec.fields().isEmpty()) { return new Unpartitioned24Writer(spec, format, appenderFactory, fileFactory, io.value(), targetFileSize); } else { - return new Partitioned24Writer( - spec, format, appenderFactory, fileFactory, io.value(), targetFileSize, writeSchema); + return new Partitioned24Writer(spec, format, appenderFactory, fileFactory, io.value(), + targetFileSize, WriterUtil.buildKeyGetter(spec, writeSchema)); } } } - private static class Unpartitioned24Writer extends UnpartitionedWriter implements DataWriter { + 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); @@ -277,19 +284,28 @@ private static class Unpartitioned24Writer extends UnpartitionedWriter implement @Override public WriterCommitMessage commit() throws IOException { - return new TaskCommit(complete()); + this.close(); + + List dataFiles = pollCompleteFiles(); + TaskResult taskResult = new TaskResult(dataFiles); + return new TaskCommit(taskResult); } } - private static class Partitioned24Writer extends PartitionedWriter implements DataWriter { + private static class Partitioned24Writer extends PartitionedWriter implements DataWriter { Partitioned24Writer(PartitionSpec spec, FileFormat format, SparkAppenderFactory appenderFactory, - OutputFileFactory fileFactory, FileIO fileIo, long targetFileSize, Schema writeSchema) { - super(spec, format, appenderFactory, fileFactory, fileIo, targetFileSize, writeSchema); + OutputFileFactory fileFactory, FileIO fileIo, long targetFileSize, + Function keyGetter) { + super(spec, format, appenderFactory, fileFactory, fileIo, targetFileSize, keyGetter); } @Override public WriterCommitMessage commit() throws IOException { - return new TaskCommit(complete()); + this.close(); + + List dataFiles = pollCompleteFiles(); + TaskResult taskResult = new TaskResult(dataFiles); + return new TaskCommit(taskResult); } } } diff --git a/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchWrite.java b/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchWrite.java index e6b36f429eb8..c965ab1dc36f 100644 --- a/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchWrite.java +++ b/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchWrite.java @@ -21,13 +21,16 @@ import java.io.IOException; import java.util.Arrays; +import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Optional; +import java.util.function.Function; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.OverwriteFiles; +import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.ReplacePartitions; import org.apache.iceberg.Schema; @@ -42,6 +45,9 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.taskio.OutputFileFactory; +import org.apache.iceberg.taskio.PartitionedWriter; +import org.apache.iceberg.taskio.UnpartitionedWriter; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.Tasks; import org.apache.spark.broadcast.Broadcast; @@ -285,12 +291,14 @@ public DataWriter createWriter(int partitionId, long taskId, long e return new Unpartitioned3Writer(spec, format, appenderFactory, fileFactory, io.value(), targetFileSize); } else { return new Partitioned3Writer( - spec, format, appenderFactory, fileFactory, io.value(), targetFileSize, writeSchema); + spec, format, appenderFactory, fileFactory, io.value(), targetFileSize, + WriterUtil.buildKeyGetter(spec, writeSchema)); } } } - private static class Unpartitioned3Writer extends UnpartitionedWriter implements DataWriter { + 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); @@ -298,19 +306,29 @@ private static class Unpartitioned3Writer extends UnpartitionedWriter implements @Override public WriterCommitMessage commit() throws IOException { - return new TaskCommit(complete()); + this.close(); + + List dataFiles = pollCompleteFiles(); + TaskResult taskResult = new TaskResult(dataFiles); + return new TaskCommit(taskResult); } } - private static class Partitioned3Writer extends PartitionedWriter implements DataWriter { + private static class Partitioned3Writer extends PartitionedWriter + implements DataWriter { Partitioned3Writer(PartitionSpec spec, FileFormat format, SparkAppenderFactory appenderFactory, - OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema writeSchema) { - super(spec, format, appenderFactory, fileFactory, io, targetFileSize, writeSchema); + OutputFileFactory fileFactory, FileIO io, long targetFileSize, + Function keyGetter) { + super(spec, format, appenderFactory, fileFactory, io, targetFileSize, keyGetter); } @Override public WriterCommitMessage commit() throws IOException { - return new TaskCommit(complete()); + this.close(); + + List dataFiles = pollCompleteFiles(); + TaskResult taskResult = new TaskResult(dataFiles); + return new TaskCommit(taskResult); } } } From 5f1b29ee019e7379e95671099c41209aab14dd2a Mon Sep 17 00:00:00 2001 From: openinx Date: Thu, 16 Jul 2020 17:48:58 +0800 Subject: [PATCH 02/16] Minor fixes --- .../apache/iceberg/taskio/BaseTaskWriter.java | 42 ++++++----- .../taskio/PartitionedFanoutWriter.java | 75 +++++++++++++++++++ .../iceberg/taskio/PartitionedWriter.java | 39 ++++------ .../iceberg/taskio/UnpartitionedWriter.java | 19 +---- .../iceberg/flink/TaskWriterFactory.java | 61 +++++++++++++++ .../iceberg/spark/source/RowDataRewriter.java | 2 +- .../iceberg/spark/source/WriterUtil.java | 6 -- .../apache/iceberg/spark/source/Writer.java | 6 +- .../iceberg/spark/source/SparkBatchWrite.java | 6 +- 9 files changed, 184 insertions(+), 72 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/taskio/PartitionedFanoutWriter.java create mode 100644 flink/src/main/java/org/apache/iceberg/flink/TaskWriterFactory.java diff --git a/core/src/main/java/org/apache/iceberg/taskio/BaseTaskWriter.java b/core/src/main/java/org/apache/iceberg/taskio/BaseTaskWriter.java index 15ea4dd493e7..4fe83ee61df6 100644 --- a/core/src/main/java/org/apache/iceberg/taskio/BaseTaskWriter.java +++ b/core/src/main/java/org/apache/iceberg/taskio/BaseTaskWriter.java @@ -34,8 +34,9 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.Tasks; -public abstract class BaseTaskWriter implements TaskWriter { +abstract class BaseTaskWriter implements TaskWriter { protected static final int ROWS_DIVISOR = 1000; private final List completedFiles = Lists.newArrayList(); @@ -46,8 +47,8 @@ public abstract class BaseTaskWriter implements TaskWriter { private final FileIO io; private final long targetFileSize; - BaseTaskWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, FileIO io, long targetFileSize) { + protected BaseTaskWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, FileIO io, long targetFileSize) { this.spec = spec; this.format = format; this.appenderFactory = appenderFactory; @@ -56,6 +57,19 @@ public abstract class BaseTaskWriter implements TaskWriter { this.targetFileSize = targetFileSize; } + + @Override + public void abort() throws IOException { + close(); + + // clean up files created by this writer + Tasks.foreach(completedFiles) + .throwFailureWhenFinished() + .noRetry() + .run(file -> io.deleteFile(file.path().toString())); + } + + @Override public List pollCompleteFiles() { if (completedFiles.size() > 0) { List dataFiles = ImmutableList.copyOf(completedFiles); @@ -66,10 +80,6 @@ public List pollCompleteFiles() { } } - protected FileIO io() { - return this.io; - } - protected OutputFileFactory outputFileFactory() { return this.fileFactory; } @@ -81,15 +91,11 @@ WrappedFileAppender createWrappedFileAppender(PartitionKey partitionKey, return new WrappedFileAppender(partitionKey, outputFile, appender); } - void closeWrappedFileAppender(WrappedFileAppender appender) throws IOException { - DataFile dataFile = appender.closeAndBuildDataFile(); - completedFiles.add(dataFile); - } - class WrappedFileAppender { private final PartitionKey partitionKey; private final EncryptedOutputFile encryptedOutputFile; private final FileAppender appender; + private boolean closed = false; private long currentRows = 0; @@ -110,29 +116,27 @@ boolean shouldRollToNewFile() { currentRows % ROWS_DIVISOR == 0 && appender.length() >= targetFileSize; } - private void close() throws IOException { + void close() throws IOException { + // Close the file appender firstly. if (!closed) { appender.close(); closed = true; } - } - - private DataFile closeAndBuildDataFile() throws IOException { - // Close the file appender firstly. - this.close(); // metrics are only valid after the appender is closed. Metrics metrics = appender.metrics(); long fileSizeInBytes = appender.length(); List splitOffsets = appender.splitOffsets(); - return DataFiles.builder(spec) + DataFile dataFile = DataFiles.builder(spec) .withEncryptedOutputFile(encryptedOutputFile) .withFileSizeInBytes(fileSizeInBytes) .withPartition(partitionKey) .withMetrics(metrics) .withSplitOffsets(splitOffsets) .build(); + + completedFiles.add(dataFile); } } } diff --git a/core/src/main/java/org/apache/iceberg/taskio/PartitionedFanoutWriter.java b/core/src/main/java/org/apache/iceberg/taskio/PartitionedFanoutWriter.java new file mode 100644 index 000000000000..b618518c4a07 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/taskio/PartitionedFanoutWriter.java @@ -0,0 +1,75 @@ +/* + * 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.taskio; + +import java.io.IOException; +import java.util.Iterator; +import java.util.Map; +import java.util.function.Function; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +public class PartitionedFanoutWriter extends BaseTaskWriter { + private final Function keyGetter; + private final Map writers = Maps.newHashMap(); + + public PartitionedFanoutWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, FileIO io, long targetFileSize, + Function keyGetter) { + super(spec, format, appenderFactory, fileFactory, io, targetFileSize); + this.keyGetter = keyGetter; + } + + @Override + public void write(T row) throws IOException { + PartitionKey partitionKey = keyGetter.apply(row); + + WrappedFileAppender writer = writers.get(partitionKey); + if (writer == null) { + writer = createWrappedFileAppender(partitionKey, () -> outputFileFactory().newOutputFile(partitionKey)); + // NOTICE: we need to copy a new partition key here, in case of messing up the keys in writers. + writers.put(partitionKey.copy(), writer); + } + writer.add(row); + + // Roll the writer if reach the target file size. + if (writer.shouldRollToNewFile()) { + writer.close(); + writers.remove(partitionKey); + } + } + + @Override + public void close() throws IOException { + if (!writers.isEmpty()) { + + Iterator> iterator = writers.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry entry = iterator.next(); + entry.getValue().close(); + // Remove from the writers after closed. + iterator.remove(); + } + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/taskio/PartitionedWriter.java b/core/src/main/java/org/apache/iceberg/taskio/PartitionedWriter.java index 4a08034c8f23..2e566e828b1d 100644 --- a/core/src/main/java/org/apache/iceberg/taskio/PartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/taskio/PartitionedWriter.java @@ -28,32 +28,31 @@ import org.apache.iceberg.io.FileIO; 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 class PartitionedWriter extends BaseTaskWriter { private static final Logger LOG = LoggerFactory.getLogger(PartitionedWriter.class); - private PartitionKey currentKey = null; - private WrappedFileAppender currentAppender = null; - private final Function partitionKeyGetter; + private final Function keyGetter; private final Set completedPartitions = Sets.newHashSet(); + private PartitionKey currentKey = null; + private WrappedFileAppender currentAppender = null; public PartitionedWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, OutputFileFactory fileFactory, FileIO io, long targetFileSize, - Function partitionKeyGetter) { + Function keyGetter) { super(spec, format, appenderFactory, fileFactory, io, targetFileSize); - this.partitionKeyGetter = partitionKeyGetter; + this.keyGetter = keyGetter; } @Override public void write(T row) throws IOException { - PartitionKey key = partitionKeyGetter.apply(row); + PartitionKey key = keyGetter.apply(row); if (!key.equals(currentKey)) { - closeCurrentWriter(); + closeCurrent(); completedPartitions.add(currentKey); if (completedPartitions.contains(key)) { @@ -64,34 +63,28 @@ public void write(T row) throws IOException { } currentKey = key.copy(); + } - createWrappedFileAppender(currentKey, () -> outputFileFactory().newOutputFile(currentKey)); + if (currentAppender == null) { + currentAppender = createWrappedFileAppender(currentKey, () -> outputFileFactory().newOutputFile(currentKey)); } currentAppender.add(row); - } - - @Override - public void abort() throws IOException { - closeCurrentWriter(); - - // clean up files created by this writer - Tasks.foreach(pollCompleteFiles()) - .throwFailureWhenFinished() - .noRetry() - .run(file -> io().deleteFile(file.path().toString())); + if (currentAppender.shouldRollToNewFile()) { + closeCurrent(); + } } @Override public void close() throws IOException { - closeCurrentWriter(); + closeCurrent(); } - private void closeCurrentWriter() throws IOException { + private void closeCurrent() throws IOException { if (currentAppender != null) { // Close the current file appender and put the generated DataFile to completeDataFiles. - closeWrappedFileAppender(currentAppender); + currentAppender.close(); // Reset the current appender to be null. currentAppender = null; diff --git a/core/src/main/java/org/apache/iceberg/taskio/UnpartitionedWriter.java b/core/src/main/java/org/apache/iceberg/taskio/UnpartitionedWriter.java index 57549bdac9ad..91b56b7b696b 100644 --- a/core/src/main/java/org/apache/iceberg/taskio/UnpartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/taskio/UnpartitionedWriter.java @@ -23,7 +23,6 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.util.Tasks; public class UnpartitionedWriter extends BaseTaskWriter { @@ -42,30 +41,20 @@ public void write(T record) throws IOException { // Roll the writer if reach the target file size. if (currentAppender.shouldRollToNewFile()) { - closeCurrentWriter(); + closeCurrent(); } } @Override - public void abort() throws IOException { - closeCurrentWriter(); - - // clean up files created by this writer - Tasks.foreach(pollCompleteFiles()) - .throwFailureWhenFinished() - .noRetry() - .run(file -> io().deleteFile(file.path().toString())); - } - public void close() throws IOException { - closeCurrentWriter(); + closeCurrent(); } - private void closeCurrentWriter() throws IOException { + private void closeCurrent() throws IOException { if (currentAppender != null) { // Close the current file appender and put the generated DataFile to completeDataFiles. - closeWrappedFileAppender(currentAppender); + currentAppender.close(); // Reset the current appender to be null. currentAppender = null; diff --git a/flink/src/main/java/org/apache/iceberg/flink/TaskWriterFactory.java b/flink/src/main/java/org/apache/iceberg/flink/TaskWriterFactory.java new file mode 100644 index 000000000000..83f8ac7a6f87 --- /dev/null +++ b/flink/src/main/java/org/apache/iceberg/flink/TaskWriterFactory.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.flink; + +import java.util.function.Function; +import org.apache.flink.types.Row; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.taskio.FileAppenderFactory; +import org.apache.iceberg.taskio.OutputFileFactory; +import org.apache.iceberg.taskio.PartitionedFanoutWriter; +import org.apache.iceberg.taskio.TaskWriter; +import org.apache.iceberg.taskio.UnpartitionedWriter; + +public class TaskWriterFactory { + private TaskWriterFactory() { + } + + private static Function buildKeyGetter(PartitionSpec spec, Schema schema) { + PartitionKey partitionKey = new PartitionKey(spec, schema); + RowWrapper rowWrapper = new RowWrapper(schema.asStruct()); + + return row -> { + partitionKey.partition(rowWrapper.wrap(row)); + return partitionKey; + }; + } + + public static TaskWriter createTaskWriter(Schema schema, PartitionSpec spec, + FileAppenderFactory fileAppenderFactory, + OutputFileFactory outputFileFactory, + long targetFileSizeBytes, + FileFormat fileFormat) { + if (spec.fields().isEmpty()) { + return new UnpartitionedWriter<>(fileAppenderFactory, outputFileFactory, targetFileSizeBytes, fileFormat); + } else { + Function keyGetter = buildKeyGetter(spec, schema); + return new PartitionedFanoutWriter<>(spec, fileAppenderFactory, outputFileFactory, + keyGetter, targetFileSizeBytes, fileFormat); + } + } +} 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 0591696fb0d2..e4d119dccbd8 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 @@ -117,7 +117,7 @@ private TaskResult rewriteDataForTask(CombinedScanTask task) throws Exception { dataReader.close(); dataReader = null; - return WriterUtil.createTaskResult(writer.pollCompleteFiles()); + return new TaskResult(writer.pollCompleteFiles()); } catch (Throwable originalThrowable) { try { diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/WriterUtil.java b/spark/src/main/java/org/apache/iceberg/spark/source/WriterUtil.java index e6cc53f40575..016515e07e3f 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/WriterUtil.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/WriterUtil.java @@ -19,9 +19,7 @@ package org.apache.iceberg.spark.source; -import java.util.List; import java.util.function.Function; -import org.apache.iceberg.DataFile; import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -41,8 +39,4 @@ static Function buildKeyGetter(PartitionSpec spec, Sc return key; }; } - - static TaskResult createTaskResult(List dataFiles) { - return new TaskResult(dataFiles); - } } 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 256f64055c98..acd6d317efa0 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 @@ -287,8 +287,7 @@ public WriterCommitMessage commit() throws IOException { this.close(); List dataFiles = pollCompleteFiles(); - TaskResult taskResult = new TaskResult(dataFiles); - return new TaskCommit(taskResult); + return new TaskCommit(new TaskResult(dataFiles)); } } @@ -304,8 +303,7 @@ public WriterCommitMessage commit() throws IOException { this.close(); List dataFiles = pollCompleteFiles(); - TaskResult taskResult = new TaskResult(dataFiles); - return new TaskCommit(taskResult); + return new TaskCommit(new TaskResult(dataFiles)); } } } diff --git a/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchWrite.java b/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchWrite.java index c965ab1dc36f..df39163f3a87 100644 --- a/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchWrite.java +++ b/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchWrite.java @@ -309,8 +309,7 @@ public WriterCommitMessage commit() throws IOException { this.close(); List dataFiles = pollCompleteFiles(); - TaskResult taskResult = new TaskResult(dataFiles); - return new TaskCommit(taskResult); + return new TaskCommit(new TaskResult(dataFiles)); } } @@ -327,8 +326,7 @@ public WriterCommitMessage commit() throws IOException { this.close(); List dataFiles = pollCompleteFiles(); - TaskResult taskResult = new TaskResult(dataFiles); - return new TaskCommit(taskResult); + return new TaskCommit(new TaskResult(dataFiles)); } } } From 14e4dc5ef381213d417c2a68273584e68f51a931 Mon Sep 17 00:00:00 2001 From: openinx Date: Thu, 16 Jul 2020 18:25:42 +0800 Subject: [PATCH 03/16] Add flink task writers and unit tests. --- .../apache/iceberg/taskio/BaseTaskWriter.java | 33 ++- .../taskio/PartitionedFanoutWriter.java | 9 +- .../iceberg/taskio/PartitionedWriter.java | 4 +- .../iceberg/taskio/UnpartitionedWriter.java | 5 +- .../iceberg/flink/TaskWriterFactory.java | 70 +++++- .../apache/iceberg/flink/SimpleDataUtil.java | 85 +++++++ .../apache/iceberg/flink/TestTaskWriters.java | 208 ++++++++++++++++++ 7 files changed, 397 insertions(+), 17 deletions(-) create mode 100644 flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java create mode 100644 flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java diff --git a/core/src/main/java/org/apache/iceberg/taskio/BaseTaskWriter.java b/core/src/main/java/org/apache/iceberg/taskio/BaseTaskWriter.java index 4fe83ee61df6..a1258c8d885d 100644 --- a/core/src/main/java/org/apache/iceberg/taskio/BaseTaskWriter.java +++ b/core/src/main/java/org/apache/iceberg/taskio/BaseTaskWriter.java @@ -35,10 +35,14 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.Tasks; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; abstract class BaseTaskWriter implements TaskWriter { protected static final int ROWS_DIVISOR = 1000; + private static final Logger LOG = LoggerFactory.getLogger(BaseTaskWriter.class); + private final List completedFiles = Lists.newArrayList(); private final PartitionSpec spec; private final FileFormat format; @@ -47,6 +51,8 @@ abstract class BaseTaskWriter implements TaskWriter { private final FileIO io; private final long targetFileSize; + private boolean closed = false; + protected BaseTaskWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, OutputFileFactory fileFactory, FileIO io, long targetFileSize) { this.spec = spec; @@ -59,7 +65,7 @@ protected BaseTaskWriter(PartitionSpec spec, FileFormat format, FileAppenderFact @Override - public void abort() throws IOException { + public void abort() { close(); // clean up files created by this writer @@ -69,6 +75,16 @@ public void abort() throws IOException { .run(file -> io.deleteFile(file.path().toString())); } + @Override + public final void write(T record) throws IOException { + if (closed) { + throw new IOException("The writer has been closed."); + } + internalWrite(record); + } + + protected abstract void internalWrite(T record) throws IOException; + @Override public List pollCompleteFiles() { if (completedFiles.size() > 0) { @@ -80,6 +96,21 @@ public List pollCompleteFiles() { } } + @Override + public final void close() { + if (!this.closed) { + try { + internalClose(); + } catch (IOException e) { + LOG.warn("Failed to close the writer: ", e); + } finally { + this.closed = true; + } + } + } + + protected abstract void internalClose() throws IOException; + protected OutputFileFactory outputFileFactory() { return this.fileFactory; } diff --git a/core/src/main/java/org/apache/iceberg/taskio/PartitionedFanoutWriter.java b/core/src/main/java/org/apache/iceberg/taskio/PartitionedFanoutWriter.java index b618518c4a07..2441f5337d1e 100644 --- a/core/src/main/java/org/apache/iceberg/taskio/PartitionedFanoutWriter.java +++ b/core/src/main/java/org/apache/iceberg/taskio/PartitionedFanoutWriter.java @@ -41,14 +41,15 @@ public PartitionedFanoutWriter(PartitionSpec spec, FileFormat format, FileAppend } @Override - public void write(T row) throws IOException { + public void internalWrite(T row) throws IOException { PartitionKey partitionKey = keyGetter.apply(row); WrappedFileAppender writer = writers.get(partitionKey); if (writer == null) { - writer = createWrappedFileAppender(partitionKey, () -> outputFileFactory().newOutputFile(partitionKey)); // NOTICE: we need to copy a new partition key here, in case of messing up the keys in writers. - writers.put(partitionKey.copy(), writer); + PartitionKey copiedKey = partitionKey.copy(); + writer = createWrappedFileAppender(copiedKey, () -> outputFileFactory().newOutputFile(partitionKey)); + writers.put(copiedKey, writer); } writer.add(row); @@ -60,7 +61,7 @@ public void write(T row) throws IOException { } @Override - public void close() throws IOException { + public void internalClose() throws IOException { if (!writers.isEmpty()) { Iterator> iterator = writers.entrySet().iterator(); diff --git a/core/src/main/java/org/apache/iceberg/taskio/PartitionedWriter.java b/core/src/main/java/org/apache/iceberg/taskio/PartitionedWriter.java index 2e566e828b1d..132f0802badc 100644 --- a/core/src/main/java/org/apache/iceberg/taskio/PartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/taskio/PartitionedWriter.java @@ -48,7 +48,7 @@ public PartitionedWriter(PartitionSpec spec, FileFormat format, FileAppenderFact } @Override - public void write(T row) throws IOException { + public void internalWrite(T row) throws IOException { PartitionKey key = keyGetter.apply(row); if (!key.equals(currentKey)) { @@ -76,7 +76,7 @@ public void write(T row) throws IOException { } @Override - public void close() throws IOException { + public void internalClose() throws IOException { closeCurrent(); } diff --git a/core/src/main/java/org/apache/iceberg/taskio/UnpartitionedWriter.java b/core/src/main/java/org/apache/iceberg/taskio/UnpartitionedWriter.java index 91b56b7b696b..94609df77074 100644 --- a/core/src/main/java/org/apache/iceberg/taskio/UnpartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/taskio/UnpartitionedWriter.java @@ -33,7 +33,8 @@ public UnpartitionedWriter(PartitionSpec spec, FileFormat format, FileAppenderFa super(spec, format, appenderFactory, fileFactory, io, targetFileSize); } - public void write(T record) throws IOException { + @Override + public void internalWrite(T record) throws IOException { if (currentAppender == null) { currentAppender = createWrappedFileAppender(null, outputFileFactory()::newOutputFile); } @@ -46,7 +47,7 @@ public void write(T record) throws IOException { } @Override - public void close() throws IOException { + public void internalClose() throws IOException { closeCurrent(); } diff --git a/flink/src/main/java/org/apache/iceberg/flink/TaskWriterFactory.java b/flink/src/main/java/org/apache/iceberg/flink/TaskWriterFactory.java index 83f8ac7a6f87..7f5629ad874c 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/TaskWriterFactory.java +++ b/flink/src/main/java/org/apache/iceberg/flink/TaskWriterFactory.java @@ -19,12 +19,23 @@ package org.apache.iceberg.flink; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Map; import java.util.function.Function; import org.apache.flink.types.Row; 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.flink.data.FlinkAvroWriter; +import org.apache.iceberg.flink.data.FlinkParquetWriters; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.taskio.FileAppenderFactory; import org.apache.iceberg.taskio.OutputFileFactory; import org.apache.iceberg.taskio.PartitionedFanoutWriter; @@ -45,17 +56,60 @@ private static Function buildKeyGetter(PartitionSpec spec, Sc }; } - public static TaskWriter createTaskWriter(Schema schema, PartitionSpec spec, - FileAppenderFactory fileAppenderFactory, - OutputFileFactory outputFileFactory, - long targetFileSizeBytes, - FileFormat fileFormat) { + public static TaskWriter createTaskWriter(Schema schema, + PartitionSpec spec, + FileFormat format, + FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, + FileIO io, + long targetFileSizeBytes) { if (spec.fields().isEmpty()) { - return new UnpartitionedWriter<>(fileAppenderFactory, outputFileFactory, targetFileSizeBytes, fileFormat); + return new UnpartitionedWriter<>(spec, format, appenderFactory, fileFactory, io, targetFileSizeBytes); } else { Function keyGetter = buildKeyGetter(spec, schema); - return new PartitionedFanoutWriter<>(spec, fileAppenderFactory, outputFileFactory, - keyGetter, targetFileSizeBytes, fileFormat); + return new PartitionedFanoutWriter<>(spec, format, appenderFactory, fileFactory, io, + targetFileSizeBytes, keyGetter); + } + } + + static class FlinkFileAppenderFactory implements FileAppenderFactory { + private final Schema schema; + private final Map props; + + FlinkFileAppenderFactory(Schema schema, Map props) { + this.schema = schema; + this.props = props; + } + + @Override + public FileAppender newAppender(OutputFile outputFile, FileFormat format) { + MetricsConfig metricsConfig = MetricsConfig.fromProperties(props); + try { + switch (format) { + case PARQUET: + return Parquet.write(outputFile) + .createWriterFunc(FlinkParquetWriters::buildWriter) + .setAll(props) + .metricsConfig(metricsConfig) + .schema(schema) + .overwrite() + .build(); + + case AVRO: + return Avro.write(outputFile) + .createWriterFunc(FlinkAvroWriter::new) + .setAll(props) + .schema(schema) + .overwrite() + .build(); + + case ORC: + 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 new file mode 100644 index 000000000000..ddda027e4f9b --- /dev/null +++ b/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.flink; + +import java.io.IOException; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.junit.Assert; + +public class SimpleDataUtil { + + private SimpleDataUtil() { + } + + static final Schema SCHEMA = new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get()) + ); + + static final Record RECORD = GenericRecord.create(SCHEMA); + + static final Comparator COMPARATOR = (r1, r2) -> { + int ret = Integer.compare((Integer) r1.getField("id"), (Integer) r2.getField("id")); + if (ret != 0) { + return ret; + } + return ((String) r1.getField("data")).compareTo((String) r2.getField("data")); + }; + + static Table createTable(String path, Map properties, boolean partitioned) { + PartitionSpec spec; + if (partitioned) { + spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); + } else { + spec = PartitionSpec.unpartitioned(); + } + return new HadoopTables().create(SCHEMA, spec, properties, path); + } + + static Record createRecord(int id, String data) { + return RECORD.copy(ImmutableMap.of("id", id, "data", data)); + } + + static void assertTableRecords(String tablePath, List expected) throws IOException { + Preconditions.checkArgument(expected != null, "expected records shouldn't be null"); + Table newTable = new HadoopTables().load(tablePath); + List results; + try (CloseableIterable iterable = (CloseableIterable) IcebergGenerics.read(newTable).build()) { + results = Lists.newArrayList(iterable); + } + expected.sort(COMPARATOR); + results.sort(COMPARATOR); + Assert.assertEquals("Should produce the expected record", expected, results); + } +} diff --git a/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java b/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java new file mode 100644 index 000000000000..ef691d87dd0d --- /dev/null +++ b/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.flink; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.taskio.OutputFileFactory; +import org.apache.iceberg.taskio.TaskWriter; +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 TestTaskWriters { + private static final Configuration CONF = new Configuration(); + private static final long TARGET_FILE_SIZE = 128 * 1024 * 1024; + + @Rule + public final TemporaryFolder tempFolder = new TemporaryFolder(); + + // TODO add ORC unit test once the readers and writers are ready. + @Parameterized.Parameters(name = "format = {0}, partitioned = {1}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] {"parquet", true}, + new Object[] {"parquet", false}, + new Object[] {"avro", true}, + new Object[] {"avro", false}, + }; + } + + private final FileFormat format; + private final boolean partitioned; + + private String path; + private Table table; + + public TestTaskWriters(String format, boolean partitioned) { + this.format = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH)); + this.partitioned = partitioned; + } + + @Before + public void before() throws IOException { + File folder = tempFolder.newFolder(); + path = folder.getAbsolutePath(); + + // Construct the iceberg table. + Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + table = SimpleDataUtil.createTable(path, props, partitioned); + } + + @Test + public void testWriteZeroRecord() throws IOException { + try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { + taskWriter.close(); + + List dataFiles = taskWriter.pollCompleteFiles(); + Assert.assertNotNull(dataFiles); + Assert.assertEquals(0, dataFiles.size()); + + // Close again. + taskWriter.close(); + dataFiles = taskWriter.pollCompleteFiles(); + Assert.assertNotNull(dataFiles); + Assert.assertEquals(0, dataFiles.size()); + + boolean encounterError = false; + try { + taskWriter.write(Row.of(1, "foo")); + } catch (IOException e) { + encounterError = true; + } + Assert.assertTrue("Forbid to write closed writer.", encounterError); + } + } + + @Test + public void testAbort() throws IOException { + try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { + taskWriter.write(Row.of(1, "hello")); + taskWriter.write(Row.of(2, "world")); + + taskWriter.abort(); + List dataFiles = taskWriter.pollCompleteFiles(); + + int expectedFiles = partitioned ? 2 : 1; + Assert.assertEquals(expectedFiles, dataFiles.size()); + + FileSystem fs = FileSystem.get(CONF); + for (DataFile dataFile : dataFiles) { + Assert.assertFalse(fs.exists(new Path(dataFile.path().toString()))); + } + } + } + + @Test + public void testPollCompleteFiles() throws IOException { + try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { + taskWriter.write(Row.of(1, "a")); + taskWriter.write(Row.of(2, "b")); + taskWriter.write(Row.of(3, "c")); + taskWriter.write(Row.of(4, "d")); + + List dataFiles = taskWriter.pollCompleteFiles(); + Assert.assertEquals(0, dataFiles.size()); + + taskWriter.close(); + dataFiles = taskWriter.pollCompleteFiles(); + int expectedFiles = partitioned ? 4 : 1; + Assert.assertEquals(expectedFiles, dataFiles.size()); + + FileSystem fs = FileSystem.get(CONF); + for (DataFile dataFile : dataFiles) { + Assert.assertTrue(fs.exists(new Path(dataFile.path().toString()))); + } + AppendFiles appendFiles = table.newAppend(); + dataFiles.forEach(appendFiles::appendFile); + appendFiles.commit(); + + // Assert the data rows. + SimpleDataUtil.assertTableRecords(path, Lists.newArrayList( + SimpleDataUtil.createRecord(1, "a"), + SimpleDataUtil.createRecord(2, "b"), + SimpleDataUtil.createRecord(3, "c"), + SimpleDataUtil.createRecord(4, "d") + )); + + dataFiles = taskWriter.pollCompleteFiles(); + Assert.assertEquals(0, dataFiles.size()); + } + } + + @Test + public void testRollingWithTargetFileSize() throws IOException { + try (TaskWriter taskWriter = createTaskWriter(4)) { + List rows = Lists.newArrayListWithCapacity(8000); + List records = Lists.newArrayListWithCapacity(8000); + for (int i = 0; i < 2000; i++) { + for (String data : new String[] {"a", "b", "c", "d"}) { + rows.add(Row.of(i, data)); + records.add(SimpleDataUtil.createRecord(i, data)); + } + } + + for (Row row : rows) { + taskWriter.write(row); + } + + List dataFiles = taskWriter.pollCompleteFiles(); + Assert.assertEquals(8, dataFiles.size()); + + AppendFiles appendFiles = table.newAppend(); + dataFiles.forEach(appendFiles::appendFile); + appendFiles.commit(); + + // Assert the data rows. + SimpleDataUtil.assertTableRecords(path, records); + } + } + + private TaskWriter createTaskWriter(long targetFileSize) { + TaskWriterFactory.FlinkFileAppenderFactory appenderFactory = + new TaskWriterFactory.FlinkFileAppenderFactory(table.schema(), table.properties()); + + OutputFileFactory outputFileFactory = new OutputFileFactory(table.spec(), format, table.locationProvider(), + table.io(), table.encryption(), 1, 1); + + return TaskWriterFactory.createTaskWriter(table.schema(), table.spec(), format, + appenderFactory, outputFileFactory, table.io(), targetFileSize); + } +} From f1ccdfd691eb3062ba66da6ea21470eb37f65c16 Mon Sep 17 00:00:00 2001 From: openinx Date: Thu, 16 Jul 2020 21:14:31 +0800 Subject: [PATCH 04/16] Adjust the unit tests. --- .../taskio/PartitionedFanoutWriter.java | 6 +-- .../apache/iceberg/flink/TestTaskWriters.java | 43 ++++++++++++++++--- 2 files changed, 38 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/taskio/PartitionedFanoutWriter.java b/core/src/main/java/org/apache/iceberg/taskio/PartitionedFanoutWriter.java index 2441f5337d1e..971649dd7dfd 100644 --- a/core/src/main/java/org/apache/iceberg/taskio/PartitionedFanoutWriter.java +++ b/core/src/main/java/org/apache/iceberg/taskio/PartitionedFanoutWriter.java @@ -63,11 +63,9 @@ public void internalWrite(T row) throws IOException { @Override public void internalClose() throws IOException { if (!writers.isEmpty()) { - - Iterator> iterator = writers.entrySet().iterator(); + Iterator iterator = writers.values().iterator(); while (iterator.hasNext()) { - Map.Entry entry = iterator.next(); - entry.getValue().close(); + iterator.next().close(); // Remove from the writers after closed. iterator.remove(); } diff --git a/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java b/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java index ef691d87dd0d..6beac8d0dd09 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java +++ b/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java @@ -101,13 +101,7 @@ public void testWriteZeroRecord() throws IOException { Assert.assertNotNull(dataFiles); Assert.assertEquals(0, dataFiles.size()); - boolean encounterError = false; - try { - taskWriter.write(Row.of(1, "foo")); - } catch (IOException e) { - encounterError = true; - } - Assert.assertTrue("Forbid to write closed writer.", encounterError); + assertWriteDisabled(taskWriter, Row.of(1, "ping")); } } @@ -127,6 +121,8 @@ public void testAbort() throws IOException { for (DataFile dataFile : dataFiles) { Assert.assertFalse(fs.exists(new Path(dataFile.path().toString()))); } + + assertWriteDisabled(taskWriter, Row.of(1, "ping")); } } @@ -167,6 +163,28 @@ public void testPollCompleteFiles() throws IOException { } } + @Test + public void testCopiedCompleteFiles() throws IOException { + try (TaskWriter taskWriter = createTaskWriter(4)) { + List rows = Lists.newArrayListWithCapacity(1000); + for (int i = 0; i < 1000; i++) { + rows.add(Row.of(i, "a")); + } + + for (Row row : rows) { + taskWriter.write(row); + } + + List dataFiles = taskWriter.pollCompleteFiles(); + Assert.assertEquals(1, dataFiles.size()); + + Assert.assertEquals(0, taskWriter.pollCompleteFiles().size()); + dataFiles.add(null); + Assert.assertEquals("The copied data files should not effect the complete file cache", + 0, taskWriter.pollCompleteFiles().size()); + } + } + @Test public void testRollingWithTargetFileSize() throws IOException { try (TaskWriter taskWriter = createTaskWriter(4)) { @@ -195,6 +213,17 @@ public void testRollingWithTargetFileSize() throws IOException { } } + private void assertWriteDisabled(TaskWriter taskWriter, Row row) { + String errorMessage = null; + try { + taskWriter.write(row); + } catch (IOException e) { + errorMessage = e.getMessage(); + } + Assert.assertTrue("Forbid to write a closed writer", + errorMessage != null && errorMessage.contains("The writer has been closed.")); + } + private TaskWriter createTaskWriter(long targetFileSize) { TaskWriterFactory.FlinkFileAppenderFactory appenderFactory = new TaskWriterFactory.FlinkFileAppenderFactory(table.schema(), table.properties()); From 8fe27c2104d32886978b004d112927c69936eefe Mon Sep 17 00:00:00 2001 From: openinx Date: Fri, 17 Jul 2020 09:57:07 +0800 Subject: [PATCH 05/16] Addressing the failure unit tests. --- .../org/apache/iceberg/flink/TestTaskWriters.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java b/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java index 6beac8d0dd09..19c754dd27ed 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java +++ b/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java @@ -29,6 +29,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; @@ -177,11 +178,13 @@ public void testCopiedCompleteFiles() throws IOException { List dataFiles = taskWriter.pollCompleteFiles(); Assert.assertEquals(1, dataFiles.size()); + AssertHelpers.assertThrows("Complete file list are immutable", UnsupportedOperationException.class, + () -> dataFiles.add(null)); - Assert.assertEquals(0, taskWriter.pollCompleteFiles().size()); - dataFiles.add(null); - Assert.assertEquals("The copied data files should not effect the complete file cache", - 0, taskWriter.pollCompleteFiles().size()); + List emptyList = taskWriter.pollCompleteFiles(); + Assert.assertEquals(0, emptyList.size()); + AssertHelpers.assertThrows("Empty complete file list are immutable", UnsupportedOperationException.class, + () -> emptyList.add(null)); } } From a586c5112f348fae3de210dcc2b6e6a8ce47e6e3 Mon Sep 17 00:00:00 2001 From: openinx Date: Fri, 17 Jul 2020 10:44:28 +0800 Subject: [PATCH 06/16] Add unit test and more javadoc --- .../taskio/PartitionedFanoutWriter.java | 2 +- .../org/apache/iceberg/taskio/TaskWriter.java | 19 +++++++++++++++++++ .../iceberg/taskio/UnpartitionedWriter.java | 2 +- .../apache/iceberg/flink/TestTaskWriters.java | 6 +++++- 4 files changed, 26 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/taskio/PartitionedFanoutWriter.java b/core/src/main/java/org/apache/iceberg/taskio/PartitionedFanoutWriter.java index 971649dd7dfd..7f9fdd55f3d8 100644 --- a/core/src/main/java/org/apache/iceberg/taskio/PartitionedFanoutWriter.java +++ b/core/src/main/java/org/apache/iceberg/taskio/PartitionedFanoutWriter.java @@ -53,7 +53,7 @@ public void internalWrite(T row) throws IOException { } writer.add(row); - // Roll the writer if reach the target file size. + // Close the writer if reach the target file size. if (writer.shouldRollToNewFile()) { writer.close(); writers.remove(partitionKey); diff --git a/core/src/main/java/org/apache/iceberg/taskio/TaskWriter.java b/core/src/main/java/org/apache/iceberg/taskio/TaskWriter.java index 7b848253c442..dc1c3a75ea31 100644 --- a/core/src/main/java/org/apache/iceberg/taskio/TaskWriter.java +++ b/core/src/main/java/org/apache/iceberg/taskio/TaskWriter.java @@ -24,11 +24,30 @@ import java.util.List; import org.apache.iceberg.DataFile; +/** + * The writer interface could accept records and provide the generated data files. + * + * @param to indicate the record data type. + */ public interface TaskWriter extends Closeable { + /** + * Write the row into the data files. + */ void write(T row) throws IOException; + /** + * Close the writer and delete the completed files if possible when aborting. + * + * @throws IOException if any IO error happen. + */ void abort() throws IOException; + /** + * Get the completed data files and clear them from the cache. NOTICE: if call this method without + * {@link TaskWriter#close()} then the current opening data file won't be seen in the result list. + * + * @return the cached completed data files of this task writer. + */ List pollCompleteFiles(); } diff --git a/core/src/main/java/org/apache/iceberg/taskio/UnpartitionedWriter.java b/core/src/main/java/org/apache/iceberg/taskio/UnpartitionedWriter.java index 94609df77074..13894e3574e8 100644 --- a/core/src/main/java/org/apache/iceberg/taskio/UnpartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/taskio/UnpartitionedWriter.java @@ -40,7 +40,7 @@ public void internalWrite(T record) throws IOException { } currentAppender.add(record); - // Roll the writer if reach the target file size. + // Close the writer if reach the target file size. if (currentAppender.shouldRollToNewFile()) { closeCurrent(); } diff --git a/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java b/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java index 19c754dd27ed..95b8737e49b5 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java +++ b/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java @@ -82,7 +82,7 @@ public void before() throws IOException { File folder = tempFolder.newFolder(); path = folder.getAbsolutePath(); - // Construct the iceberg table. + // Construct the iceberg table with the specified file format. Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); table = SimpleDataUtil.createTable(path, props, partitioned); } @@ -185,6 +185,10 @@ public void testCopiedCompleteFiles() throws IOException { Assert.assertEquals(0, emptyList.size()); AssertHelpers.assertThrows("Empty complete file list are immutable", UnsupportedOperationException.class, () -> emptyList.add(null)); + + // It should not open any new file when closed a writer without writing new record. + taskWriter.close(); + Assert.assertEquals(0, taskWriter.pollCompleteFiles().size()); } } From 93a9318e9f53849fa8095c331434c3cbfabde583 Mon Sep 17 00:00:00 2001 From: openinx Date: Fri, 17 Jul 2020 11:52:20 +0800 Subject: [PATCH 07/16] Fix the broken TestRewriteDataFilesAction --- .../java/org/apache/iceberg/spark/source/RowDataRewriter.java | 2 ++ .../org/apache/iceberg/spark/source/SparkAppenderFactory.java | 1 + 2 files changed, 3 insertions(+) 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 e4d119dccbd8..a9fd6a1cc6d4 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 @@ -117,6 +117,8 @@ private TaskResult rewriteDataForTask(CombinedScanTask task) throws Exception { dataReader.close(); dataReader = null; + + writer.close(); return new TaskResult(writer.pollCompleteFiles()); } catch (Throwable originalThrowable) { diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java b/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java index 9b7392bfb371..ac0ffe9e500e 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java @@ -48,6 +48,7 @@ class SparkAppenderFactory implements FileAppenderFactory { this.dsSchema = dsSchema; } + @Override public FileAppender newAppender(OutputFile file, FileFormat fileFormat) { MetricsConfig metricsConfig = MetricsConfig.fromProperties(properties); try { From bb69950dfd0d4456ca74b78426f5aa05d4cf2dc3 Mon Sep 17 00:00:00 2001 From: openinx Date: Wed, 22 Jul 2020 14:56:22 +0800 Subject: [PATCH 08/16] Add javadoc for FileAppenderFactory --- .../apache/iceberg/taskio/BaseTaskWriter.java | 34 +------------------ .../iceberg/taskio/FileAppenderFactory.java | 12 +++++++ .../taskio/PartitionedFanoutWriter.java | 4 +-- .../iceberg/taskio/PartitionedWriter.java | 4 +-- .../iceberg/taskio/UnpartitionedWriter.java | 4 +-- .../apache/iceberg/flink/TestTaskWriters.java | 15 -------- 6 files changed, 19 insertions(+), 54 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/taskio/BaseTaskWriter.java b/core/src/main/java/org/apache/iceberg/taskio/BaseTaskWriter.java index a1258c8d885d..68e6b749ad08 100644 --- a/core/src/main/java/org/apache/iceberg/taskio/BaseTaskWriter.java +++ b/core/src/main/java/org/apache/iceberg/taskio/BaseTaskWriter.java @@ -35,14 +35,10 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.Tasks; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; abstract class BaseTaskWriter implements TaskWriter { protected static final int ROWS_DIVISOR = 1000; - private static final Logger LOG = LoggerFactory.getLogger(BaseTaskWriter.class); - private final List completedFiles = Lists.newArrayList(); private final PartitionSpec spec; private final FileFormat format; @@ -51,8 +47,6 @@ abstract class BaseTaskWriter implements TaskWriter { private final FileIO io; private final long targetFileSize; - private boolean closed = false; - protected BaseTaskWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, OutputFileFactory fileFactory, FileIO io, long targetFileSize) { this.spec = spec; @@ -63,9 +57,8 @@ protected BaseTaskWriter(PartitionSpec spec, FileFormat format, FileAppenderFact this.targetFileSize = targetFileSize; } - @Override - public void abort() { + public void abort() throws IOException { close(); // clean up files created by this writer @@ -75,16 +68,6 @@ public void abort() { .run(file -> io.deleteFile(file.path().toString())); } - @Override - public final void write(T record) throws IOException { - if (closed) { - throw new IOException("The writer has been closed."); - } - internalWrite(record); - } - - protected abstract void internalWrite(T record) throws IOException; - @Override public List pollCompleteFiles() { if (completedFiles.size() > 0) { @@ -96,21 +79,6 @@ public List pollCompleteFiles() { } } - @Override - public final void close() { - if (!this.closed) { - try { - internalClose(); - } catch (IOException e) { - LOG.warn("Failed to close the writer: ", e); - } finally { - this.closed = true; - } - } - } - - protected abstract void internalClose() throws IOException; - protected OutputFileFactory outputFileFactory() { return this.fileFactory; } diff --git a/core/src/main/java/org/apache/iceberg/taskio/FileAppenderFactory.java b/core/src/main/java/org/apache/iceberg/taskio/FileAppenderFactory.java index f72d757f3459..c8922e63ebd3 100644 --- a/core/src/main/java/org/apache/iceberg/taskio/FileAppenderFactory.java +++ b/core/src/main/java/org/apache/iceberg/taskio/FileAppenderFactory.java @@ -23,7 +23,19 @@ import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.OutputFile; +/** + * Factory to create a new {@link FileAppender} to write records. + * + * @param data type of the rows to append. + */ public interface FileAppenderFactory { + /** + * Create a new {@link FileAppender}. + * + * @param outputFile indicate the file location to write. + * @param fileFormat File format. + * @return a newly created {@link FileAppender} + */ FileAppender newAppender(OutputFile outputFile, FileFormat fileFormat); } diff --git a/core/src/main/java/org/apache/iceberg/taskio/PartitionedFanoutWriter.java b/core/src/main/java/org/apache/iceberg/taskio/PartitionedFanoutWriter.java index 7f9fdd55f3d8..504099f15b2a 100644 --- a/core/src/main/java/org/apache/iceberg/taskio/PartitionedFanoutWriter.java +++ b/core/src/main/java/org/apache/iceberg/taskio/PartitionedFanoutWriter.java @@ -41,7 +41,7 @@ public PartitionedFanoutWriter(PartitionSpec spec, FileFormat format, FileAppend } @Override - public void internalWrite(T row) throws IOException { + public void write(T row) throws IOException { PartitionKey partitionKey = keyGetter.apply(row); WrappedFileAppender writer = writers.get(partitionKey); @@ -61,7 +61,7 @@ public void internalWrite(T row) throws IOException { } @Override - public void internalClose() throws IOException { + public void close() throws IOException { if (!writers.isEmpty()) { Iterator iterator = writers.values().iterator(); while (iterator.hasNext()) { diff --git a/core/src/main/java/org/apache/iceberg/taskio/PartitionedWriter.java b/core/src/main/java/org/apache/iceberg/taskio/PartitionedWriter.java index 132f0802badc..2e566e828b1d 100644 --- a/core/src/main/java/org/apache/iceberg/taskio/PartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/taskio/PartitionedWriter.java @@ -48,7 +48,7 @@ public PartitionedWriter(PartitionSpec spec, FileFormat format, FileAppenderFact } @Override - public void internalWrite(T row) throws IOException { + public void write(T row) throws IOException { PartitionKey key = keyGetter.apply(row); if (!key.equals(currentKey)) { @@ -76,7 +76,7 @@ public void internalWrite(T row) throws IOException { } @Override - public void internalClose() throws IOException { + public void close() throws IOException { closeCurrent(); } diff --git a/core/src/main/java/org/apache/iceberg/taskio/UnpartitionedWriter.java b/core/src/main/java/org/apache/iceberg/taskio/UnpartitionedWriter.java index 13894e3574e8..0ad1f86719a7 100644 --- a/core/src/main/java/org/apache/iceberg/taskio/UnpartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/taskio/UnpartitionedWriter.java @@ -34,7 +34,7 @@ public UnpartitionedWriter(PartitionSpec spec, FileFormat format, FileAppenderFa } @Override - public void internalWrite(T record) throws IOException { + public void write(T record) throws IOException { if (currentAppender == null) { currentAppender = createWrappedFileAppender(null, outputFileFactory()::newOutputFile); } @@ -47,7 +47,7 @@ public void internalWrite(T record) throws IOException { } @Override - public void internalClose() throws IOException { + public void close() throws IOException { closeCurrent(); } diff --git a/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java b/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java index 95b8737e49b5..50ac6a36df87 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java +++ b/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java @@ -101,8 +101,6 @@ public void testWriteZeroRecord() throws IOException { dataFiles = taskWriter.pollCompleteFiles(); Assert.assertNotNull(dataFiles); Assert.assertEquals(0, dataFiles.size()); - - assertWriteDisabled(taskWriter, Row.of(1, "ping")); } } @@ -122,8 +120,6 @@ public void testAbort() throws IOException { for (DataFile dataFile : dataFiles) { Assert.assertFalse(fs.exists(new Path(dataFile.path().toString()))); } - - assertWriteDisabled(taskWriter, Row.of(1, "ping")); } } @@ -220,17 +216,6 @@ public void testRollingWithTargetFileSize() throws IOException { } } - private void assertWriteDisabled(TaskWriter taskWriter, Row row) { - String errorMessage = null; - try { - taskWriter.write(row); - } catch (IOException e) { - errorMessage = e.getMessage(); - } - Assert.assertTrue("Forbid to write a closed writer", - errorMessage != null && errorMessage.contains("The writer has been closed.")); - } - private TaskWriter createTaskWriter(long targetFileSize) { TaskWriterFactory.FlinkFileAppenderFactory appenderFactory = new TaskWriterFactory.FlinkFileAppenderFactory(table.schema(), table.properties()); From abb0b5d6a7608cc9cd5ed8e9e1ad289c5cdacd06 Mon Sep 17 00:00:00 2001 From: openinx Date: Wed, 22 Jul 2020 15:29:00 +0800 Subject: [PATCH 09/16] More unit tests --- .../apache/iceberg/taskio/BaseTaskWriter.java | 26 ++++++------- .../apache/iceberg/flink/SimpleDataUtil.java | 36 +++++++++-------- .../apache/iceberg/flink/TestTaskWriters.java | 39 +++++++++++++++++++ 3 files changed, 71 insertions(+), 30 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/taskio/BaseTaskWriter.java b/core/src/main/java/org/apache/iceberg/taskio/BaseTaskWriter.java index 68e6b749ad08..70511883178d 100644 --- a/core/src/main/java/org/apache/iceberg/taskio/BaseTaskWriter.java +++ b/core/src/main/java/org/apache/iceberg/taskio/BaseTaskWriter.java @@ -120,22 +120,22 @@ void close() throws IOException { if (!closed) { appender.close(); closed = true; - } - // metrics are only valid after the appender is closed. - Metrics metrics = appender.metrics(); - long fileSizeInBytes = appender.length(); - List splitOffsets = appender.splitOffsets(); + // metrics are only valid after the appender is closed. + Metrics metrics = appender.metrics(); + long fileSizeInBytes = appender.length(); + List splitOffsets = appender.splitOffsets(); - DataFile dataFile = DataFiles.builder(spec) - .withEncryptedOutputFile(encryptedOutputFile) - .withFileSizeInBytes(fileSizeInBytes) - .withPartition(partitionKey) - .withMetrics(metrics) - .withSplitOffsets(splitOffsets) - .build(); + DataFile dataFile = DataFiles.builder(spec) + .withEncryptedOutputFile(encryptedOutputFile) + .withFileSizeInBytes(fileSizeInBytes) + .withPartition(partitionKey) + .withMetrics(metrics) + .withSplitOffsets(splitOffsets) + .build(); - completedFiles.add(dataFile); + completedFiles.add(dataFile); + } } } } 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 ddda027e4f9b..2ac907a3658e 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java +++ b/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java @@ -20,9 +20,10 @@ package org.apache.iceberg.flink; import java.io.IOException; -import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.Set; +import org.apache.flink.types.Row; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; @@ -32,8 +33,8 @@ import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; import org.junit.Assert; @@ -49,14 +50,6 @@ private SimpleDataUtil() { static final Record RECORD = GenericRecord.create(SCHEMA); - static final Comparator COMPARATOR = (r1, r2) -> { - int ret = Integer.compare((Integer) r1.getField("id"), (Integer) r2.getField("id")); - if (ret != 0) { - return ret; - } - return ((String) r1.getField("data")).compareTo((String) r2.getField("data")); - }; - static Table createTable(String path, Map properties, boolean partitioned) { PartitionSpec spec; if (partitioned) { @@ -67,19 +60,28 @@ static Table createTable(String path, Map properties, boolean pa return new HadoopTables().create(SCHEMA, spec, properties, path); } - static Record createRecord(int id, String data) { - return RECORD.copy(ImmutableMap.of("id", id, "data", data)); + static Record createRecord(Integer id, String data) { + Record record = RECORD.copy(); + record.setField("id", id); + record.setField("data", data); + return record; + } + + static void assertTableRows(String tablePath, List rows) throws IOException { + List records = Lists.newArrayList(); + for (Row row : rows) { + records.add(createRecord((Integer) row.getField(0), (String) row.getField(1))); + } + assertTableRecords(tablePath, records); } static void assertTableRecords(String tablePath, List expected) throws IOException { Preconditions.checkArgument(expected != null, "expected records shouldn't be null"); Table newTable = new HadoopTables().load(tablePath); - List results; + Set resultSet; try (CloseableIterable iterable = (CloseableIterable) IcebergGenerics.read(newTable).build()) { - results = Lists.newArrayList(iterable); + resultSet = Sets.newHashSet(iterable); } - expected.sort(COMPARATOR); - results.sort(COMPARATOR); - Assert.assertEquals("Should produce the expected record", expected, results); + Assert.assertEquals("Should produce the expected record", resultSet, Sets.newHashSet(expected)); } } diff --git a/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java b/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java index 50ac6a36df87..985793df672b 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java +++ b/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java @@ -35,6 +35,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.data.RandomData; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.taskio.OutputFileFactory; @@ -104,6 +105,25 @@ public void testWriteZeroRecord() throws IOException { } } + @Test + public void testCloseTwice() throws IOException { + try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { + taskWriter.write(Row.of(1, "hello")); + taskWriter.write(Row.of(2, "world")); + taskWriter.close(); // The first close + taskWriter.close(); // The second close + + int expectedFiles = partitioned ? 2 : 1; + List dataFiles = taskWriter.pollCompleteFiles(); + Assert.assertEquals(expectedFiles, dataFiles.size()); + + FileSystem fs = FileSystem.get(CONF); + for (DataFile dataFile : dataFiles) { + Assert.assertTrue(fs.exists(new Path(dataFile.path().toString()))); + } + } + } + @Test public void testAbort() throws IOException { try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { @@ -216,6 +236,25 @@ public void testRollingWithTargetFileSize() throws IOException { } } + @Test + public void testRandomData() throws IOException { + try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { + Iterable rows = RandomData.generate(SimpleDataUtil.SCHEMA, 100, 1996); + for (Row row : rows) { + taskWriter.write(row); + } + + taskWriter.close(); + List dataFiles = taskWriter.pollCompleteFiles(); + AppendFiles appendFiles = table.newAppend(); + dataFiles.forEach(appendFiles::appendFile); + appendFiles.commit(); + + // Assert the data rows. + SimpleDataUtil.assertTableRows(path, Lists.newArrayList(rows)); + } + } + private TaskWriter createTaskWriter(long targetFileSize) { TaskWriterFactory.FlinkFileAppenderFactory appenderFactory = new TaskWriterFactory.FlinkFileAppenderFactory(table.schema(), table.properties()); From 9f1fa70d55eb563199395ebc0274a290d5713c39 Mon Sep 17 00:00:00 2001 From: openinx Date: Thu, 23 Jul 2020 10:52:40 +0800 Subject: [PATCH 10/16] Addressing the comment. --- .../{taskio => tasks}/BaseTaskWriter.java | 10 ++-- .../FileAppenderFactory.java | 4 +- .../{taskio => tasks}/OutputFileFactory.java | 2 +- .../PartitionedFanoutWriter.java | 2 +- .../{taskio => tasks}/PartitionedWriter.java | 2 +- .../iceberg/{taskio => tasks}/TaskWriter.java | 9 ++-- .../UnpartitionedWriter.java | 2 +- .../iceberg/flink/TaskWriterFactory.java | 10 ++-- .../apache/iceberg/flink/TestTaskWriters.java | 53 +++++++++---------- .../iceberg/spark/source/RowDataRewriter.java | 10 ++-- .../spark/source/SparkAppenderFactory.java | 2 +- .../apache/iceberg/spark/source/Writer.java | 10 ++-- .../iceberg/spark/source/SparkBatchWrite.java | 10 ++-- 13 files changed, 62 insertions(+), 64 deletions(-) rename core/src/main/java/org/apache/iceberg/{taskio => tasks}/BaseTaskWriter.java (95%) rename core/src/main/java/org/apache/iceberg/{taskio => tasks}/FileAppenderFactory.java (92%) rename core/src/main/java/org/apache/iceberg/{taskio => tasks}/OutputFileFactory.java (98%) rename core/src/main/java/org/apache/iceberg/{taskio => tasks}/PartitionedFanoutWriter.java (98%) rename core/src/main/java/org/apache/iceberg/{taskio => tasks}/PartitionedWriter.java (98%) rename core/src/main/java/org/apache/iceberg/{taskio => tasks}/TaskWriter.java (84%) rename core/src/main/java/org/apache/iceberg/{taskio => tasks}/UnpartitionedWriter.java (98%) diff --git a/core/src/main/java/org/apache/iceberg/taskio/BaseTaskWriter.java b/core/src/main/java/org/apache/iceberg/tasks/BaseTaskWriter.java similarity index 95% rename from core/src/main/java/org/apache/iceberg/taskio/BaseTaskWriter.java rename to core/src/main/java/org/apache/iceberg/tasks/BaseTaskWriter.java index 70511883178d..bbe5c8b61754 100644 --- a/core/src/main/java/org/apache/iceberg/taskio/BaseTaskWriter.java +++ b/core/src/main/java/org/apache/iceberg/tasks/BaseTaskWriter.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iceberg.taskio; +package org.apache.iceberg.tasks; import java.io.IOException; import java.util.Collections; @@ -69,11 +69,11 @@ public void abort() throws IOException { } @Override - public List pollCompleteFiles() { + public List complete() throws IOException { + close(); + if (completedFiles.size() > 0) { - List dataFiles = ImmutableList.copyOf(completedFiles); - completedFiles.clear(); - return dataFiles; + return ImmutableList.copyOf(completedFiles); } else { return Collections.emptyList(); } diff --git a/core/src/main/java/org/apache/iceberg/taskio/FileAppenderFactory.java b/core/src/main/java/org/apache/iceberg/tasks/FileAppenderFactory.java similarity index 92% rename from core/src/main/java/org/apache/iceberg/taskio/FileAppenderFactory.java rename to core/src/main/java/org/apache/iceberg/tasks/FileAppenderFactory.java index c8922e63ebd3..e109422abd02 100644 --- a/core/src/main/java/org/apache/iceberg/taskio/FileAppenderFactory.java +++ b/core/src/main/java/org/apache/iceberg/tasks/FileAppenderFactory.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iceberg.taskio; +package org.apache.iceberg.tasks; import org.apache.iceberg.FileFormat; import org.apache.iceberg.io.FileAppender; @@ -33,7 +33,7 @@ public interface FileAppenderFactory { /** * Create a new {@link FileAppender}. * - * @param outputFile indicate the file location to write. + * @param outputFile an OutputFile used to create an output stream. * @param fileFormat File format. * @return a newly created {@link FileAppender} */ diff --git a/core/src/main/java/org/apache/iceberg/taskio/OutputFileFactory.java b/core/src/main/java/org/apache/iceberg/tasks/OutputFileFactory.java similarity index 98% rename from core/src/main/java/org/apache/iceberg/taskio/OutputFileFactory.java rename to core/src/main/java/org/apache/iceberg/tasks/OutputFileFactory.java index 8b09bbefa4c0..5d5e03935519 100644 --- a/core/src/main/java/org/apache/iceberg/taskio/OutputFileFactory.java +++ b/core/src/main/java/org/apache/iceberg/tasks/OutputFileFactory.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iceberg.taskio; +package org.apache.iceberg.tasks; import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; diff --git a/core/src/main/java/org/apache/iceberg/taskio/PartitionedFanoutWriter.java b/core/src/main/java/org/apache/iceberg/tasks/PartitionedFanoutWriter.java similarity index 98% rename from core/src/main/java/org/apache/iceberg/taskio/PartitionedFanoutWriter.java rename to core/src/main/java/org/apache/iceberg/tasks/PartitionedFanoutWriter.java index 504099f15b2a..446ba7cddf2d 100644 --- a/core/src/main/java/org/apache/iceberg/taskio/PartitionedFanoutWriter.java +++ b/core/src/main/java/org/apache/iceberg/tasks/PartitionedFanoutWriter.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iceberg.taskio; +package org.apache.iceberg.tasks; import java.io.IOException; import java.util.Iterator; diff --git a/core/src/main/java/org/apache/iceberg/taskio/PartitionedWriter.java b/core/src/main/java/org/apache/iceberg/tasks/PartitionedWriter.java similarity index 98% rename from core/src/main/java/org/apache/iceberg/taskio/PartitionedWriter.java rename to core/src/main/java/org/apache/iceberg/tasks/PartitionedWriter.java index 2e566e828b1d..826a4d3b9d8f 100644 --- a/core/src/main/java/org/apache/iceberg/taskio/PartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/tasks/PartitionedWriter.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iceberg.taskio; +package org.apache.iceberg.tasks; import java.io.IOException; import java.util.Set; diff --git a/core/src/main/java/org/apache/iceberg/taskio/TaskWriter.java b/core/src/main/java/org/apache/iceberg/tasks/TaskWriter.java similarity index 84% rename from core/src/main/java/org/apache/iceberg/taskio/TaskWriter.java rename to core/src/main/java/org/apache/iceberg/tasks/TaskWriter.java index dc1c3a75ea31..f757c0a4343e 100644 --- a/core/src/main/java/org/apache/iceberg/taskio/TaskWriter.java +++ b/core/src/main/java/org/apache/iceberg/tasks/TaskWriter.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iceberg.taskio; +package org.apache.iceberg.tasks; import java.io.Closeable; import java.io.IOException; @@ -44,10 +44,9 @@ public interface TaskWriter extends Closeable { void abort() throws IOException; /** - * Get the completed data files and clear them from the cache. NOTICE: if call this method without - * {@link TaskWriter#close()} then the current opening data file won't be seen in the result list. + * Get the completed data files and clear them from the cache. * - * @return the cached completed data files of this task writer. + * @return the completed data files of this task writer. */ - List pollCompleteFiles(); + List complete() throws IOException; } diff --git a/core/src/main/java/org/apache/iceberg/taskio/UnpartitionedWriter.java b/core/src/main/java/org/apache/iceberg/tasks/UnpartitionedWriter.java similarity index 98% rename from core/src/main/java/org/apache/iceberg/taskio/UnpartitionedWriter.java rename to core/src/main/java/org/apache/iceberg/tasks/UnpartitionedWriter.java index 0ad1f86719a7..fe0e4bd08e99 100644 --- a/core/src/main/java/org/apache/iceberg/taskio/UnpartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/tasks/UnpartitionedWriter.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iceberg.taskio; +package org.apache.iceberg.tasks; import java.io.IOException; import org.apache.iceberg.FileFormat; diff --git a/flink/src/main/java/org/apache/iceberg/flink/TaskWriterFactory.java b/flink/src/main/java/org/apache/iceberg/flink/TaskWriterFactory.java index 7f5629ad874c..39423cc5ae0f 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/TaskWriterFactory.java +++ b/flink/src/main/java/org/apache/iceberg/flink/TaskWriterFactory.java @@ -36,11 +36,11 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.parquet.Parquet; -import org.apache.iceberg.taskio.FileAppenderFactory; -import org.apache.iceberg.taskio.OutputFileFactory; -import org.apache.iceberg.taskio.PartitionedFanoutWriter; -import org.apache.iceberg.taskio.TaskWriter; -import org.apache.iceberg.taskio.UnpartitionedWriter; +import org.apache.iceberg.tasks.FileAppenderFactory; +import org.apache.iceberg.tasks.OutputFileFactory; +import org.apache.iceberg.tasks.PartitionedFanoutWriter; +import org.apache.iceberg.tasks.TaskWriter; +import org.apache.iceberg.tasks.UnpartitionedWriter; public class TaskWriterFactory { private TaskWriterFactory() { diff --git a/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java b/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java index 985793df672b..45be78f06264 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java +++ b/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java @@ -38,8 +38,8 @@ import org.apache.iceberg.flink.data.RandomData; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.taskio.OutputFileFactory; -import org.apache.iceberg.taskio.TaskWriter; +import org.apache.iceberg.tasks.OutputFileFactory; +import org.apache.iceberg.tasks.TaskWriter; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -93,13 +93,13 @@ public void testWriteZeroRecord() throws IOException { try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.close(); - List dataFiles = taskWriter.pollCompleteFiles(); + List dataFiles = taskWriter.complete(); Assert.assertNotNull(dataFiles); Assert.assertEquals(0, dataFiles.size()); // Close again. taskWriter.close(); - dataFiles = taskWriter.pollCompleteFiles(); + dataFiles = taskWriter.complete(); Assert.assertNotNull(dataFiles); Assert.assertEquals(0, dataFiles.size()); } @@ -114,7 +114,7 @@ public void testCloseTwice() throws IOException { taskWriter.close(); // The second close int expectedFiles = partitioned ? 2 : 1; - List dataFiles = taskWriter.pollCompleteFiles(); + List dataFiles = taskWriter.complete(); Assert.assertEquals(expectedFiles, dataFiles.size()); FileSystem fs = FileSystem.get(CONF); @@ -131,7 +131,7 @@ public void testAbort() throws IOException { taskWriter.write(Row.of(2, "world")); taskWriter.abort(); - List dataFiles = taskWriter.pollCompleteFiles(); + List dataFiles = taskWriter.complete(); int expectedFiles = partitioned ? 2 : 1; Assert.assertEquals(expectedFiles, dataFiles.size()); @@ -144,25 +144,30 @@ public void testAbort() throws IOException { } @Test - public void testPollCompleteFiles() throws IOException { + public void testCompleteFiles() throws IOException { try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.write(Row.of(1, "a")); taskWriter.write(Row.of(2, "b")); taskWriter.write(Row.of(3, "c")); taskWriter.write(Row.of(4, "d")); - List dataFiles = taskWriter.pollCompleteFiles(); - Assert.assertEquals(0, dataFiles.size()); - - taskWriter.close(); - dataFiles = taskWriter.pollCompleteFiles(); + List dataFiles = taskWriter.complete(); int expectedFiles = partitioned ? 4 : 1; Assert.assertEquals(expectedFiles, dataFiles.size()); + dataFiles = taskWriter.complete(); + Assert.assertEquals(expectedFiles, dataFiles.size()); + + taskWriter.write(Row.of(5, "e")); + dataFiles = taskWriter.complete(); + expectedFiles = partitioned ? 5 : 2; + Assert.assertEquals(expectedFiles, dataFiles.size()); + FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { Assert.assertTrue(fs.exists(new Path(dataFile.path().toString()))); } + AppendFiles appendFiles = table.newAppend(); dataFiles.forEach(appendFiles::appendFile); appendFiles.commit(); @@ -172,16 +177,14 @@ public void testPollCompleteFiles() throws IOException { SimpleDataUtil.createRecord(1, "a"), SimpleDataUtil.createRecord(2, "b"), SimpleDataUtil.createRecord(3, "c"), - SimpleDataUtil.createRecord(4, "d") + SimpleDataUtil.createRecord(4, "d"), + SimpleDataUtil.createRecord(5, "e") )); - - dataFiles = taskWriter.pollCompleteFiles(); - Assert.assertEquals(0, dataFiles.size()); } } @Test - public void testCopiedCompleteFiles() throws IOException { + public void testImmutableCompleteFiles() throws IOException { try (TaskWriter taskWriter = createTaskWriter(4)) { List rows = Lists.newArrayListWithCapacity(1000); for (int i = 0; i < 1000; i++) { @@ -192,19 +195,15 @@ public void testCopiedCompleteFiles() throws IOException { taskWriter.write(row); } - List dataFiles = taskWriter.pollCompleteFiles(); + List dataFiles = taskWriter.complete(); Assert.assertEquals(1, dataFiles.size()); AssertHelpers.assertThrows("Complete file list are immutable", UnsupportedOperationException.class, () -> dataFiles.add(null)); - List emptyList = taskWriter.pollCompleteFiles(); - Assert.assertEquals(0, emptyList.size()); + List dataFilesAgain = taskWriter.complete(); + Assert.assertEquals(1, dataFilesAgain.size()); AssertHelpers.assertThrows("Empty complete file list are immutable", UnsupportedOperationException.class, - () -> emptyList.add(null)); - - // It should not open any new file when closed a writer without writing new record. - taskWriter.close(); - Assert.assertEquals(0, taskWriter.pollCompleteFiles().size()); + () -> dataFilesAgain.add(null)); } } @@ -224,7 +223,7 @@ public void testRollingWithTargetFileSize() throws IOException { taskWriter.write(row); } - List dataFiles = taskWriter.pollCompleteFiles(); + List dataFiles = taskWriter.complete(); Assert.assertEquals(8, dataFiles.size()); AppendFiles appendFiles = table.newAppend(); @@ -245,7 +244,7 @@ public void testRandomData() throws IOException { } taskWriter.close(); - List dataFiles = taskWriter.pollCompleteFiles(); + List dataFiles = taskWriter.complete(); AppendFiles appendFiles = table.newAppend(); dataFiles.forEach(appendFiles::appendFile); appendFiles.commit(); 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 a9fd6a1cc6d4..4c8fd2a4b466 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 @@ -36,10 +36,10 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.taskio.OutputFileFactory; -import org.apache.iceberg.taskio.PartitionedWriter; -import org.apache.iceberg.taskio.TaskWriter; -import org.apache.iceberg.taskio.UnpartitionedWriter; +import org.apache.iceberg.tasks.OutputFileFactory; +import org.apache.iceberg.tasks.PartitionedWriter; +import org.apache.iceberg.tasks.TaskWriter; +import org.apache.iceberg.tasks.UnpartitionedWriter; import org.apache.spark.TaskContext; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.broadcast.Broadcast; @@ -119,7 +119,7 @@ private TaskResult rewriteDataForTask(CombinedScanTask task) throws Exception { dataReader = null; writer.close(); - return new TaskResult(writer.pollCompleteFiles()); + return new TaskResult(writer.complete()); } catch (Throwable originalThrowable) { try { diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java b/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java index ac0ffe9e500e..cb013db301d4 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java @@ -33,7 +33,7 @@ import org.apache.iceberg.spark.data.SparkAvroWriter; import org.apache.iceberg.spark.data.SparkOrcWriter; import org.apache.iceberg.spark.data.SparkParquetWriters; -import org.apache.iceberg.taskio.FileAppenderFactory; +import org.apache.iceberg.tasks.FileAppenderFactory; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.types.StructType; 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 acd6d317efa0..d803a2a13feb 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 @@ -43,9 +43,9 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.taskio.OutputFileFactory; -import org.apache.iceberg.taskio.PartitionedWriter; -import org.apache.iceberg.taskio.UnpartitionedWriter; +import org.apache.iceberg.tasks.OutputFileFactory; +import org.apache.iceberg.tasks.PartitionedWriter; +import org.apache.iceberg.tasks.UnpartitionedWriter; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.Tasks; import org.apache.spark.broadcast.Broadcast; @@ -286,7 +286,7 @@ private static class Unpartitioned24Writer extends UnpartitionedWriter dataFiles = pollCompleteFiles(); + List dataFiles = complete(); return new TaskCommit(new TaskResult(dataFiles)); } } @@ -302,7 +302,7 @@ private static class Partitioned24Writer extends PartitionedWriter public WriterCommitMessage commit() throws IOException { this.close(); - List dataFiles = pollCompleteFiles(); + List dataFiles = complete(); return new TaskCommit(new TaskResult(dataFiles)); } } diff --git a/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchWrite.java b/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchWrite.java index df39163f3a87..0ef93514dfb0 100644 --- a/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchWrite.java +++ b/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchWrite.java @@ -45,9 +45,9 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.taskio.OutputFileFactory; -import org.apache.iceberg.taskio.PartitionedWriter; -import org.apache.iceberg.taskio.UnpartitionedWriter; +import org.apache.iceberg.tasks.OutputFileFactory; +import org.apache.iceberg.tasks.PartitionedWriter; +import org.apache.iceberg.tasks.UnpartitionedWriter; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.Tasks; import org.apache.spark.broadcast.Broadcast; @@ -308,7 +308,7 @@ private static class Unpartitioned3Writer extends UnpartitionedWriter dataFiles = pollCompleteFiles(); + List dataFiles = complete(); return new TaskCommit(new TaskResult(dataFiles)); } } @@ -325,7 +325,7 @@ private static class Partitioned3Writer extends PartitionedWriter public WriterCommitMessage commit() throws IOException { this.close(); - List dataFiles = pollCompleteFiles(); + List dataFiles = complete(); return new TaskCommit(new TaskResult(dataFiles)); } } From 499a48f30667b9f3b00fea9de01421bbba78a4a0 Mon Sep 17 00:00:00 2001 From: openinx Date: Thu, 23 Jul 2020 12:37:57 +0800 Subject: [PATCH 11/16] Remove the keyGetter and use the abstract partitioned(..) method. --- .../iceberg/{tasks => io}/BaseTaskWriter.java | 109 ++++++++++-------- .../{tasks => io}/FileAppenderFactory.java | 4 +- .../{tasks => io}/OutputFileFactory.java | 5 +- .../PartitionedFanoutWriter.java | 37 +++--- .../{tasks => io}/PartitionedWriter.java | 31 ++--- .../iceberg/{tasks => io}/TaskWriter.java | 4 +- .../{tasks => io}/UnpartitionedWriter.java | 12 +- .../iceberg/flink/TaskWriterFactory.java | 45 +++++--- .../apache/iceberg/flink/TestTaskWriters.java | 4 +- .../iceberg/spark/source/RowDataRewriter.java | 16 +-- .../spark/source/SparkAppenderFactory.java | 2 +- .../spark/source/SparkPartitionedWriter.java | 51 ++++++++ .../iceberg/spark/source/WriterUtil.java | 42 ------- .../apache/iceberg/spark/source/Writer.java | 13 ++- .../iceberg/spark/source/SparkBatchWrite.java | 14 +-- 15 files changed, 203 insertions(+), 186 deletions(-) rename core/src/main/java/org/apache/iceberg/{tasks => io}/BaseTaskWriter.java (56%) rename core/src/main/java/org/apache/iceberg/{tasks => io}/FileAppenderFactory.java (91%) rename core/src/main/java/org/apache/iceberg/{tasks => io}/OutputFileFactory.java (94%) rename core/src/main/java/org/apache/iceberg/{tasks => io}/PartitionedFanoutWriter.java (68%) rename core/src/main/java/org/apache/iceberg/{tasks => io}/PartitionedWriter.java (77%) rename core/src/main/java/org/apache/iceberg/{tasks => io}/TaskWriter.java (93%) rename core/src/main/java/org/apache/iceberg/{tasks => io}/UnpartitionedWriter.java (83%) create mode 100644 spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedWriter.java delete mode 100644 spark/src/main/java/org/apache/iceberg/spark/source/WriterUtil.java diff --git a/core/src/main/java/org/apache/iceberg/tasks/BaseTaskWriter.java b/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java similarity index 56% rename from core/src/main/java/org/apache/iceberg/tasks/BaseTaskWriter.java rename to core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java index bbe5c8b61754..ba48220e6a3b 100644 --- a/core/src/main/java/org/apache/iceberg/tasks/BaseTaskWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java @@ -17,12 +17,12 @@ * under the License. */ -package org.apache.iceberg.tasks; +package org.apache.iceberg.io; +import java.io.Closeable; import java.io.IOException; import java.util.Collections; import java.util.List; -import java.util.function.Supplier; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.FileFormat; @@ -30,15 +30,11 @@ 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.FileIO; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.Tasks; abstract class BaseTaskWriter implements TaskWriter { - protected static final int ROWS_DIVISOR = 1000; - private final List completedFiles = Lists.newArrayList(); private final PartitionSpec spec; private final FileFormat format; @@ -79,63 +75,80 @@ public List complete() throws IOException { } } - protected OutputFileFactory outputFileFactory() { - return this.fileFactory; - } - - WrappedFileAppender createWrappedFileAppender(PartitionKey partitionKey, - Supplier outputFileSupplier) { - EncryptedOutputFile outputFile = outputFileSupplier.get(); - FileAppender appender = appenderFactory.newAppender(outputFile.encryptingOutputFile(), format); - return new WrappedFileAppender(partitionKey, outputFile, appender); - } - - class WrappedFileAppender { + class RollingFileAppender implements Closeable { + private static final int ROWS_DIVISOR = 1000; private final PartitionKey partitionKey; - private final EncryptedOutputFile encryptedOutputFile; - private final FileAppender appender; - private boolean closed = false; + private EncryptedOutputFile currentFile = null; + private FileAppender currentAppender = null; private long currentRows = 0; - WrappedFileAppender(PartitionKey partitionKey, EncryptedOutputFile encryptedOutputFile, FileAppender appender) { + RollingFileAppender(PartitionKey partitionKey) { this.partitionKey = partitionKey; - this.encryptedOutputFile = encryptedOutputFile; - this.appender = appender; } - void add(T record) { - this.appender.add(record); + void add(T record) throws IOException { + if (currentAppender == null) { + openCurrent(); + } + + this.currentAppender.add(record); this.currentRows++; + + if (shouldRollToNewFile()) { + closeCurrent(); + } } - boolean shouldRollToNewFile() { + private void openCurrent() { + if (spec.fields().size() == 0) { + // unpartitioned + currentFile = fileFactory.newOutputFile(); + } else { + // partitioned + currentFile = fileFactory.newOutputFile(partitionKey); + } + currentAppender = appenderFactory.newAppender(currentFile.encryptingOutputFile(), format); + currentRows = 0; + } + + private boolean shouldRollToNewFile() { //TODO: ORC file now not support target file size before closed return !format.equals(FileFormat.ORC) && - currentRows % ROWS_DIVISOR == 0 && appender.length() >= targetFileSize; + currentRows % ROWS_DIVISOR == 0 && currentAppender.length() >= targetFileSize; } - void close() throws IOException { - // Close the file appender firstly. - if (!closed) { - appender.close(); - closed = true; - - // metrics are only valid after the appender is closed. - Metrics metrics = appender.metrics(); - long fileSizeInBytes = appender.length(); - List splitOffsets = appender.splitOffsets(); - - DataFile dataFile = DataFiles.builder(spec) - .withEncryptedOutputFile(encryptedOutputFile) - .withFileSizeInBytes(fileSizeInBytes) - .withPartition(partitionKey) - .withMetrics(metrics) - .withSplitOffsets(splitOffsets) - .build(); - - completedFiles.add(dataFile); + private void closeCurrent() throws IOException { + if (currentAppender != null) { + currentAppender.close(); + // metrics are only valid after the appender is closed + Metrics metrics = currentAppender.metrics(); + long fileSizeInBytes = currentAppender.length(); + List splitOffsets = currentAppender.splitOffsets(); + this.currentAppender = null; + + if (metrics.recordCount() == 0L) { + io.deleteFile(currentFile.encryptingOutputFile()); + } else { + DataFile dataFile = DataFiles.builder(spec) + .withEncryptionKeyMetadata(currentFile.keyMetadata()) + .withPath(currentFile.encryptingOutputFile().location()) + .withFileSizeInBytes(fileSizeInBytes) + .withPartition(spec.fields().size() == 0 ? null : partitionKey) // set null if unpartitioned + .withMetrics(metrics) + .withSplitOffsets(splitOffsets) + .build(); + completedFiles.add(dataFile); + } + + this.currentFile = null; + this.currentRows = 0; } } + + @Override + public void close() throws IOException { + closeCurrent(); + } } } diff --git a/core/src/main/java/org/apache/iceberg/tasks/FileAppenderFactory.java b/core/src/main/java/org/apache/iceberg/io/FileAppenderFactory.java similarity index 91% rename from core/src/main/java/org/apache/iceberg/tasks/FileAppenderFactory.java rename to core/src/main/java/org/apache/iceberg/io/FileAppenderFactory.java index e109422abd02..9afdca460f0a 100644 --- a/core/src/main/java/org/apache/iceberg/tasks/FileAppenderFactory.java +++ b/core/src/main/java/org/apache/iceberg/io/FileAppenderFactory.java @@ -17,11 +17,9 @@ * under the License. */ -package org.apache.iceberg.tasks; +package org.apache.iceberg.io; import org.apache.iceberg.FileFormat; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.io.OutputFile; /** * Factory to create a new {@link FileAppender} to write records. diff --git a/core/src/main/java/org/apache/iceberg/tasks/OutputFileFactory.java b/core/src/main/java/org/apache/iceberg/io/OutputFileFactory.java similarity index 94% rename from core/src/main/java/org/apache/iceberg/tasks/OutputFileFactory.java rename to core/src/main/java/org/apache/iceberg/io/OutputFileFactory.java index 5d5e03935519..e68f4c5ad2ec 100644 --- a/core/src/main/java/org/apache/iceberg/tasks/OutputFileFactory.java +++ b/core/src/main/java/org/apache/iceberg/io/OutputFileFactory.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iceberg.tasks; +package org.apache.iceberg.io; import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; @@ -26,9 +26,6 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.LocationProvider; -import org.apache.iceberg.io.OutputFile; public class OutputFileFactory { private final PartitionSpec spec; diff --git a/core/src/main/java/org/apache/iceberg/tasks/PartitionedFanoutWriter.java b/core/src/main/java/org/apache/iceberg/io/PartitionedFanoutWriter.java similarity index 68% rename from core/src/main/java/org/apache/iceberg/tasks/PartitionedFanoutWriter.java rename to core/src/main/java/org/apache/iceberg/io/PartitionedFanoutWriter.java index 446ba7cddf2d..99fd4adbf705 100644 --- a/core/src/main/java/org/apache/iceberg/tasks/PartitionedFanoutWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/PartitionedFanoutWriter.java @@ -17,53 +17,52 @@ * under the License. */ -package org.apache.iceberg.tasks; +package org.apache.iceberg.io; import java.io.IOException; import java.util.Iterator; import java.util.Map; -import java.util.function.Function; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -public class PartitionedFanoutWriter extends BaseTaskWriter { - private final Function keyGetter; - private final Map writers = Maps.newHashMap(); +public abstract class PartitionedFanoutWriter extends BaseTaskWriter { + private final Map writers = Maps.newHashMap(); public PartitionedFanoutWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, FileIO io, long targetFileSize, - Function keyGetter) { + OutputFileFactory fileFactory, FileIO io, long targetFileSize) { super(spec, format, appenderFactory, fileFactory, io, targetFileSize); - this.keyGetter = keyGetter; } + /** + * 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 = keyGetter.apply(row); + PartitionKey partitionKey = partition(row); - WrappedFileAppender writer = writers.get(partitionKey); + RollingFileAppender 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 = createWrappedFileAppender(copiedKey, () -> outputFileFactory().newOutputFile(partitionKey)); + writer = new RollingFileAppender(copiedKey); writers.put(copiedKey, writer); } - writer.add(row); - // Close the writer if reach the target file size. - if (writer.shouldRollToNewFile()) { - writer.close(); - writers.remove(partitionKey); - } + writer.add(row); } @Override public void close() throws IOException { if (!writers.isEmpty()) { - Iterator iterator = writers.values().iterator(); + Iterator iterator = writers.values().iterator(); while (iterator.hasNext()) { iterator.next().close(); // Remove from the writers after closed. diff --git a/core/src/main/java/org/apache/iceberg/tasks/PartitionedWriter.java b/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java similarity index 77% rename from core/src/main/java/org/apache/iceberg/tasks/PartitionedWriter.java rename to core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java index 826a4d3b9d8f..30afdafb33e0 100644 --- a/core/src/main/java/org/apache/iceberg/tasks/PartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java @@ -17,39 +17,43 @@ * under the License. */ -package org.apache.iceberg.tasks; +package org.apache.iceberg.io; import java.io.IOException; import java.util.Set; -import java.util.function.Function; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.io.FileIO; 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 class PartitionedWriter extends BaseTaskWriter { +public abstract class PartitionedWriter extends BaseTaskWriter { private static final Logger LOG = LoggerFactory.getLogger(PartitionedWriter.class); - private final Function keyGetter; private final Set completedPartitions = Sets.newHashSet(); private PartitionKey currentKey = null; - private WrappedFileAppender currentAppender = null; + private RollingFileAppender currentAppender = null; public PartitionedWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, FileIO io, long targetFileSize, - Function keyGetter) { + OutputFileFactory fileFactory, FileIO io, long targetFileSize) { super(spec, format, appenderFactory, fileFactory, io, targetFileSize); - this.keyGetter = keyGetter; } + /** + * 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 key = keyGetter.apply(row); + PartitionKey key = partition(row); if (!key.equals(currentKey)) { closeCurrent(); @@ -66,13 +70,10 @@ public void write(T row) throws IOException { } if (currentAppender == null) { - currentAppender = createWrappedFileAppender(currentKey, () -> outputFileFactory().newOutputFile(currentKey)); + currentAppender = new RollingFileAppender(currentKey); } currentAppender.add(row); - if (currentAppender.shouldRollToNewFile()) { - closeCurrent(); - } } @Override @@ -83,7 +84,7 @@ public void close() throws IOException { private void closeCurrent() throws IOException { if (currentAppender != null) { - // Close the current file appender and put the generated DataFile to completeDataFiles. + // Close the current file appender. currentAppender.close(); // Reset the current appender to be null. diff --git a/core/src/main/java/org/apache/iceberg/tasks/TaskWriter.java b/core/src/main/java/org/apache/iceberg/io/TaskWriter.java similarity index 93% rename from core/src/main/java/org/apache/iceberg/tasks/TaskWriter.java rename to core/src/main/java/org/apache/iceberg/io/TaskWriter.java index f757c0a4343e..438ea4d5c65c 100644 --- a/core/src/main/java/org/apache/iceberg/tasks/TaskWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/TaskWriter.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.iceberg.tasks; +package org.apache.iceberg.io; import java.io.Closeable; import java.io.IOException; @@ -44,7 +44,7 @@ public interface TaskWriter extends Closeable { void abort() throws IOException; /** - * Get the completed data files and clear them from the cache. + * 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/tasks/UnpartitionedWriter.java b/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java similarity index 83% rename from core/src/main/java/org/apache/iceberg/tasks/UnpartitionedWriter.java rename to core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java index fe0e4bd08e99..5f160f7ae82a 100644 --- a/core/src/main/java/org/apache/iceberg/tasks/UnpartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java @@ -17,16 +17,15 @@ * under the License. */ -package org.apache.iceberg.tasks; +package org.apache.iceberg.io; import java.io.IOException; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.io.FileIO; public class UnpartitionedWriter extends BaseTaskWriter { - private WrappedFileAppender currentAppender = null; + private RollingFileAppender currentAppender = null; public UnpartitionedWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, OutputFileFactory fileFactory, FileIO io, long targetFileSize) { @@ -36,14 +35,9 @@ public UnpartitionedWriter(PartitionSpec spec, FileFormat format, FileAppenderFa @Override public void write(T record) throws IOException { if (currentAppender == null) { - currentAppender = createWrappedFileAppender(null, outputFileFactory()::newOutputFile); + currentAppender = new RollingFileAppender(null); } currentAppender.add(record); - - // Close the writer if reach the target file size. - if (currentAppender.shouldRollToNewFile()) { - closeCurrent(); - } } @Override diff --git a/flink/src/main/java/org/apache/iceberg/flink/TaskWriterFactory.java b/flink/src/main/java/org/apache/iceberg/flink/TaskWriterFactory.java index 39423cc5ae0f..71e5d4fd2921 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/TaskWriterFactory.java +++ b/flink/src/main/java/org/apache/iceberg/flink/TaskWriterFactory.java @@ -22,7 +22,6 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.util.Map; -import java.util.function.Function; import org.apache.flink.types.Row; import org.apache.iceberg.FileFormat; import org.apache.iceberg.MetricsConfig; @@ -33,29 +32,19 @@ import org.apache.iceberg.flink.data.FlinkAvroWriter; 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.OutputFile; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.io.PartitionedFanoutWriter; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.UnpartitionedWriter; import org.apache.iceberg.parquet.Parquet; -import org.apache.iceberg.tasks.FileAppenderFactory; -import org.apache.iceberg.tasks.OutputFileFactory; -import org.apache.iceberg.tasks.PartitionedFanoutWriter; -import org.apache.iceberg.tasks.TaskWriter; -import org.apache.iceberg.tasks.UnpartitionedWriter; public class TaskWriterFactory { private TaskWriterFactory() { } - private static Function buildKeyGetter(PartitionSpec spec, Schema schema) { - PartitionKey partitionKey = new PartitionKey(spec, schema); - RowWrapper rowWrapper = new RowWrapper(schema.asStruct()); - - return row -> { - partitionKey.partition(rowWrapper.wrap(row)); - return partitionKey; - }; - } - public static TaskWriter createTaskWriter(Schema schema, PartitionSpec spec, FileFormat format, @@ -66,9 +55,27 @@ public static TaskWriter createTaskWriter(Schema schema, if (spec.fields().isEmpty()) { return new UnpartitionedWriter<>(spec, format, appenderFactory, fileFactory, io, targetFileSizeBytes); } else { - Function keyGetter = buildKeyGetter(spec, schema); - return new PartitionedFanoutWriter<>(spec, format, appenderFactory, fileFactory, io, - targetFileSizeBytes, keyGetter); + return new RowPartitionedFanoutWriter(spec, format, appenderFactory, fileFactory, + io, targetFileSizeBytes, schema); + } + } + + static class RowPartitionedFanoutWriter extends PartitionedFanoutWriter { + + private final PartitionKey partitionKey; + private final RowWrapper rowWrapper; + + RowPartitionedFanoutWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema schema) { + super(spec, format, appenderFactory, fileFactory, io, targetFileSize); + this.partitionKey = new PartitionKey(spec, schema); + this.rowWrapper = new RowWrapper(schema.asStruct()); + } + + @Override + protected PartitionKey partition(Row row) { + partitionKey.partition(rowWrapper.wrap(row)); + return partitionKey; } } diff --git a/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java b/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java index 45be78f06264..bca917447f24 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java +++ b/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java @@ -36,10 +36,10 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.data.RandomData; +import org.apache.iceberg.io.OutputFileFactory; +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.apache.iceberg.tasks.OutputFileFactory; -import org.apache.iceberg.tasks.TaskWriter; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; 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 4c8fd2a4b466..2a8a895887ee 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 @@ -35,15 +35,15 @@ import org.apache.iceberg.encryption.EncryptionManager; 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.UnpartitionedWriter; import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.tasks.OutputFileFactory; -import org.apache.iceberg.tasks.PartitionedWriter; -import org.apache.iceberg.tasks.TaskWriter; -import org.apache.iceberg.tasks.UnpartitionedWriter; import org.apache.spark.TaskContext; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.broadcast.Broadcast; import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.types.StructType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -96,8 +96,8 @@ private TaskResult rewriteDataForTask(CombinedScanTask task) throws Exception { RowDataReader dataReader = new RowDataReader( task, schema, schema, nameMapping, io.value(), encryptionManager.value(), caseSensitive); - SparkAppenderFactory appenderFactory = new SparkAppenderFactory( - properties, schema, SparkSchemaUtil.convert(schema)); + StructType structType = SparkSchemaUtil.convert(schema); + SparkAppenderFactory appenderFactory = new SparkAppenderFactory(properties, schema, structType); OutputFileFactory fileFactory = new OutputFileFactory( spec, format, locations, io.value(), encryptionManager.value(), partitionId, taskId); @@ -105,8 +105,8 @@ private TaskResult rewriteDataForTask(CombinedScanTask task) throws Exception { if (spec.fields().isEmpty()) { writer = new UnpartitionedWriter<>(spec, format, appenderFactory, fileFactory, io.value(), Long.MAX_VALUE); } else { - writer = new PartitionedWriter<>(spec, format, appenderFactory, fileFactory, io.value(), Long.MAX_VALUE, - WriterUtil.buildKeyGetter(spec, schema)); + writer = new SparkPartitionedWriter(spec, format, appenderFactory, fileFactory, io.value(), Long.MAX_VALUE, + schema, structType); } try { diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java b/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java index cb013db301d4..bc9bbd946e03 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/SparkAppenderFactory.java @@ -27,13 +27,13 @@ import org.apache.iceberg.avro.Avro; import org.apache.iceberg.exceptions.RuntimeIOException; 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; import org.apache.iceberg.spark.data.SparkAvroWriter; import org.apache.iceberg.spark.data.SparkOrcWriter; import org.apache.iceberg.spark.data.SparkParquetWriters; -import org.apache.iceberg.tasks.FileAppenderFactory; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.types.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 new file mode 100644 index 000000000000..f81a09926d85 --- /dev/null +++ b/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedWriter.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.spark.source; + +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.FileAppenderFactory; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.io.PartitionedWriter; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.types.StructType; + +public class SparkPartitionedWriter extends PartitionedWriter { + private final PartitionKey partitionKey; + private final InternalRowWrapper internalRowWrapper; + + 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); + this.partitionKey = new PartitionKey(spec, schema); + this.internalRowWrapper = new InternalRowWrapper(sparkSchema); + } + + @Override + protected PartitionKey partition(InternalRow row) { + partitionKey.partition(internalRowWrapper.wrap(row)); + return partitionKey; + } +} diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/WriterUtil.java b/spark/src/main/java/org/apache/iceberg/spark/source/WriterUtil.java deleted file mode 100644 index 016515e07e3f..000000000000 --- a/spark/src/main/java/org/apache/iceberg/spark/source/WriterUtil.java +++ /dev/null @@ -1,42 +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.spark.source; - -import java.util.function.Function; -import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.spark.sql.catalyst.InternalRow; - -class WriterUtil { - private WriterUtil() { - } - - static Function buildKeyGetter(PartitionSpec spec, Schema schema) { - PartitionKey key = new PartitionKey(spec, schema); - InternalRowWrapper wrapper = new InternalRowWrapper(SparkSchemaUtil.convert(schema)); - - return row -> { - key.partition(wrapper.wrap(row)); - return key; - }; - } -} 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 d803a2a13feb..ee1860c3d125 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 @@ -25,11 +25,9 @@ import java.util.Locale; import java.util.Map; import java.util.Optional; -import java.util.function.Function; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.ReplacePartitions; import org.apache.iceberg.Schema; @@ -40,6 +38,8 @@ import org.apache.iceberg.encryption.EncryptionManager; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.io.UnpartitionedWriter; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -270,7 +270,7 @@ public DataWriter createDataWriter(int partitionId, long taskId, lo return new Unpartitioned24Writer(spec, format, appenderFactory, fileFactory, io.value(), targetFileSize); } else { return new Partitioned24Writer(spec, format, appenderFactory, fileFactory, io.value(), - targetFileSize, WriterUtil.buildKeyGetter(spec, writeSchema)); + targetFileSize, writeSchema, dsSchema); } } } @@ -291,11 +291,12 @@ public WriterCommitMessage commit() throws IOException { } } - private static class Partitioned24Writer extends PartitionedWriter implements DataWriter { + private static class Partitioned24Writer extends SparkPartitionedWriter implements DataWriter { + Partitioned24Writer(PartitionSpec spec, FileFormat format, SparkAppenderFactory appenderFactory, OutputFileFactory fileFactory, FileIO fileIo, long targetFileSize, - Function keyGetter) { - super(spec, format, appenderFactory, fileFactory, fileIo, targetFileSize, keyGetter); + Schema schema, StructType sparkSchema) { + super(spec, format, appenderFactory, fileFactory, fileIo, targetFileSize, schema, sparkSchema); } @Override diff --git a/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchWrite.java b/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchWrite.java index 0ef93514dfb0..5bdce9f82ac4 100644 --- a/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchWrite.java +++ b/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchWrite.java @@ -25,12 +25,10 @@ import java.util.Locale; import java.util.Map; import java.util.Optional; -import java.util.function.Function; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.OverwriteFiles; -import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.ReplacePartitions; import org.apache.iceberg.Schema; @@ -42,6 +40,8 @@ import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.io.UnpartitionedWriter; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -291,8 +291,7 @@ public DataWriter createWriter(int partitionId, long taskId, long e return new Unpartitioned3Writer(spec, format, appenderFactory, fileFactory, io.value(), targetFileSize); } else { return new Partitioned3Writer( - spec, format, appenderFactory, fileFactory, io.value(), targetFileSize, - WriterUtil.buildKeyGetter(spec, writeSchema)); + spec, format, appenderFactory, fileFactory, io.value(), targetFileSize, writeSchema, dsSchema); } } } @@ -313,12 +312,11 @@ public WriterCommitMessage commit() throws IOException { } } - private static class Partitioned3Writer extends PartitionedWriter - implements DataWriter { + private static class Partitioned3Writer extends SparkPartitionedWriter implements DataWriter { Partitioned3Writer(PartitionSpec spec, FileFormat format, SparkAppenderFactory appenderFactory, OutputFileFactory fileFactory, FileIO io, long targetFileSize, - Function keyGetter) { - super(spec, format, appenderFactory, fileFactory, io, targetFileSize, keyGetter); + Schema schema, StructType sparkSchema) { + super(spec, format, appenderFactory, fileFactory, io, targetFileSize, schema, sparkSchema); } @Override From 21be53a686aab6c6be54334a7598377d148b4856 Mon Sep 17 00:00:00 2001 From: openinx Date: Thu, 23 Jul 2020 15:34:43 +0800 Subject: [PATCH 12/16] Make the few public classes/methods to be package-access --- .../iceberg/flink/TaskWriterFactory.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/flink/src/main/java/org/apache/iceberg/flink/TaskWriterFactory.java b/flink/src/main/java/org/apache/iceberg/flink/TaskWriterFactory.java index 71e5d4fd2921..bb22b78045ed 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/TaskWriterFactory.java +++ b/flink/src/main/java/org/apache/iceberg/flink/TaskWriterFactory.java @@ -41,17 +41,17 @@ import org.apache.iceberg.io.UnpartitionedWriter; import org.apache.iceberg.parquet.Parquet; -public class TaskWriterFactory { +class TaskWriterFactory { private TaskWriterFactory() { } - public static TaskWriter createTaskWriter(Schema schema, - PartitionSpec spec, - FileFormat format, - FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, - FileIO io, - long targetFileSizeBytes) { + static TaskWriter createTaskWriter(Schema schema, + PartitionSpec spec, + FileFormat format, + FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, + FileIO io, + long targetFileSizeBytes) { if (spec.fields().isEmpty()) { return new UnpartitionedWriter<>(spec, format, appenderFactory, fileFactory, io, targetFileSizeBytes); } else { @@ -60,7 +60,7 @@ public static TaskWriter createTaskWriter(Schema schema, } } - static class RowPartitionedFanoutWriter extends PartitionedFanoutWriter { + private static class RowPartitionedFanoutWriter extends PartitionedFanoutWriter { private final PartitionKey partitionKey; private final RowWrapper rowWrapper; From 61dde9b99619e1de92de96358ae39239c5416179 Mon Sep 17 00:00:00 2001 From: openinx Date: Mon, 27 Jul 2020 12:28:39 +0800 Subject: [PATCH 13/16] Addressing the comments --- .../org/apache/iceberg/io/BaseTaskWriter.java | 8 ++-- .../apache/iceberg/io/PartitionedWriter.java | 20 ++++---- .../iceberg/io/UnpartitionedWriter.java | 14 +++--- .../flink}/PartitionedFanoutWriter.java | 21 ++++---- .../iceberg/flink/TaskWriterFactory.java | 1 - site/docs/javadoc/0.9.0/serialized-form.html | 2 +- .../iceberg/spark/source/RowDataRewriter.java | 12 ++--- .../iceberg/spark/source/TaskResult.java | 48 ------------------- .../apache/iceberg/spark/source/Writer.java | 20 ++++---- .../iceberg/spark/source/SparkBatchWrite.java | 18 ++++--- 10 files changed, 64 insertions(+), 100 deletions(-) rename {core/src/main/java/org/apache/iceberg/io => flink/src/main/java/org/apache/iceberg/flink}/PartitionedFanoutWriter.java (80%) delete mode 100644 spark/src/main/java/org/apache/iceberg/spark/source/TaskResult.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 ba48220e6a3b..42d82443eed0 100644 --- a/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java @@ -34,7 +34,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.Tasks; -abstract class BaseTaskWriter implements TaskWriter { +public abstract class BaseTaskWriter implements TaskWriter { private final List completedFiles = Lists.newArrayList(); private final PartitionSpec spec; private final FileFormat format; @@ -75,7 +75,7 @@ public List complete() throws IOException { } } - class RollingFileAppender implements Closeable { + protected class RollingFileWriter implements Closeable { private static final int ROWS_DIVISOR = 1000; private final PartitionKey partitionKey; @@ -83,11 +83,11 @@ class RollingFileAppender implements Closeable { private FileAppender currentAppender = null; private long currentRows = 0; - RollingFileAppender(PartitionKey partitionKey) { + public RollingFileWriter(PartitionKey partitionKey) { this.partitionKey = partitionKey; } - void add(T record) throws IOException { + public void add(T record) throws IOException { if (currentAppender == null) { openCurrent(); } 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 30afdafb33e0..29871e04bf7e 100644 --- a/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java @@ -35,7 +35,7 @@ public abstract class PartitionedWriter extends BaseTaskWriter { private final Set completedPartitions = Sets.newHashSet(); private PartitionKey currentKey = null; - private RollingFileAppender currentAppender = null; + private RollingFileWriter currentWriter = null; public PartitionedWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, OutputFileFactory fileFactory, FileIO io, long targetFileSize) { @@ -57,7 +57,11 @@ public void write(T row) throws IOException { if (!key.equals(currentKey)) { closeCurrent(); - completedPartitions.add(currentKey); + + if (currentKey != null) { + // if the key is null, there was no previous current key + completedPartitions.add(currentKey); + } if (completedPartitions.contains(key)) { // if rows are not correctly grouped, detect and fail the write @@ -69,11 +73,11 @@ public void write(T row) throws IOException { currentKey = key.copy(); } - if (currentAppender == null) { - currentAppender = new RollingFileAppender(currentKey); + if (currentWriter == null) { + currentWriter = new RollingFileWriter(currentKey); } - currentAppender.add(row); + currentWriter.add(row); } @Override @@ -82,13 +86,13 @@ public void close() throws IOException { } private void closeCurrent() throws IOException { - if (currentAppender != null) { + if (currentWriter != null) { // Close the current file appender. - currentAppender.close(); + currentWriter.close(); // Reset the current appender to be null. - currentAppender = null; + currentWriter = 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 5f160f7ae82a..9ca5a6971788 100644 --- a/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java @@ -25,7 +25,7 @@ public class UnpartitionedWriter extends BaseTaskWriter { - private RollingFileAppender currentAppender = null; + private RollingFileWriter currentWriter = null; public UnpartitionedWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, OutputFileFactory fileFactory, FileIO io, long targetFileSize) { @@ -34,10 +34,10 @@ public UnpartitionedWriter(PartitionSpec spec, FileFormat format, FileAppenderFa @Override public void write(T record) throws IOException { - if (currentAppender == null) { - currentAppender = new RollingFileAppender(null); + if (currentWriter == null) { + currentWriter = new RollingFileWriter(null); } - currentAppender.add(record); + currentWriter.add(record); } @Override @@ -46,13 +46,13 @@ public void close() throws IOException { } private void closeCurrent() throws IOException { - if (currentAppender != null) { + if (currentWriter != null) { // Close the current file appender and put the generated DataFile to completeDataFiles. - currentAppender.close(); + currentWriter.close(); // Reset the current appender to be null. - currentAppender = null; + currentWriter = null; } } } diff --git a/core/src/main/java/org/apache/iceberg/io/PartitionedFanoutWriter.java b/flink/src/main/java/org/apache/iceberg/flink/PartitionedFanoutWriter.java similarity index 80% rename from core/src/main/java/org/apache/iceberg/io/PartitionedFanoutWriter.java rename to flink/src/main/java/org/apache/iceberg/flink/PartitionedFanoutWriter.java index 99fd4adbf705..f8a6875f0ac4 100644 --- a/core/src/main/java/org/apache/iceberg/io/PartitionedFanoutWriter.java +++ b/flink/src/main/java/org/apache/iceberg/flink/PartitionedFanoutWriter.java @@ -17,18 +17,21 @@ * under the License. */ -package org.apache.iceberg.io; +package org.apache.iceberg.flink; import java.io.IOException; -import java.util.Iterator; import java.util.Map; import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.io.BaseTaskWriter; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFileFactory; import org.apache.iceberg.relocated.com.google.common.collect.Maps; public abstract class PartitionedFanoutWriter extends BaseTaskWriter { - private final Map writers = Maps.newHashMap(); + private final Map writers = Maps.newHashMap(); public PartitionedFanoutWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, OutputFileFactory fileFactory, FileIO io, long targetFileSize) { @@ -48,11 +51,11 @@ public PartitionedFanoutWriter(PartitionSpec spec, FileFormat format, FileAppend public void write(T row) throws IOException { PartitionKey partitionKey = partition(row); - RollingFileAppender 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 RollingFileAppender(copiedKey); + writer = new RollingFileWriter(copiedKey); writers.put(copiedKey, writer); } @@ -62,12 +65,10 @@ public void write(T row) throws IOException { @Override public void close() throws IOException { if (!writers.isEmpty()) { - Iterator iterator = writers.values().iterator(); - while (iterator.hasNext()) { - iterator.next().close(); - // Remove from the writers after closed. - iterator.remove(); + for (PartitionKey key : writers.keySet()) { + writers.get(key).close(); } + writers.clear(); } } } diff --git a/flink/src/main/java/org/apache/iceberg/flink/TaskWriterFactory.java b/flink/src/main/java/org/apache/iceberg/flink/TaskWriterFactory.java index bb22b78045ed..2bd6e24977f9 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/TaskWriterFactory.java +++ b/flink/src/main/java/org/apache/iceberg/flink/TaskWriterFactory.java @@ -36,7 +36,6 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.io.OutputFileFactory; -import org.apache.iceberg.io.PartitionedFanoutWriter; import org.apache.iceberg.io.TaskWriter; import org.apache.iceberg.io.UnpartitionedWriter; import org.apache.iceberg.parquet.Parquet; diff --git a/site/docs/javadoc/0.9.0/serialized-form.html b/site/docs/javadoc/0.9.0/serialized-form.html index 529779979b4c..68dbe159a6ed 100644 --- a/site/docs/javadoc/0.9.0/serialized-form.html +++ b/site/docs/javadoc/0.9.0/serialized-form.html @@ -1031,7 +1031,7 @@

spec

  • -

    Class org.apache.iceberg.spark.source.SparkBatchWrite.TaskCommit extends org.apache.iceberg.spark.source.TaskResult implements Serializable

    +

    Class org.apache.iceberg.spark.source.SparkBatchWrite.TaskCommit implements Serializable

  • 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 2a8a895887ee..3774c9dec385 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 @@ -20,7 +20,7 @@ package org.apache.iceberg.spark.source; import java.io.Serializable; -import java.util.Arrays; +import java.util.Collection; import java.util.List; import java.util.Locale; import java.util.Map; @@ -81,14 +81,14 @@ public RowDataRewriter(Table table, PartitionSpec spec, boolean caseSensitive, } public List rewriteDataForTasks(JavaRDD taskRDD) { - JavaRDD taskCommitRDD = taskRDD.map(this::rewriteDataForTask); + JavaRDD> dataFilesRDD = taskRDD.map(this::rewriteDataForTask); - return taskCommitRDD.collect().stream() - .flatMap(taskCommit -> Arrays.stream(taskCommit.files())) + return dataFilesRDD.collect().stream() + .flatMap(Collection::stream) .collect(Collectors.toList()); } - private TaskResult rewriteDataForTask(CombinedScanTask task) throws Exception { + private List rewriteDataForTask(CombinedScanTask task) throws Exception { TaskContext context = TaskContext.get(); int partitionId = context.partitionId(); long taskId = context.taskAttemptId(); @@ -119,7 +119,7 @@ private TaskResult rewriteDataForTask(CombinedScanTask task) throws Exception { dataReader = null; writer.close(); - return new TaskResult(writer.complete()); + return writer.complete(); } catch (Throwable originalThrowable) { try { diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/TaskResult.java b/spark/src/main/java/org/apache/iceberg/spark/source/TaskResult.java deleted file mode 100644 index 1a0b8fcf0caa..000000000000 --- a/spark/src/main/java/org/apache/iceberg/spark/source/TaskResult.java +++ /dev/null @@ -1,48 +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.spark.source; - -import java.io.Serializable; -import java.util.List; -import org.apache.iceberg.DataFile; - -class TaskResult implements Serializable { - private final DataFile[] files; - - TaskResult() { - this.files = new DataFile[0]; - } - - TaskResult(DataFile file) { - this.files = new DataFile[] { file }; - } - - TaskResult(List files) { - this.files = files.toArray(new DataFile[files.size()]); - } - - TaskResult(DataFile[] files) { - this.files = files; - } - - DataFile[] files() { - return files; - } -} 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 ee1860c3d125..42c1bc1e3ff7 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 @@ -217,7 +217,7 @@ protected Table table() { protected Iterable files(WriterCommitMessage[] messages) { if (messages.length > 0) { return Iterables.concat(Iterables.transform(Arrays.asList(messages), message -> message != null ? - ImmutableList.copyOf(((TaskResult) message).files()) : + ImmutableList.copyOf(((TaskCommit) message).files()) : ImmutableList.of())); } return ImmutableList.of(); @@ -228,9 +228,15 @@ public String toString() { return String.format("IcebergWrite(table=%s, format=%s)", table, format); } - private static class TaskCommit extends TaskResult implements WriterCommitMessage { - TaskCommit(TaskResult toCopy) { - super(toCopy.files()); + private static class TaskCommit implements WriterCommitMessage { + private final List taskFiles; + + TaskCommit(List files) { + this.taskFiles = files; + } + + List files() { + return this.taskFiles; } } @@ -286,8 +292,7 @@ private static class Unpartitioned24Writer extends UnpartitionedWriter dataFiles = complete(); - return new TaskCommit(new TaskResult(dataFiles)); + return new TaskCommit(complete()); } } @@ -303,8 +308,7 @@ private static class Partitioned24Writer extends SparkPartitionedWriter implemen public WriterCommitMessage commit() throws IOException { this.close(); - List dataFiles = complete(); - return new TaskCommit(new TaskResult(dataFiles)); + return new TaskCommit(complete()); } } } diff --git a/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchWrite.java b/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchWrite.java index 5bdce9f82ac4..1cfc21f54b9a 100644 --- a/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchWrite.java +++ b/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchWrite.java @@ -245,9 +245,15 @@ public String toString() { return String.format("IcebergWrite(table=%s, format=%s)", table, format); } - public static class TaskCommit extends TaskResult implements WriterCommitMessage { - TaskCommit(TaskResult result) { - super(result.files()); + public static class TaskCommit implements WriterCommitMessage { + private final List taskFiles; + + TaskCommit(List taskFiles) { + this.taskFiles = taskFiles; + } + + public List files() { + return taskFiles; } } @@ -307,8 +313,7 @@ private static class Unpartitioned3Writer extends UnpartitionedWriter dataFiles = complete(); - return new TaskCommit(new TaskResult(dataFiles)); + return new TaskCommit(complete()); } } @@ -323,8 +328,7 @@ private static class Partitioned3Writer extends SparkPartitionedWriter implement public WriterCommitMessage commit() throws IOException { this.close(); - List dataFiles = complete(); - return new TaskCommit(new TaskResult(dataFiles)); + return new TaskCommit(complete()); } } } From 8a0920796a1958c9d92e8f564b60d171c5007786 Mon Sep 17 00:00:00 2001 From: openinx Date: Mon, 27 Jul 2020 15:34:33 +0800 Subject: [PATCH 14/16] Remove the public modifiers from PartitionedFanoutWriter --- .../org/apache/iceberg/flink/PartitionedFanoutWriter.java | 6 +++--- .../main/java/org/apache/iceberg/spark/source/Writer.java | 3 --- .../org/apache/iceberg/spark/source/SparkBatchWrite.java | 3 --- 3 files changed, 3 insertions(+), 9 deletions(-) diff --git a/flink/src/main/java/org/apache/iceberg/flink/PartitionedFanoutWriter.java b/flink/src/main/java/org/apache/iceberg/flink/PartitionedFanoutWriter.java index f8a6875f0ac4..78c29c5d112b 100644 --- a/flink/src/main/java/org/apache/iceberg/flink/PartitionedFanoutWriter.java +++ b/flink/src/main/java/org/apache/iceberg/flink/PartitionedFanoutWriter.java @@ -30,11 +30,11 @@ import org.apache.iceberg.io.OutputFileFactory; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -public abstract class PartitionedFanoutWriter extends BaseTaskWriter { +abstract class PartitionedFanoutWriter extends BaseTaskWriter { private final Map writers = Maps.newHashMap(); - public PartitionedFanoutWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, FileIO io, long targetFileSize) { + PartitionedFanoutWriter(PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, FileIO io, long targetFileSize) { super(spec, format, appenderFactory, fileFactory, io, targetFileSize); } 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 42c1bc1e3ff7..41f632303f5b 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 @@ -43,9 +43,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.tasks.OutputFileFactory; -import org.apache.iceberg.tasks.PartitionedWriter; -import org.apache.iceberg.tasks.UnpartitionedWriter; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.Tasks; import org.apache.spark.broadcast.Broadcast; diff --git a/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchWrite.java b/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchWrite.java index 1cfc21f54b9a..ad988346681c 100644 --- a/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchWrite.java +++ b/spark3/src/main/java/org/apache/iceberg/spark/source/SparkBatchWrite.java @@ -45,9 +45,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.tasks.OutputFileFactory; -import org.apache.iceberg.tasks.PartitionedWriter; -import org.apache.iceberg.tasks.UnpartitionedWriter; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.Tasks; import org.apache.spark.broadcast.Broadcast; From 0c7423aed5800862f2830d290b204e099926b1f8 Mon Sep 17 00:00:00 2001 From: openinx Date: Fri, 31 Jul 2020 15:05:39 +0800 Subject: [PATCH 15/16] Addressing the lastest comment from Ryan Blue. --- .../org/apache/iceberg/io/BaseTaskWriter.java | 18 ++--- .../apache/iceberg/io/PartitionedWriter.java | 21 +----- .../org/apache/iceberg/io/TaskWriter.java | 3 +- .../iceberg/io/UnpartitionedWriter.java | 19 +---- .../apache/iceberg/flink/TestTaskWriters.java | 71 ++++++------------- site/docs/javadoc/0.9.0/serialized-form.html | 2 +- .../iceberg/spark/source/RowDataRewriter.java | 3 +- .../apache/iceberg/spark/source/Writer.java | 11 ++- .../iceberg/spark/source/SparkBatchWrite.java | 7 +- 9 files changed, 46 insertions(+), 109 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 42d82443eed0..8c855efe52ea 100644 --- a/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java @@ -21,7 +21,6 @@ import java.io.Closeable; import java.io.IOException; -import java.util.Collections; import java.util.List; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; @@ -30,7 +29,6 @@ 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.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.Tasks; @@ -65,14 +63,10 @@ public void abort() throws IOException { } @Override - public List complete() throws IOException { + public DataFile[] complete() throws IOException { close(); - if (completedFiles.size() > 0) { - return ImmutableList.copyOf(completedFiles); - } else { - return Collections.emptyList(); - } + return completedFiles.toArray(new DataFile[0]); } protected class RollingFileWriter implements Closeable { @@ -85,23 +79,21 @@ protected class RollingFileWriter implements Closeable { public RollingFileWriter(PartitionKey partitionKey) { this.partitionKey = partitionKey; + openCurrent(); } public void add(T record) throws IOException { - if (currentAppender == null) { - openCurrent(); - } - this.currentAppender.add(record); this.currentRows++; if (shouldRollToNewFile()) { closeCurrent(); + openCurrent(); } } private void openCurrent() { - if (spec.fields().size() == 0) { + if (partitionKey == null) { // unpartitioned currentFile = fileFactory.newOutputFile(); } else { 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 29871e04bf7e..23e457653900 100644 --- a/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java @@ -56,10 +56,9 @@ public void write(T row) throws IOException { PartitionKey key = partition(row); if (!key.equals(currentKey)) { - closeCurrent(); - if (currentKey != null) { - // if the key is null, there was no previous current key + // if the key is null, there was no previous current key and current writer. + currentWriter.close(); completedPartitions.add(currentKey); } @@ -71,9 +70,6 @@ public void write(T row) throws IOException { } currentKey = key.copy(); - } - - if (currentWriter == null) { currentWriter = new RollingFileWriter(currentKey); } @@ -82,17 +78,6 @@ public void write(T row) throws IOException { @Override public void close() throws IOException { - closeCurrent(); - } - - private void closeCurrent() throws IOException { - if (currentWriter != null) { - - // Close the current file appender. - currentWriter.close(); - - // Reset the current appender to be null. - currentWriter = null; - } + currentWriter.close(); } } 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 438ea4d5c65c..806e37de1bee 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 java.util.List; import org.apache.iceberg.DataFile; /** @@ -48,5 +47,5 @@ public interface TaskWriter extends Closeable { * * @return the completed data files of this task writer. */ - List complete() throws IOException; + DataFile[] 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 9ca5a6971788..37f3db49aef4 100644 --- a/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/UnpartitionedWriter.java @@ -25,34 +25,21 @@ public class UnpartitionedWriter extends BaseTaskWriter { - private RollingFileWriter currentWriter = null; + 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 RollingFileWriter(null); } @Override public void write(T record) throws IOException { - if (currentWriter == null) { - currentWriter = new RollingFileWriter(null); - } currentWriter.add(record); } @Override public void close() throws IOException { - closeCurrent(); - } - - private void closeCurrent() throws IOException { - if (currentWriter != null) { - - // Close the current file appender and put the generated DataFile to completeDataFiles. - currentWriter.close(); - - // Reset the current appender to be null. - currentWriter = null; - } + currentWriter.close(); } } diff --git a/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java b/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java index bca917447f24..04962025192b 100644 --- a/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java +++ b/flink/src/test/java/org/apache/iceberg/flink/TestTaskWriters.java @@ -29,7 +29,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; @@ -93,15 +92,15 @@ public void testWriteZeroRecord() throws IOException { try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.close(); - List dataFiles = taskWriter.complete(); + DataFile[] dataFiles = taskWriter.complete(); Assert.assertNotNull(dataFiles); - Assert.assertEquals(0, dataFiles.size()); + Assert.assertEquals(0, dataFiles.length); // Close again. taskWriter.close(); dataFiles = taskWriter.complete(); Assert.assertNotNull(dataFiles); - Assert.assertEquals(0, dataFiles.size()); + Assert.assertEquals(0, dataFiles.length); } } @@ -114,8 +113,8 @@ public void testCloseTwice() throws IOException { taskWriter.close(); // The second close int expectedFiles = partitioned ? 2 : 1; - List dataFiles = taskWriter.complete(); - Assert.assertEquals(expectedFiles, dataFiles.size()); + DataFile[] dataFiles = taskWriter.complete(); + Assert.assertEquals(expectedFiles, dataFiles.length); FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { @@ -131,10 +130,10 @@ public void testAbort() throws IOException { taskWriter.write(Row.of(2, "world")); taskWriter.abort(); - List dataFiles = taskWriter.complete(); + DataFile[] dataFiles = taskWriter.complete(); int expectedFiles = partitioned ? 2 : 1; - Assert.assertEquals(expectedFiles, dataFiles.size()); + Assert.assertEquals(expectedFiles, dataFiles.length); FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { @@ -151,17 +150,12 @@ public void testCompleteFiles() throws IOException { taskWriter.write(Row.of(3, "c")); taskWriter.write(Row.of(4, "d")); - List dataFiles = taskWriter.complete(); + DataFile[] dataFiles = taskWriter.complete(); int expectedFiles = partitioned ? 4 : 1; - Assert.assertEquals(expectedFiles, dataFiles.size()); + Assert.assertEquals(expectedFiles, dataFiles.length); dataFiles = taskWriter.complete(); - Assert.assertEquals(expectedFiles, dataFiles.size()); - - taskWriter.write(Row.of(5, "e")); - dataFiles = taskWriter.complete(); - expectedFiles = partitioned ? 5 : 2; - Assert.assertEquals(expectedFiles, dataFiles.size()); + Assert.assertEquals(expectedFiles, dataFiles.length); FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { @@ -169,7 +163,9 @@ public void testCompleteFiles() throws IOException { } AppendFiles appendFiles = table.newAppend(); - dataFiles.forEach(appendFiles::appendFile); + for (DataFile dataFile : dataFiles) { + appendFiles.appendFile(dataFile); + } appendFiles.commit(); // Assert the data rows. @@ -177,36 +173,11 @@ public void testCompleteFiles() throws IOException { SimpleDataUtil.createRecord(1, "a"), SimpleDataUtil.createRecord(2, "b"), SimpleDataUtil.createRecord(3, "c"), - SimpleDataUtil.createRecord(4, "d"), - SimpleDataUtil.createRecord(5, "e") + SimpleDataUtil.createRecord(4, "d") )); } } - @Test - public void testImmutableCompleteFiles() throws IOException { - try (TaskWriter taskWriter = createTaskWriter(4)) { - List rows = Lists.newArrayListWithCapacity(1000); - for (int i = 0; i < 1000; i++) { - rows.add(Row.of(i, "a")); - } - - for (Row row : rows) { - taskWriter.write(row); - } - - List dataFiles = taskWriter.complete(); - Assert.assertEquals(1, dataFiles.size()); - AssertHelpers.assertThrows("Complete file list are immutable", UnsupportedOperationException.class, - () -> dataFiles.add(null)); - - List dataFilesAgain = taskWriter.complete(); - Assert.assertEquals(1, dataFilesAgain.size()); - AssertHelpers.assertThrows("Empty complete file list are immutable", UnsupportedOperationException.class, - () -> dataFilesAgain.add(null)); - } - } - @Test public void testRollingWithTargetFileSize() throws IOException { try (TaskWriter taskWriter = createTaskWriter(4)) { @@ -223,11 +194,13 @@ public void testRollingWithTargetFileSize() throws IOException { taskWriter.write(row); } - List dataFiles = taskWriter.complete(); - Assert.assertEquals(8, dataFiles.size()); + DataFile[] dataFiles = taskWriter.complete(); + Assert.assertEquals(8, dataFiles.length); AppendFiles appendFiles = table.newAppend(); - dataFiles.forEach(appendFiles::appendFile); + for (DataFile dataFile : dataFiles) { + appendFiles.appendFile(dataFile); + } appendFiles.commit(); // Assert the data rows. @@ -244,9 +217,11 @@ public void testRandomData() throws IOException { } taskWriter.close(); - List dataFiles = taskWriter.complete(); + DataFile[] dataFiles = taskWriter.complete(); AppendFiles appendFiles = table.newAppend(); - dataFiles.forEach(appendFiles::appendFile); + for (DataFile dataFile : dataFiles) { + appendFiles.appendFile(dataFile); + } appendFiles.commit(); // Assert the data rows. diff --git a/site/docs/javadoc/0.9.0/serialized-form.html b/site/docs/javadoc/0.9.0/serialized-form.html index 68dbe159a6ed..529779979b4c 100644 --- a/site/docs/javadoc/0.9.0/serialized-form.html +++ b/site/docs/javadoc/0.9.0/serialized-form.html @@ -1031,7 +1031,7 @@

    spec

  • -

    Class org.apache.iceberg.spark.source.SparkBatchWrite.TaskCommit implements Serializable

    +

    Class org.apache.iceberg.spark.source.SparkBatchWrite.TaskCommit extends org.apache.iceberg.spark.source.TaskResult implements Serializable

  • 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 3774c9dec385..37ca56c700a4 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 @@ -38,6 +38,7 @@ import org.apache.iceberg.io.OutputFileFactory; 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; import org.apache.spark.TaskContext; import org.apache.spark.api.java.JavaRDD; @@ -119,7 +120,7 @@ private List rewriteDataForTask(CombinedScanTask task) throws Exceptio dataReader = null; writer.close(); - return writer.complete(); + return Lists.newArrayList(writer.complete()); } 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 41f632303f5b..cf06a5176b8d 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 @@ -21,7 +21,6 @@ import java.io.IOException; import java.util.Arrays; -import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Optional; @@ -226,13 +225,13 @@ public String toString() { } private static class TaskCommit implements WriterCommitMessage { - private final List taskFiles; + private final DataFile[] taskFiles; - TaskCommit(List files) { + TaskCommit(DataFile[] files) { this.taskFiles = files; } - List files() { + DataFile[] files() { return this.taskFiles; } } @@ -287,7 +286,7 @@ private static class Unpartitioned24Writer extends UnpartitionedWriter taskFiles; + private final DataFile[] taskFiles; - TaskCommit(List taskFiles) { + TaskCommit(DataFile[] taskFiles) { this.taskFiles = taskFiles; } - public List files() { + DataFile[] files() { return taskFiles; } } From e6152d19453d144bf8a7bade90ede997537c5eda Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Fri, 31 Jul 2020 11:38:37 -0700 Subject: [PATCH 16/16] Fix NullPointerException in PartitionedWriter --- .../main/java/org/apache/iceberg/io/PartitionedWriter.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) 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 23e457653900..7b5fa90e4502 100644 --- a/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java @@ -78,6 +78,8 @@ public void write(T row) throws IOException { @Override public void close() throws IOException { - currentWriter.close(); + if (currentWriter != null) { + currentWriter.close(); + } } }