rdblue commented on a change in pull request #1213: URL: https://github.com/apache/iceberg/pull/1213#discussion_r460583237
########## File path: core/src/main/java/org/apache/iceberg/io/PartitionedWriter.java ########## @@ -17,41 +17,44 @@ * under the License. */ -package org.apache.iceberg.spark.source; +package org.apache.iceberg.io; import java.io.IOException; import java.util.Set; 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.slf4j.Logger; import org.slf4j.LoggerFactory; -class PartitionedWriter extends BaseWriter { +public abstract class PartitionedWriter<T> extends BaseTaskWriter<T> { private static final Logger LOG = LoggerFactory.getLogger(PartitionedWriter.class); - private final PartitionKey key; - private final InternalRowWrapper wrapper; private final Set<PartitionKey> completedPartitions = Sets.newHashSet(); - PartitionedWriter(PartitionSpec spec, FileFormat format, SparkAppenderFactory appenderFactory, - OutputFileFactory fileFactory, FileIO io, long targetFileSize, Schema writeSchema) { + private PartitionKey currentKey = null; + private RollingFileAppender currentAppender = null; Review comment: Now that the current key is null, we will need a check before adding it to `completedPartitions` in the `write` method: ```java if (!key.equals(currentKey)) { closeCurrent(); if (currentKey != null) { // if the key is null, there was no previous current key completedPartitions.add(currentKey); } } ``` ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org