You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/07/27 00:06:50 UTC

[GitHub] [iceberg] aokolnychyi opened a new pull request #2873: Core: Add WriterFactory

aokolnychyi opened a new pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873


   This PR adds a new interface called `WriterFactory` for creating data and delete writers.
   
   This change also includes `BaseWriterFactory` to be shared by query engine integrations and a Spark writer factory.


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677125868



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/SparkWriterFactory.java
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.data.BaseWriterFactory;
+import org.apache.iceberg.io.DeleteSchemaUtil;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.data.SparkAvroWriter;
+import org.apache.iceberg.spark.data.SparkOrcWriter;
+import org.apache.iceberg.spark.data.SparkParquetWriters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+import static org.apache.iceberg.MetadataColumns.DELETE_FILE_ROW_FIELD_NAME;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.DELETE_DEFAULT_FILE_FORMAT;
+
+class SparkWriterFactory extends BaseWriterFactory<InternalRow> {
+  private StructType dataSparkType;
+  private StructType equalityDeleteSparkType;
+  private StructType positionDeleteSparkType;
+
+  SparkWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema, StructType dataSparkType,
+                     SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                     int[] equalityFieldIds, Schema equalityDeleteRowSchema, StructType equalityDeleteSparkType,
+                     SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                     StructType positionDeleteSparkType, SortOrder positionDeleteSortOrder) {
+
+    super(table, dataFileFormat, dataSchema, dataSortOrder, deleteFileFormat, equalityFieldIds, equalityDeleteRowSchema,
+        equalityDeleteSortOrder, positionDeleteRowSchema, positionDeleteSortOrder);
+
+    this.dataSparkType = dataSparkType;
+    this.equalityDeleteSparkType = equalityDeleteSparkType;
+    this.positionDeleteSparkType = positionDeleteSparkType;
+  }
+
+  static Builder builderFor(Table table) {
+    return new Builder(table);
+  }
+
+  @Override
+  protected void configureDataWrite(Avro.DataWriteBuilder builder) {
+    builder.createWriterFunc(ignored -> new SparkAvroWriter(dataSparkType()));
+  }
+
+  @Override
+  protected void configureEqualityDelete(Avro.DeleteWriteBuilder builder) {
+    builder.createWriterFunc(ignored -> new SparkAvroWriter(equalityDeleteSparkType()));
+  }
+
+  @Override
+  protected void configurePositionDelete(Avro.DeleteWriteBuilder builder) {
+    boolean withRow = positionDeleteSparkType().getFieldIndex(DELETE_FILE_ROW_FIELD_NAME).isDefined();
+    if (withRow) {
+      // SparkAvroWriter accepts just the Spark type of the row ignoring the path and pos
+      StructField rowField = positionDeleteSparkType().apply(DELETE_FILE_ROW_FIELD_NAME);
+      StructType positionDeleteRowSparkType = (StructType) rowField.dataType();
+      builder.createWriterFunc(ignored -> new SparkAvroWriter(positionDeleteRowSparkType));
+    }
+  }
+
+  @Override
+  protected void configureDataWrite(Parquet.DataWriteBuilder builder) {
+    builder.createWriterFunc(msgType -> SparkParquetWriters.buildWriter(dataSparkType(), msgType));
+  }
+
+  @Override
+  protected void configureEqualityDelete(Parquet.DeleteWriteBuilder builder) {
+    builder.createWriterFunc(msgType -> SparkParquetWriters.buildWriter(equalityDeleteSparkType(), msgType));
+  }
+
+  @Override
+  protected void configurePositionDelete(Parquet.DeleteWriteBuilder builder) {
+    builder.createWriterFunc(msgType -> SparkParquetWriters.buildWriter(positionDeleteSparkType(), msgType));
+    builder.transformPaths(path -> UTF8String.fromString(path.toString()));
+  }
+
+  @Override
+  protected void configureDataWrite(ORC.DataWriteBuilder builder) {
+    builder.createWriterFunc(SparkOrcWriter::new);
+  }
+
+  private StructType dataSparkType() {
+    if (dataSparkType == null) {
+      Preconditions.checkNotNull(dataSchema(), "Data schema must not be null");
+      this.dataSparkType = SparkSchemaUtil.convert(dataSchema());
+    }
+
+    return dataSparkType;
+  }
+
+  private StructType equalityDeleteSparkType() {
+    if (equalityDeleteSparkType == null) {
+      Preconditions.checkNotNull(equalityDeleteRowSchema(), "Equality delete schema must not be null");
+      this.equalityDeleteSparkType = SparkSchemaUtil.convert(equalityDeleteRowSchema());
+    }
+
+    return equalityDeleteSparkType;
+  }
+
+  private StructType positionDeleteSparkType() {
+    if (positionDeleteSparkType == null) {
+      // wrap the optional row schema into the position delete schema that contains path and position
+      Schema positionDeleteSchema = DeleteSchemaUtil.posDeleteSchema(positionDeleteRowSchema());
+      this.positionDeleteSparkType = SparkSchemaUtil.convert(positionDeleteSchema);
+    }
+
+    return positionDeleteSparkType;
+  }
+
+  static class Builder {
+    private final Table table;
+    private FileFormat dataFileFormat;
+    private Schema dataSchema;
+    private StructType dataSparkType;
+    private SortOrder dataSortOrder;
+    private FileFormat deleteFileFormat;
+    private int[] equalityFieldIds;
+    private Schema equalityDeleteRowSchema;
+    private StructType equalityDeleteSparkType;
+    private SortOrder equalityDeleteSortOrder;
+    private Schema positionDeleteRowSchema;
+    private StructType positionDeleteSparkType;
+    private SortOrder positionDeleteSortOrder;
+
+    Builder(Table table) {
+      this.table = table;
+
+      Map<String, String> properties = table.properties();
+
+      String dataFileFormatName = properties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT);
+      this.dataFileFormat = FileFormat.valueOf(dataFileFormatName.toUpperCase(Locale.ENGLISH));
+
+      String deleteFileFormatName = properties.getOrDefault(DELETE_DEFAULT_FILE_FORMAT, dataFileFormatName);
+      this.deleteFileFormat = FileFormat.valueOf(deleteFileFormatName.toUpperCase(Locale.ENGLISH));
+    }
+
+    Builder dataFileFormat(FileFormat newDataFileFormat) {
+      this.dataFileFormat = newDataFileFormat;
+      return this;
+    }
+
+    Builder dataSchema(Schema newDataSchema) {
+      this.dataSchema = newDataSchema;
+      return this;
+    }
+
+    Builder dataSparkType(StructType newDataSparkType) {
+      this.dataSparkType = newDataSparkType;
+      return this;
+    }
+
+    Builder dataSortOrder(SortOrder newDataSortOrder) {
+      this.dataSortOrder = newDataSortOrder;
+      return this;
+    }
+
+    Builder deleteFileFormat(FileFormat newDeleteFileFormat) {
+      this.deleteFileFormat = newDeleteFileFormat;
+      return this;
+    }
+
+    Builder equalityFieldIds(int[] newEqualityFieldIds) {
+      this.equalityFieldIds = newEqualityFieldIds;
+      return this;
+    }
+
+    Builder equalityDeleteRowSchema(Schema newEqualityDeleteRowSchema) {
+      this.equalityDeleteRowSchema = newEqualityDeleteRowSchema;
+      return this;
+    }
+
+    Builder equalityDeleteSparkType(StructType newEqualityDeleteSparkType) {
+      this.equalityDeleteSparkType = newEqualityDeleteSparkType;
+      return this;
+    }
+
+    Builder equalityDeleteSortOrder(SortOrder newEqualityDeleteSortOrder) {
+      this.equalityDeleteSortOrder = newEqualityDeleteSortOrder;
+      return this;
+    }
+
+    Builder positionDeleteRowSchema(Schema newPositionDeleteRowSchema) {
+      this.positionDeleteRowSchema = newPositionDeleteRowSchema;
+      return this;
+    }
+
+    Builder positionDeleteSparkType(StructType newPositionDeleteSparkType) {

Review comment:
       Oh.. It is only for SparkWriterFactory.. 




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r680268521



##########
File path: data/src/main/java/org/apache/iceberg/data/BaseWriterFactory.java
##########
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriterFactory;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+/**
+ * A base writer factory to be extended by query engine integrations.
+ */
+public abstract class BaseWriterFactory<T> implements WriterFactory<T> {
+  private final Table table;
+  private final FileFormat dataFileFormat;
+  private final Schema dataSchema;
+  private final SortOrder dataSortOrder;
+  private final FileFormat deleteFileFormat;
+  private final int[] equalityFieldIds;
+  private final Schema equalityDeleteRowSchema;
+  private final SortOrder equalityDeleteSortOrder;
+  private final Schema positionDeleteRowSchema;
+  private final SortOrder positionDeleteSortOrder;
+
+  protected BaseWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema,
+                              SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                              int[] equalityFieldIds, Schema equalityDeleteRowSchema,
+                              SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                              SortOrder positionDeleteSortOrder) {

Review comment:
       I'll drop it for now then.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r680271898



##########
File path: data/src/main/java/org/apache/iceberg/data/BaseWriterFactory.java
##########
@@ -0,0 +1,257 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriterFactory;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+/**
+ * A base writer factory to be extended by query engine integrations.
+ */
+public abstract class BaseWriterFactory<T> implements WriterFactory<T> {
+  private final Table table;
+  private final FileFormat dataFileFormat;
+  private final Schema dataSchema;
+  private final SortOrder dataSortOrder;
+  private final FileFormat deleteFileFormat;
+  private final int[] equalityFieldIds;
+  private final Schema equalityDeleteRowSchema;
+  private final SortOrder equalityDeleteSortOrder;
+  private final Schema positionDeleteRowSchema;
+  private final SortOrder positionDeleteSortOrder;
+
+  protected BaseWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema,
+                              SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                              int[] equalityFieldIds, Schema equalityDeleteRowSchema,
+                              SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                              SortOrder positionDeleteSortOrder) {
+    this.table = table;
+    this.dataFileFormat = dataFileFormat;
+    this.dataSchema = dataSchema;
+    this.dataSortOrder = dataSortOrder;
+    this.deleteFileFormat = deleteFileFormat;
+    this.equalityFieldIds = equalityFieldIds;
+    this.equalityDeleteRowSchema = equalityDeleteRowSchema;
+    this.equalityDeleteSortOrder = equalityDeleteSortOrder;
+    this.positionDeleteRowSchema = positionDeleteRowSchema;
+    this.positionDeleteSortOrder = positionDeleteSortOrder;
+  }
+
+  protected abstract void configureDataWrite(Avro.DataWriteBuilder builder);
+  protected abstract void configureEqualityDelete(Avro.DeleteWriteBuilder builder);
+  protected abstract void configurePositionDelete(Avro.DeleteWriteBuilder builder);
+
+  protected abstract void configureDataWrite(Parquet.DataWriteBuilder builder);
+  protected abstract void configureEqualityDelete(Parquet.DeleteWriteBuilder builder);
+  protected abstract void configurePositionDelete(Parquet.DeleteWriteBuilder builder);
+
+  protected abstract void configureDataWrite(ORC.DataWriteBuilder builder);
+
+  @Override
+  public DataWriter<T> newDataWriter(EncryptedOutputFile file, PartitionSpec spec, StructLike partition) {
+    OutputFile outputFile = file.encryptingOutputFile();
+    EncryptionKeyMetadata keyMetadata = file.keyMetadata();
+    Map<String, String> properties = table.properties();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(properties);
+
+    try {
+      switch (dataFileFormat) {
+        case AVRO:
+          Avro.DataWriteBuilder avroBuilder = Avro.writeData(outputFile)
+              .schema(dataSchema)
+              .setAll(properties)
+              .metricsConfig(metricsConfig)
+              .withSpec(spec)
+              .withPartition(partition)
+              .withKeyMetadata(keyMetadata)
+              .withSortOrder(dataSortOrder)
+              .overwrite();
+
+          configureDataWrite(avroBuilder);
+
+          return avroBuilder.build();
+
+        case PARQUET:
+          Parquet.DataWriteBuilder parquetBuilder = Parquet.writeData(outputFile)
+              .schema(dataSchema)
+              .setAll(properties)
+              .metricsConfig(metricsConfig)
+              .withSpec(spec)
+              .withPartition(partition)
+              .withKeyMetadata(keyMetadata)
+              .withSortOrder(dataSortOrder)
+              .overwrite();
+
+          configureDataWrite(parquetBuilder);
+
+          return parquetBuilder.build();
+
+        case ORC:
+          ORC.DataWriteBuilder orcBuilder = ORC.writeData(outputFile)
+              .schema(dataSchema)
+              .setAll(properties)
+              .metricsConfig(metricsConfig)
+              .withSpec(spec)
+              .withPartition(partition)
+              .withKeyMetadata(keyMetadata)
+              .withSortOrder(dataSortOrder)
+              .overwrite();
+
+          configureDataWrite(orcBuilder);
+
+          return orcBuilder.build();
+
+        default:
+          throw new UnsupportedOperationException("Unsupported data file format: " + dataFileFormat);
+      }
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public EqualityDeleteWriter<T> newEqualityDeleteWriter(EncryptedOutputFile file, PartitionSpec spec,
+                                                         StructLike partition) {
+    OutputFile outputFile = file.encryptingOutputFile();
+    EncryptionKeyMetadata keyMetadata = file.keyMetadata();
+    Map<String, String> properties = table.properties();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(properties);
+
+    try {
+      switch (deleteFileFormat) {
+        case AVRO:
+          // TODO: support metrics configs in Avro equality delete writer
+
+          Avro.DeleteWriteBuilder avroBuilder = Avro.writeDeletes(outputFile)
+              .setAll(properties)
+              .rowSchema(equalityDeleteRowSchema)
+              .equalityFieldIds(equalityFieldIds)
+              .withSpec(spec)
+              .withPartition(partition)
+              .withKeyMetadata(keyMetadata)
+              .withSortOrder(equalityDeleteSortOrder)
+              .overwrite();
+
+          configureEqualityDelete(avroBuilder);
+
+          return avroBuilder.buildEqualityWriter();
+
+        case PARQUET:
+          Parquet.DeleteWriteBuilder parquetBuilder = Parquet.writeDeletes(outputFile)
+              .setAll(properties)
+              .metricsConfig(metricsConfig)
+              .rowSchema(equalityDeleteRowSchema)
+              .equalityFieldIds(equalityFieldIds)
+              .withSpec(spec)
+              .withPartition(partition)
+              .withKeyMetadata(keyMetadata)
+              .withSortOrder(equalityDeleteSortOrder)
+              .overwrite();
+
+          configureEqualityDelete(parquetBuilder);
+
+          return parquetBuilder.buildEqualityWriter();
+
+        default:
+          throw new UnsupportedOperationException("Unsupported format for equality deletes: " + deleteFileFormat);

Review comment:
       Do we want to fail or should we just fall back to Avro?




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677636598



##########
File path: data/src/main/java/org/apache/iceberg/data/BaseWriterFactory.java
##########
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriterFactory;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+/**
+ * A base writer factory to be extended by query engine integrations.
+ */
+public abstract class BaseWriterFactory<T> implements WriterFactory<T> {
+  private final Table table;
+  private final FileFormat dataFileFormat;
+  private final Schema dataSchema;
+  private final SortOrder dataSortOrder;
+  private final FileFormat deleteFileFormat;
+  private final int[] equalityFieldIds;
+  private final Schema equalityDeleteRowSchema;
+  private final SortOrder equalityDeleteSortOrder;
+  private final Schema positionDeleteRowSchema;
+  private final SortOrder positionDeleteSortOrder;
+
+  protected BaseWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema,
+                              SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                              int[] equalityFieldIds, Schema equalityDeleteRowSchema,
+                              SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                              SortOrder positionDeleteSortOrder) {
+    this.table = table;
+    this.dataFileFormat = dataFileFormat;
+    this.dataSchema = dataSchema;
+    this.dataSortOrder = dataSortOrder;
+    this.deleteFileFormat = deleteFileFormat;
+    this.equalityFieldIds = equalityFieldIds;
+    this.equalityDeleteRowSchema = equalityDeleteRowSchema;
+    this.equalityDeleteSortOrder = equalityDeleteSortOrder;
+    this.positionDeleteRowSchema = positionDeleteRowSchema;
+    this.positionDeleteSortOrder = positionDeleteSortOrder;
+  }
+
+  protected abstract void configureDataWrite(Avro.DataWriteBuilder builder);
+  protected abstract void configureEqualityDelete(Avro.DeleteWriteBuilder builder);
+  protected abstract void configurePositionDelete(Avro.DeleteWriteBuilder builder);
+
+  protected abstract void configureDataWrite(Parquet.DataWriteBuilder builder);
+  protected abstract void configureEqualityDelete(Parquet.DeleteWriteBuilder builder);
+  protected abstract void configurePositionDelete(Parquet.DeleteWriteBuilder builder);
+
+  protected abstract void configureDataWrite(ORC.DataWriteBuilder builder);
+
+  @Override
+  public DataWriter<T> newDataWriter(EncryptedOutputFile file, PartitionSpec spec, StructLike partition) {
+    OutputFile outputFile = file.encryptingOutputFile();
+    EncryptionKeyMetadata keyMetadata = file.keyMetadata();
+    Map<String, String> properties = table.properties();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(properties);
+
+    try {
+      switch (dataFileFormat) {
+        case AVRO:
+          Avro.DataWriteBuilder avroBuilder = Avro.writeData(outputFile)
+              .schema(dataSchema)
+              .setAll(properties)
+              .metricsConfig(metricsConfig)
+              .withSpec(spec)
+              .withPartition(partition)
+              .withKeyMetadata(keyMetadata)
+              .withSortOrder(dataSortOrder)
+              .overwrite();
+
+          configureDataWrite(avroBuilder);
+
+          return avroBuilder.build();
+
+        case PARQUET:
+          Parquet.DataWriteBuilder parquetBuilder = Parquet.writeData(outputFile)
+              .schema(dataSchema)
+              .setAll(properties)
+              .metricsConfig(metricsConfig)
+              .withSpec(spec)
+              .withPartition(partition)
+              .withKeyMetadata(keyMetadata)
+              .withSortOrder(dataSortOrder)
+              .overwrite();
+
+          configureDataWrite(parquetBuilder);
+
+          return parquetBuilder.build();
+
+        case ORC:
+          ORC.DataWriteBuilder orcBuilder = ORC.writeData(outputFile)
+              .schema(dataSchema)
+              .setAll(properties)
+              .metricsConfig(metricsConfig)
+              .withSpec(spec)
+              .withPartition(partition)
+              .withKeyMetadata(keyMetadata)
+              .withSortOrder(dataSortOrder)
+              .overwrite();
+
+          configureDataWrite(orcBuilder);
+
+          return orcBuilder.build();
+
+        default:
+          throw new UnsupportedOperationException("Unsupported data file format: " + dataFileFormat);
+      }
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public EqualityDeleteWriter<T> newEqualityDeleteWriter(EncryptedOutputFile file, PartitionSpec spec,
+                                                         StructLike partition) {
+    OutputFile outputFile = file.encryptingOutputFile();
+    EncryptionKeyMetadata keyMetadata = file.keyMetadata();
+    Map<String, String> properties = table.properties();
+
+    // TODO: build and pass a correct metrics config for equality deletes

Review comment:
       I agree about using the data metric configuration for equality deletes. It wasn't there in `SparkAppenderFactory` but I think we can add it now. I'll do that in this PR and then we can address position deletes later.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r679599503



##########
File path: data/src/main/java/org/apache/iceberg/data/BaseWriterFactory.java
##########
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriterFactory;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+/**
+ * A base writer factory to be extended by query engine integrations.
+ */
+public abstract class BaseWriterFactory<T> implements WriterFactory<T> {
+  private final Table table;
+  private final FileFormat dataFileFormat;
+  private final Schema dataSchema;
+  private final SortOrder dataSortOrder;
+  private final FileFormat deleteFileFormat;
+  private final int[] equalityFieldIds;
+  private final Schema equalityDeleteRowSchema;
+  private final SortOrder equalityDeleteSortOrder;
+  private final Schema positionDeleteRowSchema;
+  private final SortOrder positionDeleteSortOrder;
+
+  protected BaseWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema,
+                              SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                              int[] equalityFieldIds, Schema equalityDeleteRowSchema,
+                              SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                              SortOrder positionDeleteSortOrder) {

Review comment:
       Agreed with @rdblue , we don't need the positionDeleteSortOrder,  this need to be removed in the next PR update.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677124642



##########
File path: data/src/main/java/org/apache/iceberg/data/BaseWriterFactory.java
##########
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriterFactory;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+/**
+ * A base writer factory to be extended by query engine integrations.
+ */
+public abstract class BaseWriterFactory<T> implements WriterFactory<T> {
+  private final Table table;
+  private final FileFormat dataFileFormat;
+  private final Schema dataSchema;
+  private final SortOrder dataSortOrder;
+  private final FileFormat deleteFileFormat;
+  private final int[] equalityFieldIds;
+  private final Schema equalityDeleteRowSchema;
+  private final SortOrder equalityDeleteSortOrder;
+  private final Schema positionDeleteRowSchema;
+  private final SortOrder positionDeleteSortOrder;
+
+  protected BaseWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema,

Review comment:
       What will be the common usage pattern of this method? The default of these parameters are coming from the table and if some configuration is set, then the values are overwritten? If so, maybe we would like to have a builder which helps with it?




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677930213



##########
File path: data/src/main/java/org/apache/iceberg/data/BaseWriterFactory.java
##########
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriterFactory;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+/**
+ * A base writer factory to be extended by query engine integrations.
+ */
+public abstract class BaseWriterFactory<T> implements WriterFactory<T> {
+  private final Table table;
+  private final FileFormat dataFileFormat;
+  private final Schema dataSchema;
+  private final SortOrder dataSortOrder;
+  private final FileFormat deleteFileFormat;
+  private final int[] equalityFieldIds;
+  private final Schema equalityDeleteRowSchema;
+  private final SortOrder equalityDeleteSortOrder;
+  private final Schema positionDeleteRowSchema;
+  private final SortOrder positionDeleteSortOrder;
+
+  protected BaseWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema,
+                              SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                              int[] equalityFieldIds, Schema equalityDeleteRowSchema,
+                              SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                              SortOrder positionDeleteSortOrder) {
+    this.table = table;

Review comment:
       Since `table` is a mutable instance and its lifecycle is binded to the catalog connection's lifecycle. Take hive catalog as an example,  If someone create a table like the following: 
   
   ```java
   Table table;
   try(Catalog catalog =  loadCatalog){
       table = catalog.loadTable(...);
   }
   ```
   
   And then we pass the `table` through the writer path,   what concern me is people may use this table instance to access the latest table state (for example `table.refresh()` to get the latest partition spec or SortOrder etc) for their purpose - which will lead to the interruption as the catalog catalog has been closed.   That's why we did not introduce pass an iceberg `Table` to the underlying writer path because it's hard to check whether the catalog connection has been closed or not.  So I recommend to just pass the detailed parameters that we need to construct the delete/data writers, such as  table properties, schema, partitionSpec, sortOrder etc, rather than providing a whole `Table` instance.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677926707



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {
+
+  /**
+   * Creates a new {@link DataWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written data belongs to

Review comment:
       Consider the following sequence of actions:
   1. create table (ts, category, data) partitioned by days(ts)
   2. day 1 adds some data files
   3. alter table add partition field category
   4. day 2 some data files
   5. delete some data where day=1 and data=xxx, in this case the delete writer should use the old partition spec. If writing using the latest spec, it would produce 1 delete file per category.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#issuecomment-890304519


   Thanks for reviewing, everyone! 


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677134339



##########
File path: data/src/main/java/org/apache/iceberg/data/BaseWriterFactory.java
##########
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriterFactory;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+/**
+ * A base writer factory to be extended by query engine integrations.
+ */
+public abstract class BaseWriterFactory<T> implements WriterFactory<T> {
+  private final Table table;
+  private final FileFormat dataFileFormat;
+  private final Schema dataSchema;
+  private final SortOrder dataSortOrder;
+  private final FileFormat deleteFileFormat;
+  private final int[] equalityFieldIds;
+  private final Schema equalityDeleteRowSchema;
+  private final SortOrder equalityDeleteSortOrder;
+  private final Schema positionDeleteRowSchema;
+  private final SortOrder positionDeleteSortOrder;
+
+  protected BaseWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema,

Review comment:
       Would this builder be very different for different implementations, or would it worth to have the common parts factored out for a basebuilder? Or we will have only a few implementations, and it does not worth the effort? 




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677615611



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {

Review comment:
       One difference there is that `FileAppenderFactory` accepts `FileFormat` while here I am saying that we actually know the format we want to use for data and deletes at the factory construction time. That's why I don't accept `FileFormat` as an argument for methods constructing writes.
   
   Is there a use case when a single txn will write both Avro and Parquet data files? We can use different file formats for data and delete files. However, do we plan to do that for the same type of files within a txn?




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r679465767



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {
+
+  /**
+   * Creates a new {@link DataWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written data belongs to

Review comment:
       @jackye1995's example is correct. The reason why we need to project `_spec` and `_partition` for all rows is that we may need to write using multiple specs within a txn. In the considered example, we need to use the old spec for deleting records in DAY 1 and we may need to use the current spec for deleting records in DAY 2 (all records for DAY 2 are actually written using the new spec).




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r680322132



##########
File path: data/src/main/java/org/apache/iceberg/data/BaseWriterFactory.java
##########
@@ -0,0 +1,257 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriterFactory;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+/**
+ * A base writer factory to be extended by query engine integrations.
+ */
+public abstract class BaseWriterFactory<T> implements WriterFactory<T> {
+  private final Table table;
+  private final FileFormat dataFileFormat;
+  private final Schema dataSchema;
+  private final SortOrder dataSortOrder;
+  private final FileFormat deleteFileFormat;
+  private final int[] equalityFieldIds;
+  private final Schema equalityDeleteRowSchema;
+  private final SortOrder equalityDeleteSortOrder;
+  private final Schema positionDeleteRowSchema;
+  private final SortOrder positionDeleteSortOrder;
+
+  protected BaseWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema,
+                              SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                              int[] equalityFieldIds, Schema equalityDeleteRowSchema,
+                              SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                              SortOrder positionDeleteSortOrder) {
+    this.table = table;
+    this.dataFileFormat = dataFileFormat;
+    this.dataSchema = dataSchema;
+    this.dataSortOrder = dataSortOrder;
+    this.deleteFileFormat = deleteFileFormat;
+    this.equalityFieldIds = equalityFieldIds;
+    this.equalityDeleteRowSchema = equalityDeleteRowSchema;
+    this.equalityDeleteSortOrder = equalityDeleteSortOrder;
+    this.positionDeleteRowSchema = positionDeleteRowSchema;
+    this.positionDeleteSortOrder = positionDeleteSortOrder;
+  }
+
+  protected abstract void configureDataWrite(Avro.DataWriteBuilder builder);
+  protected abstract void configureEqualityDelete(Avro.DeleteWriteBuilder builder);
+  protected abstract void configurePositionDelete(Avro.DeleteWriteBuilder builder);
+
+  protected abstract void configureDataWrite(Parquet.DataWriteBuilder builder);
+  protected abstract void configureEqualityDelete(Parquet.DeleteWriteBuilder builder);
+  protected abstract void configurePositionDelete(Parquet.DeleteWriteBuilder builder);
+
+  protected abstract void configureDataWrite(ORC.DataWriteBuilder builder);
+
+  @Override
+  public DataWriter<T> newDataWriter(EncryptedOutputFile file, PartitionSpec spec, StructLike partition) {
+    OutputFile outputFile = file.encryptingOutputFile();
+    EncryptionKeyMetadata keyMetadata = file.keyMetadata();
+    Map<String, String> properties = table.properties();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(properties);
+
+    try {
+      switch (dataFileFormat) {
+        case AVRO:
+          Avro.DataWriteBuilder avroBuilder = Avro.writeData(outputFile)
+              .schema(dataSchema)
+              .setAll(properties)
+              .metricsConfig(metricsConfig)
+              .withSpec(spec)
+              .withPartition(partition)
+              .withKeyMetadata(keyMetadata)
+              .withSortOrder(dataSortOrder)
+              .overwrite();
+
+          configureDataWrite(avroBuilder);
+
+          return avroBuilder.build();
+
+        case PARQUET:
+          Parquet.DataWriteBuilder parquetBuilder = Parquet.writeData(outputFile)
+              .schema(dataSchema)
+              .setAll(properties)
+              .metricsConfig(metricsConfig)
+              .withSpec(spec)
+              .withPartition(partition)
+              .withKeyMetadata(keyMetadata)
+              .withSortOrder(dataSortOrder)
+              .overwrite();
+
+          configureDataWrite(parquetBuilder);
+
+          return parquetBuilder.build();
+
+        case ORC:
+          ORC.DataWriteBuilder orcBuilder = ORC.writeData(outputFile)
+              .schema(dataSchema)
+              .setAll(properties)
+              .metricsConfig(metricsConfig)
+              .withSpec(spec)
+              .withPartition(partition)
+              .withKeyMetadata(keyMetadata)
+              .withSortOrder(dataSortOrder)
+              .overwrite();
+
+          configureDataWrite(orcBuilder);
+
+          return orcBuilder.build();
+
+        default:
+          throw new UnsupportedOperationException("Unsupported data file format: " + dataFileFormat);
+      }
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public EqualityDeleteWriter<T> newEqualityDeleteWriter(EncryptedOutputFile file, PartitionSpec spec,
+                                                         StructLike partition) {
+    OutputFile outputFile = file.encryptingOutputFile();
+    EncryptionKeyMetadata keyMetadata = file.keyMetadata();
+    Map<String, String> properties = table.properties();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(properties);
+
+    try {
+      switch (deleteFileFormat) {
+        case AVRO:
+          // TODO: support metrics configs in Avro equality delete writer
+
+          Avro.DeleteWriteBuilder avroBuilder = Avro.writeDeletes(outputFile)
+              .setAll(properties)
+              .rowSchema(equalityDeleteRowSchema)
+              .equalityFieldIds(equalityFieldIds)
+              .withSpec(spec)
+              .withPartition(partition)
+              .withKeyMetadata(keyMetadata)
+              .withSortOrder(equalityDeleteSortOrder)
+              .overwrite();
+
+          configureEqualityDelete(avroBuilder);
+
+          return avroBuilder.buildEqualityWriter();
+
+        case PARQUET:
+          Parquet.DeleteWriteBuilder parquetBuilder = Parquet.writeDeletes(outputFile)
+              .setAll(properties)
+              .metricsConfig(metricsConfig)
+              .rowSchema(equalityDeleteRowSchema)
+              .equalityFieldIds(equalityFieldIds)
+              .withSpec(spec)
+              .withPartition(partition)
+              .withKeyMetadata(keyMetadata)
+              .withSortOrder(equalityDeleteSortOrder)
+              .overwrite();
+
+          configureEqualityDelete(parquetBuilder);
+
+          return parquetBuilder.buildEqualityWriter();
+
+        default:
+          throw new UnsupportedOperationException("Unsupported format for equality deletes: " + deleteFileFormat);

Review comment:
       I'd say we better be explicit here. This will be validated early enough and I think it is safer to rely on the user to pick the format for deletes.
   
   That being said, I expect we will implement the ORC support soon. Should be just temporary.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677229843



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {
+
+  /**
+   * Creates a new {@link DataWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written data belongs to
+   * @param partition the partition written data belongs to or null if the spec is unpartitioned
+   * @return the constructed data writer
+   */
+  DataWriter<T> newDataWriter(EncryptedOutputFile file, PartitionSpec spec, StructLike partition);
+
+  /**
+   * Creates a new {@link EqualityDeleteWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written deletes belong to
+   * @param partition the partition written deletes belong to or null if the spec is unpartitioned

Review comment:
       Allowing to set the partition data to be null helps to abstract an uniformed [file-size-rolling data/delete writers](https://github.com/apache/iceberg/blob/63392d95848b21b32e6619f500e87c5342824f08/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java#L302) in the  BaseTaskWriter,  I mean in this way we don't have to design different writers for unpartitioned table and partitioned tables, that really reduces lots of writer classes.  I think we'd better to follow the existing approach.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677913738



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {

Review comment:
       > That's why I don't accept FileFormat as an argument for methods constructing writes.
   
   Yes,  we don't need the customized `FileFormat` to construct the data/delete file writers.
   
   > Is there a use case when a single txn will write both Avro and Parquet data files?
   
   For the current compute-engine + iceberg integration work,  I don't think we support writing both avro and parquet data files in the same txn.  If people really want to do this,  they will need to generate their own avro and parquet files, and finally commit those files by using Iceberg Table API. I think that's the correct practice.
   
   > We can use different file formats for data and delete files.
   
   Yes, that's really helpful.   I remember that the JOIN process between data files and positional delete files is a MERGE-SORT JOIN  because the both data files and delete files have sorted joined keys : <file_name, row_offset>.   If we have `N` positional delete files in the JOIN progress,  then we will allocate at least `N*128MB` memory for each row group (To read the whole row) from the positional delete files (If we use parquet),  that's quite huge memory consuming, which will lead to OutOfMemory in the real production.  In this case, the idea  file format for positional delete files is `AVRO` I think.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r680273670



##########
File path: data/src/test/java/org/apache/iceberg/io/TestWriterFactory.java
##########
@@ -0,0 +1,448 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.TableTestBase;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.IcebergGenerics;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataReader;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.StructLikeSet;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.MetadataColumns.DELETE_FILE_PATH;
+import static org.apache.iceberg.MetadataColumns.DELETE_FILE_POS;
+import static org.apache.iceberg.MetadataColumns.DELETE_FILE_ROW_FIELD_NAME;
+
+@RunWith(Parameterized.class)
+public abstract class TestWriterFactory<T> extends TableTestBase {
+  @Parameterized.Parameters(name = "FileFormat={0}, Partitioned={1}")
+  public static Object[] parameters() {
+    return new Object[][] {
+        new Object[]{FileFormat.AVRO, false},
+        new Object[]{FileFormat.AVRO, true},
+        new Object[]{FileFormat.PARQUET, false},
+        new Object[]{FileFormat.PARQUET, true},
+        new Object[]{FileFormat.ORC, false},
+        new Object[]{FileFormat.ORC, true}
+    };
+  }
+
+  private static final int TABLE_FORMAT_VERSION = 2;
+
+  private final FileFormat fileFormat;
+  private final boolean partitioned;
+  private final List<T> dataRows;
+
+  private StructLike partition = null;
+  private OutputFileFactory fileFactory = null;
+
+  public TestWriterFactory(FileFormat fileFormat, boolean partitioned) {
+    super(TABLE_FORMAT_VERSION);
+    this.fileFormat = fileFormat;
+    this.partitioned = partitioned;
+    this.dataRows = ImmutableList.of(
+        toRow(1, "aaa"),
+        toRow(2, "aaa"),
+        toRow(3, "aaa"),
+        toRow(4, "aaa"),
+        toRow(5, "aaa")
+    );
+  }
+
+  protected abstract WriterFactory<T> newWriterFactory(Schema dataSchema, List<Integer> equalityFieldIds,
+                                                       Schema equalityDeleteRowSchema, Schema positionDeleteRowSchema);
+
+  protected abstract T toRow(Integer id, String data);
+
+  protected abstract StructLikeSet toSet(Iterable<T> records);
+
+  protected FileFormat format() {
+    return fileFormat;
+  }
+
+  @Before
+  public void setupTable() throws Exception {
+    this.tableDir = temp.newFolder();
+    Assert.assertTrue(tableDir.delete()); // created during table creation
+
+    this.metadataDir = new File(tableDir, "metadata");
+
+    if (partitioned) {
+      this.table = create(SCHEMA, SPEC);
+      this.partition = initPartitionKey();
+    } else {
+      this.table = create(SCHEMA, PartitionSpec.unpartitioned());
+      this.partition = null;
+    }
+
+    this.fileFactory = OutputFileFactory.builderFor(table, 1, 1).format(fileFormat).build();
+  }
+
+  @Test
+  public void testDataWriter() throws IOException {
+    WriterFactory<T> writerFactory = newWriterFactory(table.schema());
+
+    DataFile dataFile = writeData(writerFactory, dataRows, table.spec(), partition);
+
+    table.newRowDelta()
+        .addRows(dataFile)
+        .commit();
+
+    Assert.assertEquals("Records should match", toSet(dataRows), actualRowSet("*"));
+  }
+
+  @Test
+  public void testEqualityDeleteWriter() throws IOException {
+    Assume.assumeFalse("ORC delete files are not supported", fileFormat == FileFormat.ORC);
+
+    List<Integer> equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId());
+    Schema equalityDeleteRowSchema = table.schema().select("id");
+    WriterFactory<T> writerFactory = newWriterFactory(table.schema(), equalityFieldIds, equalityDeleteRowSchema);
+
+    // write a data file
+    DataFile dataFile = writeData(writerFactory, dataRows, table.spec(), partition);
+
+    // commit the written data file
+    table.newRowDelta()
+        .addRows(dataFile)
+        .commit();
+
+    // write an equality delete file
+    List<T> deletes = ImmutableList.of(
+        toRow(1, "aaa"),
+        toRow(3, "bbb"),
+        toRow(5, "ccc")
+    );
+    DeleteFile deleteFile = writeEqualityDeletes(writerFactory, deletes, table.spec(), partition);
+
+    // verify the written delete file
+    GenericRecord deleteRecord = GenericRecord.create(equalityDeleteRowSchema);
+    List<Record> expectedDeletes = ImmutableList.of(
+        deleteRecord.copy("id", 1),
+        deleteRecord.copy("id", 3),
+        deleteRecord.copy("id", 5)
+    );
+    InputFile inputDeleteFile = table.io().newInputFile(deleteFile.path().toString());
+    List<Record> actualDeletes = readFile(equalityDeleteRowSchema, inputDeleteFile);
+    Assert.assertEquals("Delete records must match", expectedDeletes, actualDeletes);
+
+    // commit the written delete file
+    table.newRowDelta()
+        .addDeletes(deleteFile)
+        .commit();
+
+    // verify the delete file is applied correctly
+    List<T> expectedRows = ImmutableList.of(
+        toRow(2, "aaa"),
+        toRow(4, "aaa")
+    );
+    Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*"));
+  }
+
+  @Test
+  public void testEqualityDeleteWriterWithMultipleSpecs() throws IOException {
+    Assume.assumeFalse("ORC delete files are not supported", fileFormat == FileFormat.ORC);
+    Assume.assumeFalse("Table must start unpartitioned", partitioned);
+
+    List<Integer> equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId());
+    Schema equalityDeleteRowSchema = table.schema().select("id");
+    WriterFactory<T> writerFactory = newWriterFactory(table.schema(), equalityFieldIds, equalityDeleteRowSchema);
+
+    // write an unpartitioned data file
+    DataFile firstDataFile = writeData(writerFactory, dataRows, table.spec(), partition);
+    Assert.assertEquals("First data file must be unpartitioned", 0, firstDataFile.partition().size());
+
+    List<T> deletes = ImmutableList.of(
+        toRow(1, "aaa"),
+        toRow(2, "aaa"),
+        toRow(3, "aaa"),
+        toRow(4, "aaa")
+    );
+
+    // write an unpartitioned delete file
+    DeleteFile firstDeleteFile = writeEqualityDeletes(writerFactory, deletes, table.spec(), partition);
+    Assert.assertEquals("First delete file must be unpartitioned", 0, firstDeleteFile.partition().size());
+
+    // commit the first data and delete files
+    table.newAppend()
+        .appendFile(firstDataFile)
+        .commit();
+    table.newRowDelta()
+        .addDeletes(firstDeleteFile)
+        .commit();
+
+    // evolve the spec
+    table.updateSpec()
+        .addField("data")
+        .commit();
+
+    partition = initPartitionKey();
+
+    // write a partitioned data file
+    DataFile secondDataFile = writeData(writerFactory, dataRows, table.spec(), partition);
+    Assert.assertEquals("Second data file must be partitioned", 1, secondDataFile.partition().size());
+
+    // write a partitioned delete file
+    DeleteFile secondDeleteFile = writeEqualityDeletes(writerFactory, deletes, table.spec(), partition);
+    Assert.assertEquals("Second delete file must be artitioned", 1, secondDeleteFile.partition().size());
+
+    // commit the second data and delete files
+    table.newAppend()
+        .appendFile(secondDataFile)
+        .commit();
+    table.newRowDelta()
+        .addDeletes(secondDeleteFile)
+        .commit();
+
+    // verify both delete files are applied correctly
+    List<T> expectedRows = ImmutableList.of(
+        toRow(5, "aaa"),
+        toRow(5, "aaa")
+    );
+    Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*"));
+  }
+
+  @Test
+  public void testPositionDeleteWriter() throws IOException {
+    Assume.assumeFalse("ORC delete files are not supported", fileFormat == FileFormat.ORC);
+
+    WriterFactory<T> writerFactory = newWriterFactory(table.schema());
+
+    // write a data file
+    DataFile dataFile = writeData(writerFactory, dataRows, table.spec(), partition);
+
+    // write a position delete file
+    List<PositionDelete<T>> deletes = ImmutableList.of(
+        new PositionDelete<T>().set(dataFile.path(), 0L, null),
+        new PositionDelete<T>().set(dataFile.path(), 2L, null),
+        new PositionDelete<T>().set(dataFile.path(), 4L, null)
+    );
+    Pair<DeleteFile, CharSequenceSet> result = writePositionDeletes(writerFactory, deletes, table.spec(), partition);
+    DeleteFile deleteFile = result.first();
+    CharSequenceSet referencedDataFiles = result.second();
+
+    // verify the written delete file
+    GenericRecord deleteRecord = GenericRecord.create(DeleteSchemaUtil.pathPosSchema());
+    List<Record> expectedDeletes = ImmutableList.of(
+        deleteRecord.copy(DELETE_FILE_PATH.name(), dataFile.path(), DELETE_FILE_POS.name(), 0L),
+        deleteRecord.copy(DELETE_FILE_PATH.name(), dataFile.path(), DELETE_FILE_POS.name(), 2L),
+        deleteRecord.copy(DELETE_FILE_PATH.name(), dataFile.path(), DELETE_FILE_POS.name(), 4L)
+    );
+    InputFile inputDeleteFile = table.io().newInputFile(deleteFile.path().toString());
+    List<Record> actualDeletes = readFile(DeleteSchemaUtil.pathPosSchema(), inputDeleteFile);
+    Assert.assertEquals("Delete records must match", expectedDeletes, actualDeletes);
+
+    // commit the data and delete files
+    table.newRowDelta()
+        .addRows(dataFile)
+        .addDeletes(deleteFile)
+        .validateDataFilesExist(referencedDataFiles)
+        .validateDeletedFiles()
+        .commit();
+
+    // verify the delete file is applied correctly
+    List<T> expectedRows = ImmutableList.of(
+        toRow(2, "aaa"),
+        toRow(4, "aaa")
+    );
+    Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*"));
+  }
+
+  @Test
+  public void testPositionDeleteWriterWithRow() throws IOException {
+    Assume.assumeFalse("ORC delete files are not supported", fileFormat == FileFormat.ORC);

Review comment:
       No blockers that I know about. Just that I didn't have the time to update ORC for the feature when I added it.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r680322166



##########
File path: data/src/main/java/org/apache/iceberg/data/BaseWriterFactory.java
##########
@@ -0,0 +1,257 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriterFactory;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+/**
+ * A base writer factory to be extended by query engine integrations.
+ */
+public abstract class BaseWriterFactory<T> implements WriterFactory<T> {
+  private final Table table;
+  private final FileFormat dataFileFormat;
+  private final Schema dataSchema;
+  private final SortOrder dataSortOrder;
+  private final FileFormat deleteFileFormat;
+  private final int[] equalityFieldIds;
+  private final Schema equalityDeleteRowSchema;
+  private final SortOrder equalityDeleteSortOrder;
+  private final Schema positionDeleteRowSchema;
+  private final SortOrder positionDeleteSortOrder;
+
+  protected BaseWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema,
+                              SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                              int[] equalityFieldIds, Schema equalityDeleteRowSchema,
+                              SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                              SortOrder positionDeleteSortOrder) {
+    this.table = table;
+    this.dataFileFormat = dataFileFormat;
+    this.dataSchema = dataSchema;
+    this.dataSortOrder = dataSortOrder;
+    this.deleteFileFormat = deleteFileFormat;
+    this.equalityFieldIds = equalityFieldIds;
+    this.equalityDeleteRowSchema = equalityDeleteRowSchema;
+    this.equalityDeleteSortOrder = equalityDeleteSortOrder;
+    this.positionDeleteRowSchema = positionDeleteRowSchema;
+    this.positionDeleteSortOrder = positionDeleteSortOrder;
+  }
+
+  protected abstract void configureDataWrite(Avro.DataWriteBuilder builder);
+  protected abstract void configureEqualityDelete(Avro.DeleteWriteBuilder builder);
+  protected abstract void configurePositionDelete(Avro.DeleteWriteBuilder builder);
+
+  protected abstract void configureDataWrite(Parquet.DataWriteBuilder builder);
+  protected abstract void configureEqualityDelete(Parquet.DeleteWriteBuilder builder);
+  protected abstract void configurePositionDelete(Parquet.DeleteWriteBuilder builder);
+
+  protected abstract void configureDataWrite(ORC.DataWriteBuilder builder);
+
+  @Override
+  public DataWriter<T> newDataWriter(EncryptedOutputFile file, PartitionSpec spec, StructLike partition) {
+    OutputFile outputFile = file.encryptingOutputFile();
+    EncryptionKeyMetadata keyMetadata = file.keyMetadata();
+    Map<String, String> properties = table.properties();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(properties);
+
+    try {
+      switch (dataFileFormat) {
+        case AVRO:
+          Avro.DataWriteBuilder avroBuilder = Avro.writeData(outputFile)
+              .schema(dataSchema)
+              .setAll(properties)
+              .metricsConfig(metricsConfig)
+              .withSpec(spec)
+              .withPartition(partition)
+              .withKeyMetadata(keyMetadata)
+              .withSortOrder(dataSortOrder)
+              .overwrite();
+
+          configureDataWrite(avroBuilder);
+
+          return avroBuilder.build();
+
+        case PARQUET:
+          Parquet.DataWriteBuilder parquetBuilder = Parquet.writeData(outputFile)
+              .schema(dataSchema)
+              .setAll(properties)
+              .metricsConfig(metricsConfig)
+              .withSpec(spec)
+              .withPartition(partition)
+              .withKeyMetadata(keyMetadata)
+              .withSortOrder(dataSortOrder)
+              .overwrite();
+
+          configureDataWrite(parquetBuilder);
+
+          return parquetBuilder.build();
+
+        case ORC:
+          ORC.DataWriteBuilder orcBuilder = ORC.writeData(outputFile)
+              .schema(dataSchema)
+              .setAll(properties)
+              .metricsConfig(metricsConfig)
+              .withSpec(spec)
+              .withPartition(partition)
+              .withKeyMetadata(keyMetadata)
+              .withSortOrder(dataSortOrder)
+              .overwrite();
+
+          configureDataWrite(orcBuilder);
+
+          return orcBuilder.build();
+
+        default:
+          throw new UnsupportedOperationException("Unsupported data file format: " + dataFileFormat);
+      }
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public EqualityDeleteWriter<T> newEqualityDeleteWriter(EncryptedOutputFile file, PartitionSpec spec,
+                                                         StructLike partition) {
+    OutputFile outputFile = file.encryptingOutputFile();
+    EncryptionKeyMetadata keyMetadata = file.keyMetadata();
+    Map<String, String> properties = table.properties();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(properties);
+
+    try {
+      switch (deleteFileFormat) {
+        case AVRO:
+          // TODO: support metrics configs in Avro equality delete writer
+
+          Avro.DeleteWriteBuilder avroBuilder = Avro.writeDeletes(outputFile)
+              .setAll(properties)
+              .rowSchema(equalityDeleteRowSchema)
+              .equalityFieldIds(equalityFieldIds)
+              .withSpec(spec)
+              .withPartition(partition)
+              .withKeyMetadata(keyMetadata)
+              .withSortOrder(equalityDeleteSortOrder)
+              .overwrite();
+
+          configureEqualityDelete(avroBuilder);
+
+          return avroBuilder.buildEqualityWriter();
+
+        case PARQUET:
+          Parquet.DeleteWriteBuilder parquetBuilder = Parquet.writeDeletes(outputFile)
+              .setAll(properties)
+              .metricsConfig(metricsConfig)
+              .rowSchema(equalityDeleteRowSchema)
+              .equalityFieldIds(equalityFieldIds)
+              .withSpec(spec)
+              .withPartition(partition)
+              .withKeyMetadata(keyMetadata)
+              .withSortOrder(equalityDeleteSortOrder)
+              .overwrite();
+
+          configureEqualityDelete(parquetBuilder);
+
+          return parquetBuilder.buildEqualityWriter();
+
+        default:
+          throw new UnsupportedOperationException("Unsupported format for equality deletes: " + deleteFileFormat);
+      }
+    } catch (IOException e) {
+      throw new UncheckedIOException("Failed to create new equality delete writer", e);
+    }
+  }
+
+  @Override
+  public PositionDeleteWriter<T> newPositionDeleteWriter(EncryptedOutputFile file, PartitionSpec spec,
+                                                         StructLike partition) {
+    OutputFile outputFile = file.encryptingOutputFile();
+    EncryptionKeyMetadata keyMetadata = file.keyMetadata();
+    Map<String, String> properties = table.properties();
+
+    // TODO: build and pass a correct metrics config for position deletes

Review comment:
       Yeah, we haven been coming back to this over and over again.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677132092



##########
File path: data/src/test/java/org/apache/iceberg/io/TestWriterFactory.java
##########
@@ -0,0 +1,448 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.TableTestBase;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.IcebergGenerics;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataReader;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.StructLikeSet;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.MetadataColumns.DELETE_FILE_PATH;
+import static org.apache.iceberg.MetadataColumns.DELETE_FILE_POS;
+import static org.apache.iceberg.MetadataColumns.DELETE_FILE_ROW_FIELD_NAME;
+
+@RunWith(Parameterized.class)
+public abstract class TestWriterFactory<T> extends TableTestBase {
+  @Parameterized.Parameters(name = "FileFormat={0}, Partitioned={1}")
+  public static Object[] parameters() {
+    return new Object[][] {
+        new Object[]{FileFormat.AVRO, false},
+        new Object[]{FileFormat.AVRO, true},
+        new Object[]{FileFormat.PARQUET, false},
+        new Object[]{FileFormat.PARQUET, true},
+        new Object[]{FileFormat.ORC, false},
+        new Object[]{FileFormat.ORC, true}
+    };
+  }
+
+  private static final int TABLE_FORMAT_VERSION = 2;
+
+  private final FileFormat fileFormat;
+  private final boolean partitioned;
+  private final List<T> dataRows;
+
+  private StructLike partition = null;
+  private OutputFileFactory fileFactory = null;
+
+  public TestWriterFactory(FileFormat fileFormat, boolean partitioned) {
+    super(TABLE_FORMAT_VERSION);
+    this.fileFormat = fileFormat;
+    this.partitioned = partitioned;
+    this.dataRows = ImmutableList.of(
+        toRow(1, "aaa"),
+        toRow(2, "aaa"),
+        toRow(3, "aaa"),
+        toRow(4, "aaa"),
+        toRow(5, "aaa")
+    );
+  }
+
+  protected abstract WriterFactory<T> newWriterFactory(Schema dataSchema, List<Integer> equalityFieldIds,
+                                                       Schema equalityDeleteRowSchema, Schema positionDeleteRowSchema);
+
+  protected abstract T toRow(Integer id, String data);
+
+  protected abstract StructLikeSet toSet(Iterable<T> records);
+
+  protected FileFormat format() {
+    return fileFormat;
+  }
+
+  @Before
+  public void setupTable() throws Exception {
+    this.tableDir = temp.newFolder();
+    Assert.assertTrue(tableDir.delete()); // created during table creation
+
+    this.metadataDir = new File(tableDir, "metadata");
+
+    if (partitioned) {
+      this.table = create(SCHEMA, SPEC);
+      this.partition = initPartitionKey();
+    } else {
+      this.table = create(SCHEMA, PartitionSpec.unpartitioned());
+      this.partition = null;
+    }
+
+    this.fileFactory = OutputFileFactory.builderFor(table, 1, 1).format(fileFormat).build();
+  }
+
+  @Test
+  public void testDataWriter() throws IOException {
+    WriterFactory<T> writerFactory = newWriterFactory(table.schema());
+
+    DataFile dataFile = writeData(writerFactory, dataRows, table.spec(), partition);
+
+    table.newRowDelta()
+        .addRows(dataFile)
+        .commit();
+
+    Assert.assertEquals("Records should match", toSet(dataRows), actualRowSet("*"));
+  }
+
+  @Test
+  public void testEqualityDeleteWriter() throws IOException {
+    Assume.assumeFalse("ORC delete files are not supported", fileFormat == FileFormat.ORC);
+
+    List<Integer> equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId());
+    Schema equalityDeleteRowSchema = table.schema().select("id");
+    WriterFactory<T> writerFactory = newWriterFactory(table.schema(), equalityFieldIds, equalityDeleteRowSchema);
+
+    // write a data file
+    DataFile dataFile = writeData(writerFactory, dataRows, table.spec(), partition);
+
+    // commit the written data file
+    table.newRowDelta()
+        .addRows(dataFile)
+        .commit();
+
+    // write an equality delete file
+    List<T> deletes = ImmutableList.of(
+        toRow(1, "aaa"),
+        toRow(3, "bbb"),
+        toRow(5, "ccc")
+    );
+    DeleteFile deleteFile = writeEqualityDeletes(writerFactory, deletes, table.spec(), partition);
+
+    // verify the written delete file
+    GenericRecord deleteRecord = GenericRecord.create(equalityDeleteRowSchema);
+    List<Record> expectedDeletes = ImmutableList.of(
+        deleteRecord.copy("id", 1),
+        deleteRecord.copy("id", 3),
+        deleteRecord.copy("id", 5)
+    );
+    InputFile inputDeleteFile = table.io().newInputFile(deleteFile.path().toString());
+    List<Record> actualDeletes = readFile(equalityDeleteRowSchema, inputDeleteFile);
+    Assert.assertEquals("Delete records must match", expectedDeletes, actualDeletes);
+
+    // commit the written delete file
+    table.newRowDelta()
+        .addDeletes(deleteFile)
+        .commit();
+
+    // verify the delete file is applied correctly
+    List<T> expectedRows = ImmutableList.of(
+        toRow(2, "aaa"),
+        toRow(4, "aaa")
+    );
+    Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*"));
+  }
+
+  @Test
+  public void testEqualityDeleteWriterWithMultipleSpecs() throws IOException {
+    Assume.assumeFalse("ORC delete files are not supported", fileFormat == FileFormat.ORC);
+    Assume.assumeFalse("Table must start unpartitioned", partitioned);
+
+    List<Integer> equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId());
+    Schema equalityDeleteRowSchema = table.schema().select("id");
+    WriterFactory<T> writerFactory = newWriterFactory(table.schema(), equalityFieldIds, equalityDeleteRowSchema);
+
+    // write an unpartitioned data file
+    DataFile firstDataFile = writeData(writerFactory, dataRows, table.spec(), partition);
+    Assert.assertEquals("First data file must be unpartitioned", 0, firstDataFile.partition().size());
+
+    List<T> deletes = ImmutableList.of(
+        toRow(1, "aaa"),
+        toRow(2, "aaa"),
+        toRow(3, "aaa"),
+        toRow(4, "aaa")
+    );
+
+    // write an unpartitioned delete file
+    DeleteFile firstDeleteFile = writeEqualityDeletes(writerFactory, deletes, table.spec(), partition);
+    Assert.assertEquals("First delete file must be unpartitioned", 0, firstDeleteFile.partition().size());
+
+    // commit the first data and delete files
+    table.newAppend()
+        .appendFile(firstDataFile)
+        .commit();
+    table.newRowDelta()
+        .addDeletes(firstDeleteFile)
+        .commit();
+
+    // evolve the spec
+    table.updateSpec()
+        .addField("data")
+        .commit();
+
+    partition = initPartitionKey();
+
+    // write a partitioned data file
+    DataFile secondDataFile = writeData(writerFactory, dataRows, table.spec(), partition);
+    Assert.assertEquals("Second data file must be partitioned", 1, secondDataFile.partition().size());
+
+    // write a partitioned delete file
+    DeleteFile secondDeleteFile = writeEqualityDeletes(writerFactory, deletes, table.spec(), partition);
+    Assert.assertEquals("Second delete file must be artitioned", 1, secondDeleteFile.partition().size());
+
+    // commit the second data and delete files
+    table.newAppend()
+        .appendFile(secondDataFile)
+        .commit();
+    table.newRowDelta()
+        .addDeletes(secondDeleteFile)
+        .commit();
+
+    // verify both delete files are applied correctly
+    List<T> expectedRows = ImmutableList.of(
+        toRow(5, "aaa"),
+        toRow(5, "aaa")
+    );
+    Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*"));
+  }
+
+  @Test
+  public void testPositionDeleteWriter() throws IOException {
+    Assume.assumeFalse("ORC delete files are not supported", fileFormat == FileFormat.ORC);
+
+    WriterFactory<T> writerFactory = newWriterFactory(table.schema());
+
+    // write a data file
+    DataFile dataFile = writeData(writerFactory, dataRows, table.spec(), partition);
+
+    // write a position delete file
+    List<PositionDelete<T>> deletes = ImmutableList.of(
+        new PositionDelete<T>().set(dataFile.path(), 0L, null),
+        new PositionDelete<T>().set(dataFile.path(), 2L, null),
+        new PositionDelete<T>().set(dataFile.path(), 4L, null)
+    );
+    Pair<DeleteFile, CharSequenceSet> result = writePositionDeletes(writerFactory, deletes, table.spec(), partition);
+    DeleteFile deleteFile = result.first();
+    CharSequenceSet referencedDataFiles = result.second();
+
+    // verify the written delete file
+    GenericRecord deleteRecord = GenericRecord.create(DeleteSchemaUtil.pathPosSchema());
+    List<Record> expectedDeletes = ImmutableList.of(
+        deleteRecord.copy(DELETE_FILE_PATH.name(), dataFile.path(), DELETE_FILE_POS.name(), 0L),
+        deleteRecord.copy(DELETE_FILE_PATH.name(), dataFile.path(), DELETE_FILE_POS.name(), 2L),
+        deleteRecord.copy(DELETE_FILE_PATH.name(), dataFile.path(), DELETE_FILE_POS.name(), 4L)
+    );
+    InputFile inputDeleteFile = table.io().newInputFile(deleteFile.path().toString());
+    List<Record> actualDeletes = readFile(DeleteSchemaUtil.pathPosSchema(), inputDeleteFile);
+    Assert.assertEquals("Delete records must match", expectedDeletes, actualDeletes);
+
+    // commit the data and delete files
+    table.newRowDelta()
+        .addRows(dataFile)
+        .addDeletes(deleteFile)
+        .validateDataFilesExist(referencedDataFiles)
+        .validateDeletedFiles()
+        .commit();
+
+    // verify the delete file is applied correctly
+    List<T> expectedRows = ImmutableList.of(
+        toRow(2, "aaa"),
+        toRow(4, "aaa")
+    );
+    Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*"));
+  }
+
+  @Test
+  public void testPositionDeleteWriterWithRow() throws IOException {
+    Assume.assumeFalse("ORC delete files are not supported", fileFormat == FileFormat.ORC);

Review comment:
       Do we know why ORC deletes are left out? Any serious blockers, or just there were no interest yet to implement them?
   Thanks, Peter 




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r680271115



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {
+
+  /**
+   * Creates a new {@link DataWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written data belongs to

Review comment:
       One more thing: the deletes can also be added to the unpartitioned spec for global deletes. That avoids the need to find all of the categories that exist in this example.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677130061



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {
+
+  /**
+   * Creates a new {@link DataWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written data belongs to
+   * @param partition the partition written data belongs to or null if the spec is unpartitioned
+   * @return the constructed data writer
+   */
+  DataWriter<T> newDataWriter(EncryptedOutputFile file, PartitionSpec spec, StructLike partition);
+
+  /**
+   * Creates a new {@link EqualityDeleteWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written deletes belong to
+   * @param partition the partition written deletes belong to or null if the spec is unpartitioned

Review comment:
       Ideally, we would have an extra method without the spec and partition. However, we did not reserve any particular spec ID as unpartitioned. That's why we still need a correct spec even for unpartitioned writes as we need to store a correct spec ID in the metadata. We could offer a method without the partition struct but I am not sure how valuable that is.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r680264510



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {
+
+  /**
+   * Creates a new {@link DataWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written data belongs to

Review comment:
       Another thing to keep in mind is that when encoding deletes, we need to encode them for all of the partition specs that are in use in the table. In @jackye1995's example, I think the predicate needs to be added to `spec=1/day=1` and `spec=2/day=1/category=*` because deletes aren't scoped to just the current spec and there could be data in the new spec.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677025324



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {

Review comment:
       I am planning to use WriterFactory in subsequent PRs when adding new TaskWriters.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#issuecomment-889535397


   One open question is about position delete sort order [here](https://github.com/apache/iceberg/pull/2873#discussion_r677780139).


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677135677



##########
File path: data/src/main/java/org/apache/iceberg/data/BaseWriterFactory.java
##########
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriterFactory;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+/**
+ * A base writer factory to be extended by query engine integrations.
+ */
+public abstract class BaseWriterFactory<T> implements WriterFactory<T> {
+  private final Table table;
+  private final FileFormat dataFileFormat;
+  private final Schema dataSchema;
+  private final SortOrder dataSortOrder;
+  private final FileFormat deleteFileFormat;
+  private final int[] equalityFieldIds;
+  private final Schema equalityDeleteRowSchema;
+  private final SortOrder equalityDeleteSortOrder;
+  private final Schema positionDeleteRowSchema;
+  private final SortOrder positionDeleteSortOrder;
+
+  protected BaseWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema,

Review comment:
       I can try prototyping a common builder tomorrow.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677241975



##########
File path: data/src/main/java/org/apache/iceberg/data/BaseWriterFactory.java
##########
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriterFactory;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+/**
+ * A base writer factory to be extended by query engine integrations.
+ */
+public abstract class BaseWriterFactory<T> implements WriterFactory<T> {
+  private final Table table;
+  private final FileFormat dataFileFormat;
+  private final Schema dataSchema;
+  private final SortOrder dataSortOrder;
+  private final FileFormat deleteFileFormat;
+  private final int[] equalityFieldIds;
+  private final Schema equalityDeleteRowSchema;
+  private final SortOrder equalityDeleteSortOrder;
+  private final Schema positionDeleteRowSchema;
+  private final SortOrder positionDeleteSortOrder;
+
+  protected BaseWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema,
+                              SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                              int[] equalityFieldIds, Schema equalityDeleteRowSchema,
+                              SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                              SortOrder positionDeleteSortOrder) {
+    this.table = table;
+    this.dataFileFormat = dataFileFormat;
+    this.dataSchema = dataSchema;
+    this.dataSortOrder = dataSortOrder;
+    this.deleteFileFormat = deleteFileFormat;
+    this.equalityFieldIds = equalityFieldIds;
+    this.equalityDeleteRowSchema = equalityDeleteRowSchema;
+    this.equalityDeleteSortOrder = equalityDeleteSortOrder;
+    this.positionDeleteRowSchema = positionDeleteRowSchema;
+    this.positionDeleteSortOrder = positionDeleteSortOrder;
+  }
+
+  protected abstract void configureDataWrite(Avro.DataWriteBuilder builder);
+  protected abstract void configureEqualityDelete(Avro.DeleteWriteBuilder builder);
+  protected abstract void configurePositionDelete(Avro.DeleteWriteBuilder builder);
+
+  protected abstract void configureDataWrite(Parquet.DataWriteBuilder builder);
+  protected abstract void configureEqualityDelete(Parquet.DeleteWriteBuilder builder);
+  protected abstract void configurePositionDelete(Parquet.DeleteWriteBuilder builder);
+
+  protected abstract void configureDataWrite(ORC.DataWriteBuilder builder);
+
+  @Override
+  public DataWriter<T> newDataWriter(EncryptedOutputFile file, PartitionSpec spec, StructLike partition) {
+    OutputFile outputFile = file.encryptingOutputFile();
+    EncryptionKeyMetadata keyMetadata = file.keyMetadata();
+    Map<String, String> properties = table.properties();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(properties);
+
+    try {
+      switch (dataFileFormat) {
+        case AVRO:
+          Avro.DataWriteBuilder avroBuilder = Avro.writeData(outputFile)
+              .schema(dataSchema)
+              .setAll(properties)
+              .metricsConfig(metricsConfig)
+              .withSpec(spec)
+              .withPartition(partition)
+              .withKeyMetadata(keyMetadata)
+              .withSortOrder(dataSortOrder)
+              .overwrite();
+
+          configureDataWrite(avroBuilder);
+
+          return avroBuilder.build();
+
+        case PARQUET:
+          Parquet.DataWriteBuilder parquetBuilder = Parquet.writeData(outputFile)
+              .schema(dataSchema)
+              .setAll(properties)
+              .metricsConfig(metricsConfig)
+              .withSpec(spec)
+              .withPartition(partition)
+              .withKeyMetadata(keyMetadata)
+              .withSortOrder(dataSortOrder)
+              .overwrite();
+
+          configureDataWrite(parquetBuilder);
+
+          return parquetBuilder.build();
+
+        case ORC:
+          ORC.DataWriteBuilder orcBuilder = ORC.writeData(outputFile)
+              .schema(dataSchema)
+              .setAll(properties)
+              .metricsConfig(metricsConfig)
+              .withSpec(spec)
+              .withPartition(partition)
+              .withKeyMetadata(keyMetadata)
+              .withSortOrder(dataSortOrder)
+              .overwrite();
+
+          configureDataWrite(orcBuilder);
+
+          return orcBuilder.build();
+
+        default:
+          throw new UnsupportedOperationException("Unsupported data file format: " + dataFileFormat);
+      }
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public EqualityDeleteWriter<T> newEqualityDeleteWriter(EncryptedOutputFile file, PartitionSpec spec,
+                                                         StructLike partition) {
+    OutputFile outputFile = file.encryptingOutputFile();
+    EncryptionKeyMetadata keyMetadata = file.keyMetadata();
+    Map<String, String> properties = table.properties();
+
+    // TODO: build and pass a correct metrics config for equality deletes

Review comment:
       To simplify the configuration keys, I think the equality delete writer could use the same metric configs as the data writer, because in theory the equality delete files has a column set which is actually a subset of the user defined table schema columns.
   
   But for positional delete writers,  it's a different user story. Because its columns are file_path and offset, the file_path shouldn't be truncated even if people set a `write.metadata.metrics.default=truncate(16)` because we should filter the data files to join as much as possible for efficient. 
   
   I'm OK to propose another separate PR to address the above issues for metric configs.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r679599620



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {
+
+  /**
+   * Creates a new {@link DataWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written data belongs to
+   * @param partition the partition written data belongs to or null if the spec is unpartitioned
+   * @return the constructed data writer
+   */
+  DataWriter<T> newDataWriter(EncryptedOutputFile file, PartitionSpec spec, StructLike partition);
+
+  /**
+   * Creates a new {@link EqualityDeleteWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written deletes belong to
+   * @param partition the partition written deletes belong to or null if the spec is unpartitioned
+   * @return the constructed equality delete writer
+   */
+  EqualityDeleteWriter<T> newEqualityDeleteWriter(EncryptedOutputFile file, PartitionSpec spec, StructLike partition);
+
+  /**
+   * Creates a new {@link PositionDeleteWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written deletes belong to
+   * @param partition the partition written deletes belong to or null if the spec is unpartitioned
+   * @return the constructed position delete writer
+   */
+  PositionDeleteWriter<T> newPositionDeleteWriter(EncryptedOutputFile file, PartitionSpec spec, StructLike partition);

Review comment:
       Okay,  let's follow the rule.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677627466



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {
+
+  /**
+   * Creates a new {@link DataWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written data belongs to

Review comment:
       There is actually a use case for that in Spark and I think it applies to other engines too. Imagine we have a table with multiple specs. In Spark, we plan to project `_spec` and `_partition` metadata columns. Whenever we a delete file, we have to use the partition spec the referenced data rows belong to.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677124642



##########
File path: data/src/main/java/org/apache/iceberg/data/BaseWriterFactory.java
##########
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriterFactory;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+/**
+ * A base writer factory to be extended by query engine integrations.
+ */
+public abstract class BaseWriterFactory<T> implements WriterFactory<T> {
+  private final Table table;
+  private final FileFormat dataFileFormat;
+  private final Schema dataSchema;
+  private final SortOrder dataSortOrder;
+  private final FileFormat deleteFileFormat;
+  private final int[] equalityFieldIds;
+  private final Schema equalityDeleteRowSchema;
+  private final SortOrder equalityDeleteSortOrder;
+  private final Schema positionDeleteRowSchema;
+  private final SortOrder positionDeleteSortOrder;
+
+  protected BaseWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema,

Review comment:
       What will be the common usage pattern of this method? The default of these parameters are coming from the table and if some configuration is set, then the values are overwritten? If so, maybe we would like to have a builder which helps with it?? 




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677792038



##########
File path: data/src/main/java/org/apache/iceberg/data/BaseWriterFactory.java
##########
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriterFactory;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+/**
+ * A base writer factory to be extended by query engine integrations.
+ */
+public abstract class BaseWriterFactory<T> implements WriterFactory<T> {
+  private final Table table;
+  private final FileFormat dataFileFormat;
+  private final Schema dataSchema;
+  private final SortOrder dataSortOrder;
+  private final FileFormat deleteFileFormat;
+  private final int[] equalityFieldIds;
+  private final Schema equalityDeleteRowSchema;
+  private final SortOrder equalityDeleteSortOrder;
+  private final Schema positionDeleteRowSchema;
+  private final SortOrder positionDeleteSortOrder;
+
+  protected BaseWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema,
+                              SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                              int[] equalityFieldIds, Schema equalityDeleteRowSchema,
+                              SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                              SortOrder positionDeleteSortOrder) {

Review comment:
       I think the spec says it is:
   
   > The rows in the delete file must be sorted by `file_path` then `position` to optimize filtering rows while scanning. 
   
   If not sorted, I believe that `Deletes.streamingFilter` algorithm would break:
   
   https://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/deletes/Deletes.java#L211-L219




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677935662



##########
File path: data/src/main/java/org/apache/iceberg/data/BaseWriterFactory.java
##########
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriterFactory;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+/**
+ * A base writer factory to be extended by query engine integrations.
+ */
+public abstract class BaseWriterFactory<T> implements WriterFactory<T> {
+  private final Table table;
+  private final FileFormat dataFileFormat;
+  private final Schema dataSchema;
+  private final SortOrder dataSortOrder;
+  private final FileFormat deleteFileFormat;
+  private final int[] equalityFieldIds;
+  private final Schema equalityDeleteRowSchema;
+  private final SortOrder equalityDeleteSortOrder;
+  private final Schema positionDeleteRowSchema;
+  private final SortOrder positionDeleteSortOrder;
+
+  protected BaseWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema,
+                              SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                              int[] equalityFieldIds, Schema equalityDeleteRowSchema,
+                              SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                              SortOrder positionDeleteSortOrder) {
+    this.table = table;

Review comment:
       I think the `Table` passed in here is supposed to be an instance of the `SerializableTable` and refreshing is not allowed. So the problem described should not exist. But maybe we should make this explicit, at least in the doc of the end factory builder.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r680287733



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {
+
+  /**
+   * Creates a new {@link DataWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written data belongs to
+   * @param partition the partition written data belongs to or null if the spec is unpartitioned
+   * @return the constructed data writer
+   */
+  DataWriter<T> newDataWriter(EncryptedOutputFile file, PartitionSpec spec, StructLike partition);
+
+  /**
+   * Creates a new {@link EqualityDeleteWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written deletes belong to
+   * @param partition the partition written deletes belong to or null if the spec is unpartitioned
+   * @return the constructed equality delete writer
+   */
+  EqualityDeleteWriter<T> newEqualityDeleteWriter(EncryptedOutputFile file, PartitionSpec spec, StructLike partition);
+
+  /**
+   * Creates a new {@link PositionDeleteWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written deletes belong to
+   * @param partition the partition written deletes belong to or null if the spec is unpartitioned
+   * @return the constructed position delete writer
+   */
+  PositionDeleteWriter<T> newPositionDeleteWriter(EncryptedOutputFile file, PartitionSpec spec, StructLike partition);

Review comment:
       Sounds good. Resolving this.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r680272203



##########
File path: data/src/main/java/org/apache/iceberg/data/BaseWriterFactory.java
##########
@@ -0,0 +1,257 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriterFactory;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+/**
+ * A base writer factory to be extended by query engine integrations.
+ */
+public abstract class BaseWriterFactory<T> implements WriterFactory<T> {
+  private final Table table;
+  private final FileFormat dataFileFormat;
+  private final Schema dataSchema;
+  private final SortOrder dataSortOrder;
+  private final FileFormat deleteFileFormat;
+  private final int[] equalityFieldIds;
+  private final Schema equalityDeleteRowSchema;
+  private final SortOrder equalityDeleteSortOrder;
+  private final Schema positionDeleteRowSchema;
+  private final SortOrder positionDeleteSortOrder;
+
+  protected BaseWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema,
+                              SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                              int[] equalityFieldIds, Schema equalityDeleteRowSchema,
+                              SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                              SortOrder positionDeleteSortOrder) {
+    this.table = table;
+    this.dataFileFormat = dataFileFormat;
+    this.dataSchema = dataSchema;
+    this.dataSortOrder = dataSortOrder;
+    this.deleteFileFormat = deleteFileFormat;
+    this.equalityFieldIds = equalityFieldIds;
+    this.equalityDeleteRowSchema = equalityDeleteRowSchema;
+    this.equalityDeleteSortOrder = equalityDeleteSortOrder;
+    this.positionDeleteRowSchema = positionDeleteRowSchema;
+    this.positionDeleteSortOrder = positionDeleteSortOrder;
+  }
+
+  protected abstract void configureDataWrite(Avro.DataWriteBuilder builder);
+  protected abstract void configureEqualityDelete(Avro.DeleteWriteBuilder builder);
+  protected abstract void configurePositionDelete(Avro.DeleteWriteBuilder builder);
+
+  protected abstract void configureDataWrite(Parquet.DataWriteBuilder builder);
+  protected abstract void configureEqualityDelete(Parquet.DeleteWriteBuilder builder);
+  protected abstract void configurePositionDelete(Parquet.DeleteWriteBuilder builder);
+
+  protected abstract void configureDataWrite(ORC.DataWriteBuilder builder);
+
+  @Override
+  public DataWriter<T> newDataWriter(EncryptedOutputFile file, PartitionSpec spec, StructLike partition) {
+    OutputFile outputFile = file.encryptingOutputFile();
+    EncryptionKeyMetadata keyMetadata = file.keyMetadata();
+    Map<String, String> properties = table.properties();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(properties);
+
+    try {
+      switch (dataFileFormat) {
+        case AVRO:
+          Avro.DataWriteBuilder avroBuilder = Avro.writeData(outputFile)
+              .schema(dataSchema)
+              .setAll(properties)
+              .metricsConfig(metricsConfig)
+              .withSpec(spec)
+              .withPartition(partition)
+              .withKeyMetadata(keyMetadata)
+              .withSortOrder(dataSortOrder)
+              .overwrite();
+
+          configureDataWrite(avroBuilder);
+
+          return avroBuilder.build();
+
+        case PARQUET:
+          Parquet.DataWriteBuilder parquetBuilder = Parquet.writeData(outputFile)
+              .schema(dataSchema)
+              .setAll(properties)
+              .metricsConfig(metricsConfig)
+              .withSpec(spec)
+              .withPartition(partition)
+              .withKeyMetadata(keyMetadata)
+              .withSortOrder(dataSortOrder)
+              .overwrite();
+
+          configureDataWrite(parquetBuilder);
+
+          return parquetBuilder.build();
+
+        case ORC:
+          ORC.DataWriteBuilder orcBuilder = ORC.writeData(outputFile)
+              .schema(dataSchema)
+              .setAll(properties)
+              .metricsConfig(metricsConfig)
+              .withSpec(spec)
+              .withPartition(partition)
+              .withKeyMetadata(keyMetadata)
+              .withSortOrder(dataSortOrder)
+              .overwrite();
+
+          configureDataWrite(orcBuilder);
+
+          return orcBuilder.build();
+
+        default:
+          throw new UnsupportedOperationException("Unsupported data file format: " + dataFileFormat);
+      }
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public EqualityDeleteWriter<T> newEqualityDeleteWriter(EncryptedOutputFile file, PartitionSpec spec,
+                                                         StructLike partition) {
+    OutputFile outputFile = file.encryptingOutputFile();
+    EncryptionKeyMetadata keyMetadata = file.keyMetadata();
+    Map<String, String> properties = table.properties();
+    MetricsConfig metricsConfig = MetricsConfig.fromProperties(properties);
+
+    try {
+      switch (deleteFileFormat) {
+        case AVRO:
+          // TODO: support metrics configs in Avro equality delete writer
+
+          Avro.DeleteWriteBuilder avroBuilder = Avro.writeDeletes(outputFile)
+              .setAll(properties)
+              .rowSchema(equalityDeleteRowSchema)
+              .equalityFieldIds(equalityFieldIds)
+              .withSpec(spec)
+              .withPartition(partition)
+              .withKeyMetadata(keyMetadata)
+              .withSortOrder(equalityDeleteSortOrder)
+              .overwrite();
+
+          configureEqualityDelete(avroBuilder);
+
+          return avroBuilder.buildEqualityWriter();
+
+        case PARQUET:
+          Parquet.DeleteWriteBuilder parquetBuilder = Parquet.writeDeletes(outputFile)
+              .setAll(properties)
+              .metricsConfig(metricsConfig)
+              .rowSchema(equalityDeleteRowSchema)
+              .equalityFieldIds(equalityFieldIds)
+              .withSpec(spec)
+              .withPartition(partition)
+              .withKeyMetadata(keyMetadata)
+              .withSortOrder(equalityDeleteSortOrder)
+              .overwrite();
+
+          configureEqualityDelete(parquetBuilder);
+
+          return parquetBuilder.buildEqualityWriter();
+
+        default:
+          throw new UnsupportedOperationException("Unsupported format for equality deletes: " + deleteFileFormat);
+      }
+    } catch (IOException e) {
+      throw new UncheckedIOException("Failed to create new equality delete writer", e);
+    }
+  }
+
+  @Override
+  public PositionDeleteWriter<T> newPositionDeleteWriter(EncryptedOutputFile file, PartitionSpec spec,
+                                                         StructLike partition) {
+    OutputFile outputFile = file.encryptingOutputFile();
+    EncryptionKeyMetadata keyMetadata = file.keyMetadata();
+    Map<String, String> properties = table.properties();
+
+    // TODO: build and pass a correct metrics config for position deletes

Review comment:
       Hm. Seems like we should move `MetricsConfig` to use column IDs by default?




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677283245



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/SparkWriterFactory.java
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.data.BaseWriterFactory;
+import org.apache.iceberg.io.DeleteSchemaUtil;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.data.SparkAvroWriter;
+import org.apache.iceberg.spark.data.SparkOrcWriter;
+import org.apache.iceberg.spark.data.SparkParquetWriters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+import static org.apache.iceberg.MetadataColumns.DELETE_FILE_ROW_FIELD_NAME;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.DELETE_DEFAULT_FILE_FORMAT;
+
+class SparkWriterFactory extends BaseWriterFactory<InternalRow> {
+  private StructType dataSparkType;
+  private StructType equalityDeleteSparkType;
+  private StructType positionDeleteSparkType;
+
+  SparkWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema, StructType dataSparkType,
+                     SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                     int[] equalityFieldIds, Schema equalityDeleteRowSchema, StructType equalityDeleteSparkType,
+                     SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                     StructType positionDeleteSparkType, SortOrder positionDeleteSortOrder) {
+
+    super(table, dataFileFormat, dataSchema, dataSortOrder, deleteFileFormat, equalityFieldIds, equalityDeleteRowSchema,
+        equalityDeleteSortOrder, positionDeleteRowSchema, positionDeleteSortOrder);
+
+    this.dataSparkType = dataSparkType;
+    this.equalityDeleteSparkType = equalityDeleteSparkType;
+    this.positionDeleteSparkType = positionDeleteSparkType;
+  }
+
+  static Builder builderFor(Table table) {
+    return new Builder(table);
+  }
+
+  @Override
+  protected void configureDataWrite(Avro.DataWriteBuilder builder) {
+    builder.createWriterFunc(ignored -> new SparkAvroWriter(dataSparkType()));
+  }
+
+  @Override
+  protected void configureEqualityDelete(Avro.DeleteWriteBuilder builder) {
+    builder.createWriterFunc(ignored -> new SparkAvroWriter(equalityDeleteSparkType()));
+  }
+
+  @Override
+  protected void configurePositionDelete(Avro.DeleteWriteBuilder builder) {
+    boolean withRow = positionDeleteSparkType().getFieldIndex(DELETE_FILE_ROW_FIELD_NAME).isDefined();
+    if (withRow) {
+      // SparkAvroWriter accepts just the Spark type of the row ignoring the path and pos
+      StructField rowField = positionDeleteSparkType().apply(DELETE_FILE_ROW_FIELD_NAME);
+      StructType positionDeleteRowSparkType = (StructType) rowField.dataType();
+      builder.createWriterFunc(ignored -> new SparkAvroWriter(positionDeleteRowSparkType));
+    }
+  }
+
+  @Override
+  protected void configureDataWrite(Parquet.DataWriteBuilder builder) {
+    builder.createWriterFunc(msgType -> SparkParquetWriters.buildWriter(dataSparkType(), msgType));
+  }
+
+  @Override
+  protected void configureEqualityDelete(Parquet.DeleteWriteBuilder builder) {
+    builder.createWriterFunc(msgType -> SparkParquetWriters.buildWriter(equalityDeleteSparkType(), msgType));
+  }
+
+  @Override
+  protected void configurePositionDelete(Parquet.DeleteWriteBuilder builder) {
+    builder.createWriterFunc(msgType -> SparkParquetWriters.buildWriter(positionDeleteSparkType(), msgType));
+    builder.transformPaths(path -> UTF8String.fromString(path.toString()));
+  }
+
+  @Override
+  protected void configureDataWrite(ORC.DataWriteBuilder builder) {
+    builder.createWriterFunc(SparkOrcWriter::new);
+  }
+
+  private StructType dataSparkType() {
+    if (dataSparkType == null) {
+      Preconditions.checkNotNull(dataSchema(), "Data schema must not be null");
+      this.dataSparkType = SparkSchemaUtil.convert(dataSchema());
+    }
+
+    return dataSparkType;
+  }
+
+  private StructType equalityDeleteSparkType() {
+    if (equalityDeleteSparkType == null) {
+      Preconditions.checkNotNull(equalityDeleteRowSchema(), "Equality delete schema must not be null");
+      this.equalityDeleteSparkType = SparkSchemaUtil.convert(equalityDeleteRowSchema());
+    }
+
+    return equalityDeleteSparkType;
+  }
+
+  private StructType positionDeleteSparkType() {
+    if (positionDeleteSparkType == null) {
+      // wrap the optional row schema into the position delete schema that contains path and position
+      Schema positionDeleteSchema = DeleteSchemaUtil.posDeleteSchema(positionDeleteRowSchema());
+      this.positionDeleteSparkType = SparkSchemaUtil.convert(positionDeleteSchema);
+    }
+
+    return positionDeleteSparkType;
+  }
+
+  static class Builder {
+    private final Table table;
+    private FileFormat dataFileFormat;
+    private Schema dataSchema;
+    private StructType dataSparkType;
+    private SortOrder dataSortOrder;
+    private FileFormat deleteFileFormat;
+    private int[] equalityFieldIds;
+    private Schema equalityDeleteRowSchema;
+    private StructType equalityDeleteSparkType;
+    private SortOrder equalityDeleteSortOrder;
+    private Schema positionDeleteRowSchema;
+    private StructType positionDeleteSparkType;
+    private SortOrder positionDeleteSortOrder;
+
+    Builder(Table table) {
+      this.table = table;
+
+      Map<String, String> properties = table.properties();
+
+      String dataFileFormatName = properties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT);
+      this.dataFileFormat = FileFormat.valueOf(dataFileFormatName.toUpperCase(Locale.ENGLISH));
+
+      String deleteFileFormatName = properties.getOrDefault(DELETE_DEFAULT_FILE_FORMAT, dataFileFormatName);
+      this.deleteFileFormat = FileFormat.valueOf(deleteFileFormatName.toUpperCase(Locale.ENGLISH));
+    }
+
+    Builder dataFileFormat(FileFormat newDataFileFormat) {
+      this.dataFileFormat = newDataFileFormat;
+      return this;
+    }
+
+    Builder dataSchema(Schema newDataSchema) {
+      this.dataSchema = newDataSchema;
+      return this;
+    }
+
+    Builder dataSparkType(StructType newDataSparkType) {
+      this.dataSparkType = newDataSparkType;
+      return this;
+    }
+
+    Builder dataSortOrder(SortOrder newDataSortOrder) {
+      this.dataSortOrder = newDataSortOrder;
+      return this;
+    }
+
+    Builder deleteFileFormat(FileFormat newDeleteFileFormat) {
+      this.deleteFileFormat = newDeleteFileFormat;
+      return this;
+    }
+
+    Builder equalityFieldIds(int[] newEqualityFieldIds) {
+      this.equalityFieldIds = newEqualityFieldIds;
+      return this;
+    }
+
+    Builder equalityDeleteRowSchema(Schema newEqualityDeleteRowSchema) {
+      this.equalityDeleteRowSchema = newEqualityDeleteRowSchema;
+      return this;
+    }
+
+    Builder equalityDeleteSparkType(StructType newEqualityDeleteSparkType) {
+      this.equalityDeleteSparkType = newEqualityDeleteSparkType;
+      return this;
+    }
+
+    Builder equalityDeleteSortOrder(SortOrder newEqualityDeleteSortOrder) {
+      this.equalityDeleteSortOrder = newEqualityDeleteSortOrder;
+      return this;
+    }
+
+    Builder positionDeleteRowSchema(Schema newPositionDeleteRowSchema) {
+      this.positionDeleteRowSchema = newPositionDeleteRowSchema;
+      return this;
+    }
+
+    Builder positionDeleteSparkType(StructType newPositionDeleteSparkType) {

Review comment:
       I doubt people will set those parameters: 
   * dataSparkType
   * equalityDeleteSparkType
   * positionDeleteSparkType
   
   Those spark types are always transformed from iceberg schema, such as `dataSchema`, `equalityDeleteRowSchema`, `positionDeleteRowSchema`.  There's no much meaning that people convert those iceberg schema to a spark type, and then set up those spark types as the factory's default implementation. 
   
   Especially, the `positionDeleteSparkType` will need to include all the file_path, row_offset and positional delete row schema.  It is actually a relatively high threshold for users.  
   
   I'd recommend to remove them in this PR and add them back if someone think we have to provide those methods for some purposes.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677121745



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {
+
+  /**
+   * Creates a new {@link DataWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written data belongs to
+   * @param partition the partition written data belongs to or null if the spec is unpartitioned
+   * @return the constructed data writer
+   */
+  DataWriter<T> newDataWriter(EncryptedOutputFile file, PartitionSpec spec, StructLike partition);
+
+  /**
+   * Creates a new {@link EqualityDeleteWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written deletes belong to
+   * @param partition the partition written deletes belong to or null if the spec is unpartitioned

Review comment:
       Maybe better if we have an extra method for nonpartitioned table? Or using `PartitionSpec.unpartitioned()`?
   Just some random ideas, since I am not a big fan of null attributes. 




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677026451



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/SparkWriterFactory.java
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.data.BaseWriterFactory;
+import org.apache.iceberg.io.DeleteSchemaUtil;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.data.SparkAvroWriter;
+import org.apache.iceberg.spark.data.SparkOrcWriter;
+import org.apache.iceberg.spark.data.SparkParquetWriters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+import static org.apache.iceberg.MetadataColumns.DELETE_FILE_ROW_FIELD_NAME;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.DELETE_DEFAULT_FILE_FORMAT;
+
+class SparkWriterFactory extends BaseWriterFactory<InternalRow> {
+  private StructType dataSparkType;
+  private StructType equalityDeleteSparkType;
+  private StructType positionDeleteSparkType;
+
+  SparkWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema, StructType dataSparkType,
+                     SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                     int[] equalityFieldIds, Schema equalityDeleteRowSchema, StructType equalityDeleteSparkType,
+                     SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                     StructType positionDeleteSparkType, SortOrder positionDeleteSortOrder) {
+
+    super(table, dataFileFormat, dataSchema, dataSortOrder, deleteFileFormat, equalityFieldIds, equalityDeleteRowSchema,
+        equalityDeleteSortOrder, positionDeleteRowSchema, positionDeleteSortOrder);
+
+    this.dataSparkType = dataSparkType;
+    this.equalityDeleteSparkType = equalityDeleteSparkType;
+    this.positionDeleteSparkType = positionDeleteSparkType;
+  }
+
+  static Builder builderFor(Table table) {
+    return new Builder(table);
+  }
+
+  @Override
+  protected void configureDataWrite(Avro.DataWriteBuilder builder) {
+    builder.createWriterFunc(ignored -> new SparkAvroWriter(dataSparkType()));
+  }
+
+  @Override
+  protected void configureEqualityDelete(Avro.DeleteWriteBuilder builder) {
+    builder.createWriterFunc(ignored -> new SparkAvroWriter(equalityDeleteSparkType()));
+  }
+
+  @Override
+  protected void configurePositionDelete(Avro.DeleteWriteBuilder builder) {
+    boolean withRow = positionDeleteSparkType().getFieldIndex(DELETE_FILE_ROW_FIELD_NAME).isDefined();
+    if (withRow) {
+      // SparkAvroWriter accepts just the Spark type of the row ignoring the path and pos
+      StructField rowField = positionDeleteSparkType().apply(DELETE_FILE_ROW_FIELD_NAME);
+      StructType positionDeleteRowSparkType = (StructType) rowField.dataType();
+      builder.createWriterFunc(ignored -> new SparkAvroWriter(positionDeleteRowSparkType));
+    }
+  }
+
+  @Override
+  protected void configureDataWrite(Parquet.DataWriteBuilder builder) {
+    builder.createWriterFunc(msgType -> SparkParquetWriters.buildWriter(dataSparkType(), msgType));
+  }
+
+  @Override
+  protected void configureEqualityDelete(Parquet.DeleteWriteBuilder builder) {
+    builder.createWriterFunc(msgType -> SparkParquetWriters.buildWriter(equalityDeleteSparkType(), msgType));
+  }
+
+  @Override
+  protected void configurePositionDelete(Parquet.DeleteWriteBuilder builder) {
+    builder.createWriterFunc(msgType -> SparkParquetWriters.buildWriter(positionDeleteSparkType(), msgType));
+    builder.transformPaths(path -> UTF8String.fromString(path.toString()));
+  }
+
+  @Override
+  protected void configureDataWrite(ORC.DataWriteBuilder builder) {
+    builder.createWriterFunc(SparkOrcWriter::new);
+  }
+
+  private StructType dataSparkType() {
+    if (dataSparkType == null) {
+      Preconditions.checkNotNull(dataSchema(), "Data schema must not be null");
+      this.dataSparkType = SparkSchemaUtil.convert(dataSchema());
+    }
+
+    return dataSparkType;
+  }
+
+  private StructType equalityDeleteSparkType() {
+    if (equalityDeleteSparkType == null) {
+      Preconditions.checkNotNull(equalityDeleteRowSchema(), "Equality delete schema must not be null");
+      this.equalityDeleteSparkType = SparkSchemaUtil.convert(equalityDeleteRowSchema());
+    }
+
+    return equalityDeleteSparkType;
+  }
+
+  private StructType positionDeleteSparkType() {
+    if (positionDeleteSparkType == null) {
+      // wrap the optional row schema into the position delete schema that contains path and position
+      Schema positionDeleteSchema = DeleteSchemaUtil.posDeleteSchema(positionDeleteRowSchema());
+      this.positionDeleteSparkType = SparkSchemaUtil.convert(positionDeleteSchema);
+    }
+
+    return positionDeleteSparkType;
+  }
+
+  static class Builder {
+    private final Table table;
+    private FileFormat dataFileFormat;
+    private Schema dataSchema;
+    private StructType dataSparkType;
+    private SortOrder dataSortOrder;
+    private FileFormat deleteFileFormat;
+    private int[] equalityFieldIds;
+    private Schema equalityDeleteRowSchema;
+    private StructType equalityDeleteSparkType;
+    private SortOrder equalityDeleteSortOrder;
+    private Schema positionDeleteRowSchema;
+    private StructType positionDeleteSparkType;
+    private SortOrder positionDeleteSortOrder;
+
+    Builder(Table table) {
+      this.table = table;
+
+      Map<String, String> properties = table.properties();
+
+      String dataFileFormatName = properties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT);
+      this.dataFileFormat = FileFormat.valueOf(dataFileFormatName.toUpperCase(Locale.ENGLISH));
+
+      String deleteFileFormatName = properties.getOrDefault(DELETE_DEFAULT_FILE_FORMAT, dataFileFormatName);
+      this.deleteFileFormat = FileFormat.valueOf(deleteFileFormatName.toUpperCase(Locale.ENGLISH));
+    }
+
+    Builder dataFileFormat(FileFormat newDataFileFormat) {
+      this.dataFileFormat = newDataFileFormat;
+      return this;
+    }
+
+    Builder dataSchema(Schema newDataSchema) {
+      this.dataSchema = newDataSchema;
+      return this;
+    }
+
+    Builder dataSparkType(StructType newDataSparkType) {
+      this.dataSparkType = newDataSparkType;
+      return this;
+    }
+
+    Builder dataSortOrder(SortOrder newDataSortOrder) {
+      this.dataSortOrder = newDataSortOrder;
+      return this;
+    }
+
+    Builder deleteFileFormat(FileFormat newDeleteFileFormat) {
+      this.deleteFileFormat = newDeleteFileFormat;
+      return this;
+    }
+
+    Builder equalityFieldIds(int[] newEqualityFieldIds) {
+      this.equalityFieldIds = newEqualityFieldIds;
+      return this;
+    }
+
+    Builder equalityDeleteRowSchema(Schema newEqualityDeleteRowSchema) {
+      this.equalityDeleteRowSchema = newEqualityDeleteRowSchema;
+      return this;
+    }
+
+    Builder equalityDeleteSparkType(StructType newEqualityDeleteSparkType) {
+      this.equalityDeleteSparkType = newEqualityDeleteSparkType;
+      return this;
+    }
+
+    Builder equalityDeleteSortOrder(SortOrder newEqualityDeleteSortOrder) {
+      this.equalityDeleteSortOrder = newEqualityDeleteSortOrder;
+      return this;
+    }
+
+    Builder positionDeleteRowSchema(Schema newPositionDeleteRowSchema) {
+      this.positionDeleteRowSchema = newPositionDeleteRowSchema;
+      return this;
+    }
+
+    Builder positionDeleteSparkType(StructType newPositionDeleteSparkType) {

Review comment:
       I'd like to emphasize that this is a complete type (includes path and pos), not just the row schema.
   The assumption here is that we will validate the write schema before constructing a writer factory. Hence, we will already have the Spark type and may skip one more round of conversion from Iceberg to Spark.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677613699



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {

Review comment:
       Yeah, it is similar to the existing `FileAppenderFactory`.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677926707



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {
+
+  /**
+   * Creates a new {@link DataWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written data belongs to

Review comment:
       Consider the following sequence of actions:
   1. create table (ts, category, data) partitioned by days(ts)
   2. day 1 adds some data files
   3. alter table add partition field category
   4. day 2 adds some data files
   5. delete some data where day=1 and data=xxx, in this case the delete writer should use the old partition spec. If writing using the latest spec, it would produce 1 delete file per category.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r680271800



##########
File path: data/src/main/java/org/apache/iceberg/data/BaseWriterFactory.java
##########
@@ -0,0 +1,257 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriterFactory;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+/**
+ * A base writer factory to be extended by query engine integrations.
+ */
+public abstract class BaseWriterFactory<T> implements WriterFactory<T> {
+  private final Table table;
+  private final FileFormat dataFileFormat;
+  private final Schema dataSchema;
+  private final SortOrder dataSortOrder;
+  private final FileFormat deleteFileFormat;
+  private final int[] equalityFieldIds;
+  private final Schema equalityDeleteRowSchema;
+  private final SortOrder equalityDeleteSortOrder;
+  private final Schema positionDeleteRowSchema;
+  private final SortOrder positionDeleteSortOrder;
+
+  protected BaseWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema,
+                              SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                              int[] equalityFieldIds, Schema equalityDeleteRowSchema,
+                              SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                              SortOrder positionDeleteSortOrder) {
+    this.table = table;
+    this.dataFileFormat = dataFileFormat;
+    this.dataSchema = dataSchema;
+    this.dataSortOrder = dataSortOrder;
+    this.deleteFileFormat = deleteFileFormat;
+    this.equalityFieldIds = equalityFieldIds;
+    this.equalityDeleteRowSchema = equalityDeleteRowSchema;
+    this.equalityDeleteSortOrder = equalityDeleteSortOrder;
+    this.positionDeleteRowSchema = positionDeleteRowSchema;
+    this.positionDeleteSortOrder = positionDeleteSortOrder;
+  }
+
+  protected abstract void configureDataWrite(Avro.DataWriteBuilder builder);
+  protected abstract void configureEqualityDelete(Avro.DeleteWriteBuilder builder);
+  protected abstract void configurePositionDelete(Avro.DeleteWriteBuilder builder);
+
+  protected abstract void configureDataWrite(Parquet.DataWriteBuilder builder);
+  protected abstract void configureEqualityDelete(Parquet.DeleteWriteBuilder builder);
+  protected abstract void configurePositionDelete(Parquet.DeleteWriteBuilder builder);
+
+  protected abstract void configureDataWrite(ORC.DataWriteBuilder builder);

Review comment:
       It would be helpful to add a TODO for when someone implements delete files in ORC.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677213848



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {

Review comment:
       I think you mean we will initialize the `WriterFactory` with the given infomations:  schema, spec, properties etc. And then create different writers (include both equality/positional delete writer and data writers) for different partitions among different parallelism tasks. Yes, that's the current behevior that we `FileAppenderFactory` provide now, I think it's OK.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r679566133



##########
File path: data/src/main/java/org/apache/iceberg/data/BaseWriterFactory.java
##########
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriterFactory;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+/**
+ * A base writer factory to be extended by query engine integrations.
+ */
+public abstract class BaseWriterFactory<T> implements WriterFactory<T> {
+  private final Table table;
+  private final FileFormat dataFileFormat;
+  private final Schema dataSchema;
+  private final SortOrder dataSortOrder;
+  private final FileFormat deleteFileFormat;
+  private final int[] equalityFieldIds;
+  private final Schema equalityDeleteRowSchema;
+  private final SortOrder equalityDeleteSortOrder;
+  private final Schema positionDeleteRowSchema;
+  private final SortOrder positionDeleteSortOrder;
+
+  protected BaseWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema,
+                              SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                              int[] equalityFieldIds, Schema equalityDeleteRowSchema,
+                              SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                              SortOrder positionDeleteSortOrder) {

Review comment:
       No, I don't think we would ever need the position delete order to change. The order is fixed so that we can always merge deletes while reading (why it's ordered by _pos), and so that we can merge delete files using streams as well (that's why it's first ordered by _file).




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r680268403



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {
+
+  /**
+   * Creates a new {@link DataWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written data belongs to

Review comment:
       @rdblue, that applies to equality deletes and upsert use cases, right? In case of MERGE INTO, we know the source row spec id and its partition value by querying `_spec` and `_partition` metadata columns?




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r680268403



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {
+
+  /**
+   * Creates a new {@link DataWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written data belongs to

Review comment:
       @rdblue, that applies to equality deletes and upsert use cases, right? In case of MERGE INTO, we know the source row spec id and its partition value?




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#issuecomment-889460164


   Thanks for the initial review round, I'll update the PR later today.


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r679551418



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {
+
+  /**
+   * Creates a new {@link DataWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written data belongs to
+   * @param partition the partition written data belongs to or null if the spec is unpartitioned
+   * @return the constructed data writer
+   */
+  DataWriter<T> newDataWriter(EncryptedOutputFile file, PartitionSpec spec, StructLike partition);
+
+  /**
+   * Creates a new {@link EqualityDeleteWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written deletes belong to
+   * @param partition the partition written deletes belong to or null if the spec is unpartitioned
+   * @return the constructed equality delete writer
+   */
+  EqualityDeleteWriter<T> newEqualityDeleteWriter(EncryptedOutputFile file, PartitionSpec spec, StructLike partition);
+
+  /**
+   * Creates a new {@link PositionDeleteWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written deletes belong to
+   * @param partition the partition written deletes belong to or null if the spec is unpartitioned
+   * @return the constructed position delete writer
+   */
+  PositionDeleteWriter<T> newPositionDeleteWriter(EncryptedOutputFile file, PartitionSpec spec, StructLike partition);

Review comment:
       I think our codebase is a little bit inconsistent. One concern with auto alignment is that we will have to modify and re-align all params each time we add a new parameter.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677617179



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {
+
+  /**
+   * Creates a new {@link DataWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written data belongs to
+   * @param partition the partition written data belongs to or null if the spec is unpartitioned
+   * @return the constructed data writer
+   */
+  DataWriter<T> newDataWriter(EncryptedOutputFile file, PartitionSpec spec, StructLike partition);
+
+  /**
+   * Creates a new {@link EqualityDeleteWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written deletes belong to
+   * @param partition the partition written deletes belong to or null if the spec is unpartitioned

Review comment:
       I think @openinx's point is a good reason to keep the existing behavior and allow nulls.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] yyanyy commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
yyanyy commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r679438821



##########
File path: data/src/main/java/org/apache/iceberg/data/BaseWriterFactory.java
##########
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriterFactory;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+/**
+ * A base writer factory to be extended by query engine integrations.
+ */
+public abstract class BaseWriterFactory<T> implements WriterFactory<T> {
+  private final Table table;
+  private final FileFormat dataFileFormat;
+  private final Schema dataSchema;
+  private final SortOrder dataSortOrder;
+  private final FileFormat deleteFileFormat;
+  private final int[] equalityFieldIds;
+  private final Schema equalityDeleteRowSchema;
+  private final SortOrder equalityDeleteSortOrder;
+  private final Schema positionDeleteRowSchema;
+  private final SortOrder positionDeleteSortOrder;
+
+  protected BaseWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema,
+                              SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                              int[] equalityFieldIds, Schema equalityDeleteRowSchema,
+                              SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                              SortOrder positionDeleteSortOrder) {
+    this.table = table;

Review comment:
       I think it's helpful to hold a `table` for the classes that extend this class, to easily retrieve attributes like table schema and table properties for initializing this `BaseWriterFactory` constructor, but I think within this class itself, it might be more clear to explicitly state the required attributes (I believe in this case, just the table property map) than passing the entire `table` object that contains some duplications of the fields already passing in, to ensure people who modify this class later don't retrieve those attributes from this `table` object. 




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r679485352



##########
File path: data/src/main/java/org/apache/iceberg/data/BaseWriterFactory.java
##########
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriterFactory;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+/**
+ * A base writer factory to be extended by query engine integrations.
+ */
+public abstract class BaseWriterFactory<T> implements WriterFactory<T> {
+  private final Table table;
+  private final FileFormat dataFileFormat;
+  private final Schema dataSchema;
+  private final SortOrder dataSortOrder;
+  private final FileFormat deleteFileFormat;
+  private final int[] equalityFieldIds;
+  private final Schema equalityDeleteRowSchema;
+  private final SortOrder equalityDeleteSortOrder;
+  private final Schema positionDeleteRowSchema;
+  private final SortOrder positionDeleteSortOrder;
+
+  protected BaseWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema,
+                              SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                              int[] equalityFieldIds, Schema equalityDeleteRowSchema,
+                              SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                              SortOrder positionDeleteSortOrder) {
+    this.table = table;

Review comment:
       @jackye1995 is correct. We have introduced `SerializableTable` so that we can pass a read-only table object and have access to needed fields. Previously, we had to modify a lot of places every time we needed to pass a new entity to the writer. Passing a read-only object is cleaner.
   
   




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#issuecomment-889534826


   @openinx, I've updated the PR to set the metrics config for equality deletes in Parquet. Unfortunately, `Avro` delete write builders don't support it just yet, I'll submit a separate PR for that. I'll work on setting a correct config for position deletes next.


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r679566421



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {
+
+  /**
+   * Creates a new {@link DataWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written data belongs to
+   * @param partition the partition written data belongs to or null if the spec is unpartitioned
+   * @return the constructed data writer
+   */
+  DataWriter<T> newDataWriter(EncryptedOutputFile file, PartitionSpec spec, StructLike partition);
+
+  /**
+   * Creates a new {@link EqualityDeleteWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written deletes belong to
+   * @param partition the partition written deletes belong to or null if the spec is unpartitioned
+   * @return the constructed equality delete writer
+   */
+  EqualityDeleteWriter<T> newEqualityDeleteWriter(EncryptedOutputFile file, PartitionSpec spec, StructLike partition);
+
+  /**
+   * Creates a new {@link PositionDeleteWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written deletes belong to
+   * @param partition the partition written deletes belong to or null if the spec is unpartitioned
+   * @return the constructed position delete writer
+   */
+  PositionDeleteWriter<T> newPositionDeleteWriter(EncryptedOutputFile file, PartitionSpec spec, StructLike partition);

Review comment:
       +1 for not aligning parameters so that we don't re-align when renaming or changing variables. That just introduces unnecessary changes that can cause commit conflicts.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677612838



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/SparkWriterFactory.java
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.data.BaseWriterFactory;
+import org.apache.iceberg.io.DeleteSchemaUtil;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.data.SparkAvroWriter;
+import org.apache.iceberg.spark.data.SparkOrcWriter;
+import org.apache.iceberg.spark.data.SparkParquetWriters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+import static org.apache.iceberg.MetadataColumns.DELETE_FILE_ROW_FIELD_NAME;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.DELETE_DEFAULT_FILE_FORMAT;
+
+class SparkWriterFactory extends BaseWriterFactory<InternalRow> {
+  private StructType dataSparkType;
+  private StructType equalityDeleteSparkType;
+  private StructType positionDeleteSparkType;
+
+  SparkWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema, StructType dataSparkType,
+                     SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                     int[] equalityFieldIds, Schema equalityDeleteRowSchema, StructType equalityDeleteSparkType,
+                     SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                     StructType positionDeleteSparkType, SortOrder positionDeleteSortOrder) {
+
+    super(table, dataFileFormat, dataSchema, dataSortOrder, deleteFileFormat, equalityFieldIds, equalityDeleteRowSchema,
+        equalityDeleteSortOrder, positionDeleteRowSchema, positionDeleteSortOrder);
+
+    this.dataSparkType = dataSparkType;
+    this.equalityDeleteSparkType = equalityDeleteSparkType;
+    this.positionDeleteSparkType = positionDeleteSparkType;
+  }
+
+  static Builder builderFor(Table table) {
+    return new Builder(table);
+  }
+
+  @Override
+  protected void configureDataWrite(Avro.DataWriteBuilder builder) {
+    builder.createWriterFunc(ignored -> new SparkAvroWriter(dataSparkType()));
+  }
+
+  @Override
+  protected void configureEqualityDelete(Avro.DeleteWriteBuilder builder) {
+    builder.createWriterFunc(ignored -> new SparkAvroWriter(equalityDeleteSparkType()));
+  }
+
+  @Override
+  protected void configurePositionDelete(Avro.DeleteWriteBuilder builder) {
+    boolean withRow = positionDeleteSparkType().getFieldIndex(DELETE_FILE_ROW_FIELD_NAME).isDefined();
+    if (withRow) {
+      // SparkAvroWriter accepts just the Spark type of the row ignoring the path and pos
+      StructField rowField = positionDeleteSparkType().apply(DELETE_FILE_ROW_FIELD_NAME);
+      StructType positionDeleteRowSparkType = (StructType) rowField.dataType();
+      builder.createWriterFunc(ignored -> new SparkAvroWriter(positionDeleteRowSparkType));
+    }
+  }
+
+  @Override
+  protected void configureDataWrite(Parquet.DataWriteBuilder builder) {
+    builder.createWriterFunc(msgType -> SparkParquetWriters.buildWriter(dataSparkType(), msgType));
+  }
+
+  @Override
+  protected void configureEqualityDelete(Parquet.DeleteWriteBuilder builder) {
+    builder.createWriterFunc(msgType -> SparkParquetWriters.buildWriter(equalityDeleteSparkType(), msgType));
+  }
+
+  @Override
+  protected void configurePositionDelete(Parquet.DeleteWriteBuilder builder) {
+    builder.createWriterFunc(msgType -> SparkParquetWriters.buildWriter(positionDeleteSparkType(), msgType));
+    builder.transformPaths(path -> UTF8String.fromString(path.toString()));
+  }
+
+  @Override
+  protected void configureDataWrite(ORC.DataWriteBuilder builder) {
+    builder.createWriterFunc(SparkOrcWriter::new);
+  }
+
+  private StructType dataSparkType() {
+    if (dataSparkType == null) {
+      Preconditions.checkNotNull(dataSchema(), "Data schema must not be null");
+      this.dataSparkType = SparkSchemaUtil.convert(dataSchema());
+    }
+
+    return dataSparkType;
+  }
+
+  private StructType equalityDeleteSparkType() {
+    if (equalityDeleteSparkType == null) {
+      Preconditions.checkNotNull(equalityDeleteRowSchema(), "Equality delete schema must not be null");
+      this.equalityDeleteSparkType = SparkSchemaUtil.convert(equalityDeleteRowSchema());
+    }
+
+    return equalityDeleteSparkType;
+  }
+
+  private StructType positionDeleteSparkType() {
+    if (positionDeleteSparkType == null) {
+      // wrap the optional row schema into the position delete schema that contains path and position
+      Schema positionDeleteSchema = DeleteSchemaUtil.posDeleteSchema(positionDeleteRowSchema());
+      this.positionDeleteSparkType = SparkSchemaUtil.convert(positionDeleteSchema);
+    }
+
+    return positionDeleteSparkType;
+  }
+
+  static class Builder {
+    private final Table table;
+    private FileFormat dataFileFormat;
+    private Schema dataSchema;
+    private StructType dataSparkType;
+    private SortOrder dataSortOrder;
+    private FileFormat deleteFileFormat;
+    private int[] equalityFieldIds;
+    private Schema equalityDeleteRowSchema;
+    private StructType equalityDeleteSparkType;
+    private SortOrder equalityDeleteSortOrder;
+    private Schema positionDeleteRowSchema;
+    private StructType positionDeleteSparkType;
+    private SortOrder positionDeleteSortOrder;
+
+    Builder(Table table) {
+      this.table = table;
+
+      Map<String, String> properties = table.properties();
+
+      String dataFileFormatName = properties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT);
+      this.dataFileFormat = FileFormat.valueOf(dataFileFormatName.toUpperCase(Locale.ENGLISH));
+
+      String deleteFileFormatName = properties.getOrDefault(DELETE_DEFAULT_FILE_FORMAT, dataFileFormatName);
+      this.deleteFileFormat = FileFormat.valueOf(deleteFileFormatName.toUpperCase(Locale.ENGLISH));
+    }
+
+    Builder dataFileFormat(FileFormat newDataFileFormat) {
+      this.dataFileFormat = newDataFileFormat;
+      return this;
+    }
+
+    Builder dataSchema(Schema newDataSchema) {
+      this.dataSchema = newDataSchema;
+      return this;
+    }
+
+    Builder dataSparkType(StructType newDataSparkType) {
+      this.dataSparkType = newDataSparkType;
+      return this;
+    }
+
+    Builder dataSortOrder(SortOrder newDataSortOrder) {
+      this.dataSortOrder = newDataSortOrder;
+      return this;
+    }
+
+    Builder deleteFileFormat(FileFormat newDeleteFileFormat) {
+      this.deleteFileFormat = newDeleteFileFormat;
+      return this;
+    }
+
+    Builder equalityFieldIds(int[] newEqualityFieldIds) {
+      this.equalityFieldIds = newEqualityFieldIds;
+      return this;
+    }
+
+    Builder equalityDeleteRowSchema(Schema newEqualityDeleteRowSchema) {
+      this.equalityDeleteRowSchema = newEqualityDeleteRowSchema;
+      return this;
+    }
+
+    Builder equalityDeleteSparkType(StructType newEqualityDeleteSparkType) {
+      this.equalityDeleteSparkType = newEqualityDeleteSparkType;
+      return this;
+    }
+
+    Builder equalityDeleteSortOrder(SortOrder newEqualityDeleteSortOrder) {
+      this.equalityDeleteSortOrder = newEqualityDeleteSortOrder;
+      return this;
+    }
+
+    Builder positionDeleteRowSchema(Schema newPositionDeleteRowSchema) {
+      this.positionDeleteRowSchema = newPositionDeleteRowSchema;
+      return this;
+    }
+
+    Builder positionDeleteSparkType(StructType newPositionDeleteSparkType) {

Review comment:
       @openinx, all Spark types are actually optional and are not required. In fact, we don't set them in tests and they are derived from Iceberg values by `SparkWriterFactory`.
   
   ```
   private StructType dataSparkType() {
     if (dataSparkType == null) {
       Preconditions.checkNotNull(dataSchema(), "Data schema must not be null");
       this.dataSparkType = SparkSchemaUtil.convert(dataSchema());
     }
   
     return dataSparkType;
   }
   ```
   
   What I am trying to say that we usually perform the write validation in Spark and we already have the Spark type.
   
   ```
   Schema writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema);
   TypeUtil.validateWriteSchema(table.schema(), writeSchema,
       checkNullability(spark, options), checkOrdering(spark, options));
   ```
   
   Specifically, the existing `SparkAppenderFactory` actually requires the data Spark type right now.
   
   ```
   SparkAppenderFactory appenderFactory = SparkAppenderFactory.builderFor(table, writeSchema, dsSchema).build();
   ```
   
   In this PR, I made the Spark types optional. I think we will set them in general as we will always perform the write validation and will have correct Spark types.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677915872



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {
+
+  /**
+   * Creates a new {@link DataWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written data belongs to

Review comment:
       I think I can understand the point that we want to write the delete files and data files in the same partition for a given partition spec.  But I still don't get the point what's the use case that spark will write data files or delete files into an historical partition spec. Are there any other issues or PRs that I missed ? 




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677913738



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {

Review comment:
       > That's why I don't accept FileFormat as an argument for methods constructing writes.
   
   Yes,  we don't need the `FileFormat` to construct the data/delete file writers.
   
   > Is there a use case when a single txn will write both Avro and Parquet data files?
   
   For the current compute-engine + iceberg integration work,  I don't think we support writing both avro and parquet data files in the same txn.  If people really want to do this,  they will need to generate their own avro and parquet files, and finally commit those files by using Iceberg Table API. I think that's the correct practice.
   
   > We can use different file formats for data and delete files.
   
   Yes, that's really helpful.   I remember that the JOIN process between data files and positional delete files is a MERGE-SORT JOIN  because the both data files and delete files have sorted joined keys : <file_name, row_offset>.   If we have `N` positional delete files in the JOIN progress,  then we will allocate at least `N*128MB` memory for each row group (To read the whole row) from the positional delete files (If we use parquet),  that's quite huge memory consuming, which will lead to OutOfMemory in the real production.  In this case, the idea  file format for positional delete files is `AVRO` I think.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r680287721



##########
File path: data/src/main/java/org/apache/iceberg/data/BaseWriterFactory.java
##########
@@ -0,0 +1,257 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriterFactory;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+/**
+ * A base writer factory to be extended by query engine integrations.
+ */
+public abstract class BaseWriterFactory<T> implements WriterFactory<T> {
+  private final Table table;
+  private final FileFormat dataFileFormat;
+  private final Schema dataSchema;
+  private final SortOrder dataSortOrder;
+  private final FileFormat deleteFileFormat;
+  private final int[] equalityFieldIds;
+  private final Schema equalityDeleteRowSchema;
+  private final SortOrder equalityDeleteSortOrder;
+  private final Schema positionDeleteRowSchema;
+  private final SortOrder positionDeleteSortOrder;
+
+  protected BaseWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema,
+                              SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                              int[] equalityFieldIds, Schema equalityDeleteRowSchema,
+                              SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                              SortOrder positionDeleteSortOrder) {
+    this.table = table;
+    this.dataFileFormat = dataFileFormat;
+    this.dataSchema = dataSchema;
+    this.dataSortOrder = dataSortOrder;
+    this.deleteFileFormat = deleteFileFormat;
+    this.equalityFieldIds = equalityFieldIds;
+    this.equalityDeleteRowSchema = equalityDeleteRowSchema;
+    this.equalityDeleteSortOrder = equalityDeleteSortOrder;
+    this.positionDeleteRowSchema = positionDeleteRowSchema;
+    this.positionDeleteSortOrder = positionDeleteSortOrder;
+  }
+
+  protected abstract void configureDataWrite(Avro.DataWriteBuilder builder);
+  protected abstract void configureEqualityDelete(Avro.DeleteWriteBuilder builder);
+  protected abstract void configurePositionDelete(Avro.DeleteWriteBuilder builder);
+
+  protected abstract void configureDataWrite(Parquet.DataWriteBuilder builder);
+  protected abstract void configureEqualityDelete(Parquet.DeleteWriteBuilder builder);
+  protected abstract void configurePositionDelete(Parquet.DeleteWriteBuilder builder);
+
+  protected abstract void configureDataWrite(ORC.DataWriteBuilder builder);

Review comment:
       Added a note above.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r679466671



##########
File path: data/src/main/java/org/apache/iceberg/data/BaseWriterFactory.java
##########
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriterFactory;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+/**
+ * A base writer factory to be extended by query engine integrations.
+ */
+public abstract class BaseWriterFactory<T> implements WriterFactory<T> {
+  private final Table table;
+  private final FileFormat dataFileFormat;
+  private final Schema dataSchema;
+  private final SortOrder dataSortOrder;
+  private final FileFormat deleteFileFormat;
+  private final int[] equalityFieldIds;
+  private final Schema equalityDeleteRowSchema;
+  private final SortOrder equalityDeleteSortOrder;
+  private final Schema positionDeleteRowSchema;
+  private final SortOrder positionDeleteSortOrder;
+
+  protected BaseWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema,
+                              SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                              int[] equalityFieldIds, Schema equalityDeleteRowSchema,
+                              SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                              SortOrder positionDeleteSortOrder) {

Review comment:
       My bad, you are right. Will we ever want to include data columns from `row` in the sort order in position deletes?
   Thoughts, @openinx @jackye1995?




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r680287789



##########
File path: core/src/main/java/org/apache/iceberg/MetadataColumns.java
##########
@@ -44,6 +44,7 @@ private MetadataColumns() {
       Integer.MAX_VALUE - 101, "file_path", Types.StringType.get(), "Path of a file in which a deleted row is stored");
   public static final NestedField DELETE_FILE_POS = NestedField.required(
       Integer.MAX_VALUE - 102, "pos", Types.LongType.get(), "Ordinal position of a deleted row in the data file");
+  public static final String DELETE_FILE_ROW_FIELD_NAME = "row";

Review comment:
       Ok, keeping it then.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r679551686



##########
File path: data/src/main/java/org/apache/iceberg/data/BaseWriterFactory.java
##########
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriterFactory;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+/**
+ * A base writer factory to be extended by query engine integrations.
+ */
+public abstract class BaseWriterFactory<T> implements WriterFactory<T> {
+  private final Table table;
+  private final FileFormat dataFileFormat;
+  private final Schema dataSchema;
+  private final SortOrder dataSortOrder;
+  private final FileFormat deleteFileFormat;
+  private final int[] equalityFieldIds;
+  private final Schema equalityDeleteRowSchema;
+  private final SortOrder equalityDeleteSortOrder;
+  private final Schema positionDeleteRowSchema;
+  private final SortOrder positionDeleteSortOrder;
+
+  protected BaseWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema,

Review comment:
       Well, it does not seem very clean as we need to provide an accessor method for each argument, which makes the implementation kind of bulky.

##########
File path: data/src/main/java/org/apache/iceberg/data/BaseWriterFactory.java
##########
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriterFactory;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+/**
+ * A base writer factory to be extended by query engine integrations.
+ */
+public abstract class BaseWriterFactory<T> implements WriterFactory<T> {
+  private final Table table;
+  private final FileFormat dataFileFormat;
+  private final Schema dataSchema;
+  private final SortOrder dataSortOrder;
+  private final FileFormat deleteFileFormat;
+  private final int[] equalityFieldIds;
+  private final Schema equalityDeleteRowSchema;
+  private final SortOrder equalityDeleteSortOrder;
+  private final Schema positionDeleteRowSchema;
+  private final SortOrder positionDeleteSortOrder;
+
+  protected BaseWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema,

Review comment:
       I'd probably keep it separate for now.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677920953



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/SparkWriterFactory.java
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.data.BaseWriterFactory;
+import org.apache.iceberg.io.DeleteSchemaUtil;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.data.SparkAvroWriter;
+import org.apache.iceberg.spark.data.SparkOrcWriter;
+import org.apache.iceberg.spark.data.SparkParquetWriters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+import static org.apache.iceberg.MetadataColumns.DELETE_FILE_ROW_FIELD_NAME;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.DELETE_DEFAULT_FILE_FORMAT;
+
+class SparkWriterFactory extends BaseWriterFactory<InternalRow> {
+  private StructType dataSparkType;
+  private StructType equalityDeleteSparkType;
+  private StructType positionDeleteSparkType;
+
+  SparkWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema, StructType dataSparkType,
+                     SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                     int[] equalityFieldIds, Schema equalityDeleteRowSchema, StructType equalityDeleteSparkType,
+                     SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                     StructType positionDeleteSparkType, SortOrder positionDeleteSortOrder) {
+
+    super(table, dataFileFormat, dataSchema, dataSortOrder, deleteFileFormat, equalityFieldIds, equalityDeleteRowSchema,
+        equalityDeleteSortOrder, positionDeleteRowSchema, positionDeleteSortOrder);
+
+    this.dataSparkType = dataSparkType;
+    this.equalityDeleteSparkType = equalityDeleteSparkType;
+    this.positionDeleteSparkType = positionDeleteSparkType;
+  }
+
+  static Builder builderFor(Table table) {
+    return new Builder(table);
+  }
+
+  @Override
+  protected void configureDataWrite(Avro.DataWriteBuilder builder) {
+    builder.createWriterFunc(ignored -> new SparkAvroWriter(dataSparkType()));
+  }
+
+  @Override
+  protected void configureEqualityDelete(Avro.DeleteWriteBuilder builder) {
+    builder.createWriterFunc(ignored -> new SparkAvroWriter(equalityDeleteSparkType()));
+  }
+
+  @Override
+  protected void configurePositionDelete(Avro.DeleteWriteBuilder builder) {
+    boolean withRow = positionDeleteSparkType().getFieldIndex(DELETE_FILE_ROW_FIELD_NAME).isDefined();
+    if (withRow) {
+      // SparkAvroWriter accepts just the Spark type of the row ignoring the path and pos
+      StructField rowField = positionDeleteSparkType().apply(DELETE_FILE_ROW_FIELD_NAME);
+      StructType positionDeleteRowSparkType = (StructType) rowField.dataType();
+      builder.createWriterFunc(ignored -> new SparkAvroWriter(positionDeleteRowSparkType));
+    }
+  }
+
+  @Override
+  protected void configureDataWrite(Parquet.DataWriteBuilder builder) {
+    builder.createWriterFunc(msgType -> SparkParquetWriters.buildWriter(dataSparkType(), msgType));
+  }
+
+  @Override
+  protected void configureEqualityDelete(Parquet.DeleteWriteBuilder builder) {
+    builder.createWriterFunc(msgType -> SparkParquetWriters.buildWriter(equalityDeleteSparkType(), msgType));
+  }
+
+  @Override
+  protected void configurePositionDelete(Parquet.DeleteWriteBuilder builder) {
+    builder.createWriterFunc(msgType -> SparkParquetWriters.buildWriter(positionDeleteSparkType(), msgType));
+    builder.transformPaths(path -> UTF8String.fromString(path.toString()));
+  }
+
+  @Override
+  protected void configureDataWrite(ORC.DataWriteBuilder builder) {
+    builder.createWriterFunc(SparkOrcWriter::new);
+  }
+
+  private StructType dataSparkType() {
+    if (dataSparkType == null) {
+      Preconditions.checkNotNull(dataSchema(), "Data schema must not be null");
+      this.dataSparkType = SparkSchemaUtil.convert(dataSchema());
+    }
+
+    return dataSparkType;
+  }
+
+  private StructType equalityDeleteSparkType() {
+    if (equalityDeleteSparkType == null) {
+      Preconditions.checkNotNull(equalityDeleteRowSchema(), "Equality delete schema must not be null");
+      this.equalityDeleteSparkType = SparkSchemaUtil.convert(equalityDeleteRowSchema());
+    }
+
+    return equalityDeleteSparkType;
+  }
+
+  private StructType positionDeleteSparkType() {
+    if (positionDeleteSparkType == null) {
+      // wrap the optional row schema into the position delete schema that contains path and position
+      Schema positionDeleteSchema = DeleteSchemaUtil.posDeleteSchema(positionDeleteRowSchema());
+      this.positionDeleteSparkType = SparkSchemaUtil.convert(positionDeleteSchema);
+    }
+
+    return positionDeleteSparkType;
+  }
+
+  static class Builder {
+    private final Table table;
+    private FileFormat dataFileFormat;
+    private Schema dataSchema;
+    private StructType dataSparkType;
+    private SortOrder dataSortOrder;
+    private FileFormat deleteFileFormat;
+    private int[] equalityFieldIds;
+    private Schema equalityDeleteRowSchema;
+    private StructType equalityDeleteSparkType;
+    private SortOrder equalityDeleteSortOrder;
+    private Schema positionDeleteRowSchema;
+    private StructType positionDeleteSparkType;
+    private SortOrder positionDeleteSortOrder;
+
+    Builder(Table table) {
+      this.table = table;
+
+      Map<String, String> properties = table.properties();
+
+      String dataFileFormatName = properties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT);
+      this.dataFileFormat = FileFormat.valueOf(dataFileFormatName.toUpperCase(Locale.ENGLISH));
+
+      String deleteFileFormatName = properties.getOrDefault(DELETE_DEFAULT_FILE_FORMAT, dataFileFormatName);
+      this.deleteFileFormat = FileFormat.valueOf(deleteFileFormatName.toUpperCase(Locale.ENGLISH));
+    }
+
+    Builder dataFileFormat(FileFormat newDataFileFormat) {
+      this.dataFileFormat = newDataFileFormat;
+      return this;
+    }
+
+    Builder dataSchema(Schema newDataSchema) {
+      this.dataSchema = newDataSchema;
+      return this;
+    }
+
+    Builder dataSparkType(StructType newDataSparkType) {
+      this.dataSparkType = newDataSparkType;
+      return this;
+    }
+
+    Builder dataSortOrder(SortOrder newDataSortOrder) {
+      this.dataSortOrder = newDataSortOrder;
+      return this;
+    }
+
+    Builder deleteFileFormat(FileFormat newDeleteFileFormat) {
+      this.deleteFileFormat = newDeleteFileFormat;
+      return this;
+    }
+
+    Builder equalityFieldIds(int[] newEqualityFieldIds) {
+      this.equalityFieldIds = newEqualityFieldIds;
+      return this;
+    }
+
+    Builder equalityDeleteRowSchema(Schema newEqualityDeleteRowSchema) {
+      this.equalityDeleteRowSchema = newEqualityDeleteRowSchema;
+      return this;
+    }
+
+    Builder equalityDeleteSparkType(StructType newEqualityDeleteSparkType) {
+      this.equalityDeleteSparkType = newEqualityDeleteSparkType;
+      return this;
+    }
+
+    Builder equalityDeleteSortOrder(SortOrder newEqualityDeleteSortOrder) {
+      this.equalityDeleteSortOrder = newEqualityDeleteSortOrder;
+      return this;
+    }
+
+    Builder positionDeleteRowSchema(Schema newPositionDeleteRowSchema) {
+      this.positionDeleteRowSchema = newPositionDeleteRowSchema;
+      return this;
+    }
+
+    Builder positionDeleteSparkType(StructType newPositionDeleteSparkType) {

Review comment:
       Okay,  make sense !




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677217147



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {
+
+  /**
+   * Creates a new {@link DataWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written data belongs to

Review comment:
       Is there possible that we will write data/delete files to an existing historical partition spec ?   I mean almost all the cases, we will produce data/delete files in the latests partition spec,  so when opening new writers among different parallelize tasks,  there seems no need to pass an extra partition spec with the same value.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677248202



##########
File path: data/src/test/java/org/apache/iceberg/io/TestWriterFactory.java
##########
@@ -0,0 +1,448 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.TableTestBase;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.IcebergGenerics;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataReader;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.StructLikeSet;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.MetadataColumns.DELETE_FILE_PATH;
+import static org.apache.iceberg.MetadataColumns.DELETE_FILE_POS;
+import static org.apache.iceberg.MetadataColumns.DELETE_FILE_ROW_FIELD_NAME;
+
+@RunWith(Parameterized.class)
+public abstract class TestWriterFactory<T> extends TableTestBase {
+  @Parameterized.Parameters(name = "FileFormat={0}, Partitioned={1}")
+  public static Object[] parameters() {
+    return new Object[][] {
+        new Object[]{FileFormat.AVRO, false},
+        new Object[]{FileFormat.AVRO, true},
+        new Object[]{FileFormat.PARQUET, false},
+        new Object[]{FileFormat.PARQUET, true},
+        new Object[]{FileFormat.ORC, false},
+        new Object[]{FileFormat.ORC, true}
+    };
+  }
+
+  private static final int TABLE_FORMAT_VERSION = 2;
+
+  private final FileFormat fileFormat;
+  private final boolean partitioned;
+  private final List<T> dataRows;
+
+  private StructLike partition = null;
+  private OutputFileFactory fileFactory = null;
+
+  public TestWriterFactory(FileFormat fileFormat, boolean partitioned) {
+    super(TABLE_FORMAT_VERSION);
+    this.fileFormat = fileFormat;
+    this.partitioned = partitioned;
+    this.dataRows = ImmutableList.of(
+        toRow(1, "aaa"),
+        toRow(2, "aaa"),
+        toRow(3, "aaa"),
+        toRow(4, "aaa"),
+        toRow(5, "aaa")
+    );
+  }
+
+  protected abstract WriterFactory<T> newWriterFactory(Schema dataSchema, List<Integer> equalityFieldIds,
+                                                       Schema equalityDeleteRowSchema, Schema positionDeleteRowSchema);
+
+  protected abstract T toRow(Integer id, String data);
+
+  protected abstract StructLikeSet toSet(Iterable<T> records);
+
+  protected FileFormat format() {
+    return fileFormat;
+  }
+
+  @Before
+  public void setupTable() throws Exception {
+    this.tableDir = temp.newFolder();
+    Assert.assertTrue(tableDir.delete()); // created during table creation
+
+    this.metadataDir = new File(tableDir, "metadata");
+
+    if (partitioned) {
+      this.table = create(SCHEMA, SPEC);
+      this.partition = initPartitionKey();
+    } else {
+      this.table = create(SCHEMA, PartitionSpec.unpartitioned());
+      this.partition = null;
+    }
+
+    this.fileFactory = OutputFileFactory.builderFor(table, 1, 1).format(fileFormat).build();
+  }
+
+  @Test
+  public void testDataWriter() throws IOException {
+    WriterFactory<T> writerFactory = newWriterFactory(table.schema());
+
+    DataFile dataFile = writeData(writerFactory, dataRows, table.spec(), partition);
+
+    table.newRowDelta()
+        .addRows(dataFile)
+        .commit();
+
+    Assert.assertEquals("Records should match", toSet(dataRows), actualRowSet("*"));
+  }
+
+  @Test
+  public void testEqualityDeleteWriter() throws IOException {
+    Assume.assumeFalse("ORC delete files are not supported", fileFormat == FileFormat.ORC);
+
+    List<Integer> equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId());
+    Schema equalityDeleteRowSchema = table.schema().select("id");
+    WriterFactory<T> writerFactory = newWriterFactory(table.schema(), equalityFieldIds, equalityDeleteRowSchema);
+
+    // write a data file
+    DataFile dataFile = writeData(writerFactory, dataRows, table.spec(), partition);
+
+    // commit the written data file
+    table.newRowDelta()
+        .addRows(dataFile)
+        .commit();
+
+    // write an equality delete file
+    List<T> deletes = ImmutableList.of(
+        toRow(1, "aaa"),
+        toRow(3, "bbb"),
+        toRow(5, "ccc")
+    );
+    DeleteFile deleteFile = writeEqualityDeletes(writerFactory, deletes, table.spec(), partition);
+
+    // verify the written delete file
+    GenericRecord deleteRecord = GenericRecord.create(equalityDeleteRowSchema);
+    List<Record> expectedDeletes = ImmutableList.of(
+        deleteRecord.copy("id", 1),
+        deleteRecord.copy("id", 3),
+        deleteRecord.copy("id", 5)
+    );
+    InputFile inputDeleteFile = table.io().newInputFile(deleteFile.path().toString());
+    List<Record> actualDeletes = readFile(equalityDeleteRowSchema, inputDeleteFile);
+    Assert.assertEquals("Delete records must match", expectedDeletes, actualDeletes);
+
+    // commit the written delete file
+    table.newRowDelta()
+        .addDeletes(deleteFile)
+        .commit();
+
+    // verify the delete file is applied correctly
+    List<T> expectedRows = ImmutableList.of(
+        toRow(2, "aaa"),
+        toRow(4, "aaa")
+    );
+    Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*"));
+  }
+
+  @Test
+  public void testEqualityDeleteWriterWithMultipleSpecs() throws IOException {
+    Assume.assumeFalse("ORC delete files are not supported", fileFormat == FileFormat.ORC);
+    Assume.assumeFalse("Table must start unpartitioned", partitioned);
+
+    List<Integer> equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId());
+    Schema equalityDeleteRowSchema = table.schema().select("id");
+    WriterFactory<T> writerFactory = newWriterFactory(table.schema(), equalityFieldIds, equalityDeleteRowSchema);
+
+    // write an unpartitioned data file
+    DataFile firstDataFile = writeData(writerFactory, dataRows, table.spec(), partition);
+    Assert.assertEquals("First data file must be unpartitioned", 0, firstDataFile.partition().size());
+
+    List<T> deletes = ImmutableList.of(
+        toRow(1, "aaa"),
+        toRow(2, "aaa"),
+        toRow(3, "aaa"),
+        toRow(4, "aaa")
+    );
+
+    // write an unpartitioned delete file
+    DeleteFile firstDeleteFile = writeEqualityDeletes(writerFactory, deletes, table.spec(), partition);
+    Assert.assertEquals("First delete file must be unpartitioned", 0, firstDeleteFile.partition().size());
+
+    // commit the first data and delete files
+    table.newAppend()
+        .appendFile(firstDataFile)
+        .commit();
+    table.newRowDelta()
+        .addDeletes(firstDeleteFile)
+        .commit();
+
+    // evolve the spec
+    table.updateSpec()
+        .addField("data")
+        .commit();
+
+    partition = initPartitionKey();
+
+    // write a partitioned data file
+    DataFile secondDataFile = writeData(writerFactory, dataRows, table.spec(), partition);
+    Assert.assertEquals("Second data file must be partitioned", 1, secondDataFile.partition().size());
+
+    // write a partitioned delete file
+    DeleteFile secondDeleteFile = writeEqualityDeletes(writerFactory, deletes, table.spec(), partition);
+    Assert.assertEquals("Second delete file must be artitioned", 1, secondDeleteFile.partition().size());
+
+    // commit the second data and delete files
+    table.newAppend()
+        .appendFile(secondDataFile)
+        .commit();
+    table.newRowDelta()
+        .addDeletes(secondDeleteFile)
+        .commit();
+
+    // verify both delete files are applied correctly
+    List<T> expectedRows = ImmutableList.of(
+        toRow(5, "aaa"),
+        toRow(5, "aaa")
+    );
+    Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*"));
+  }
+
+  @Test
+  public void testPositionDeleteWriter() throws IOException {
+    Assume.assumeFalse("ORC delete files are not supported", fileFormat == FileFormat.ORC);
+
+    WriterFactory<T> writerFactory = newWriterFactory(table.schema());
+
+    // write a data file
+    DataFile dataFile = writeData(writerFactory, dataRows, table.spec(), partition);
+
+    // write a position delete file
+    List<PositionDelete<T>> deletes = ImmutableList.of(
+        new PositionDelete<T>().set(dataFile.path(), 0L, null),
+        new PositionDelete<T>().set(dataFile.path(), 2L, null),
+        new PositionDelete<T>().set(dataFile.path(), 4L, null)
+    );
+    Pair<DeleteFile, CharSequenceSet> result = writePositionDeletes(writerFactory, deletes, table.spec(), partition);
+    DeleteFile deleteFile = result.first();
+    CharSequenceSet referencedDataFiles = result.second();
+
+    // verify the written delete file
+    GenericRecord deleteRecord = GenericRecord.create(DeleteSchemaUtil.pathPosSchema());
+    List<Record> expectedDeletes = ImmutableList.of(
+        deleteRecord.copy(DELETE_FILE_PATH.name(), dataFile.path(), DELETE_FILE_POS.name(), 0L),
+        deleteRecord.copy(DELETE_FILE_PATH.name(), dataFile.path(), DELETE_FILE_POS.name(), 2L),
+        deleteRecord.copy(DELETE_FILE_PATH.name(), dataFile.path(), DELETE_FILE_POS.name(), 4L)
+    );
+    InputFile inputDeleteFile = table.io().newInputFile(deleteFile.path().toString());
+    List<Record> actualDeletes = readFile(DeleteSchemaUtil.pathPosSchema(), inputDeleteFile);
+    Assert.assertEquals("Delete records must match", expectedDeletes, actualDeletes);
+
+    // commit the data and delete files
+    table.newRowDelta()
+        .addRows(dataFile)
+        .addDeletes(deleteFile)
+        .validateDataFilesExist(referencedDataFiles)
+        .validateDeletedFiles()
+        .commit();
+
+    // verify the delete file is applied correctly
+    List<T> expectedRows = ImmutableList.of(
+        toRow(2, "aaa"),
+        toRow(4, "aaa")
+    );
+    Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*"));
+  }
+
+  @Test
+  public void testPositionDeleteWriterWithRow() throws IOException {
+    Assume.assumeFalse("ORC delete files are not supported", fileFormat == FileFormat.ORC);

Review comment:
       Yes, I think just because we still don't provide the correct ORC positional/equality delete writers,  but I'm not sure whether there is a potential blockers that prevent us to provide ORC positional/equality delete writer.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677121745



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {
+
+  /**
+   * Creates a new {@link DataWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written data belongs to
+   * @param partition the partition written data belongs to or null if the spec is unpartitioned
+   * @return the constructed data writer
+   */
+  DataWriter<T> newDataWriter(EncryptedOutputFile file, PartitionSpec spec, StructLike partition);
+
+  /**
+   * Creates a new {@link EqualityDeleteWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written deletes belong to
+   * @param partition the partition written deletes belong to or null if the spec is unpartitioned

Review comment:
       Maybe better if we have an extra method for nonpartitioned table? Or are we planning on using `PartitionSpec.unpartitioned()`?
   Just some random ideas, since I am not a big fan of null attributes. 




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677781511



##########
File path: data/src/main/java/org/apache/iceberg/data/BaseWriterFactory.java
##########
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriterFactory;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+/**
+ * A base writer factory to be extended by query engine integrations.
+ */
+public abstract class BaseWriterFactory<T> implements WriterFactory<T> {
+  private final Table table;
+  private final FileFormat dataFileFormat;
+  private final Schema dataSchema;
+  private final SortOrder dataSortOrder;
+  private final FileFormat deleteFileFormat;
+  private final int[] equalityFieldIds;
+  private final Schema equalityDeleteRowSchema;
+  private final SortOrder equalityDeleteSortOrder;
+  private final Schema positionDeleteRowSchema;
+  private final SortOrder positionDeleteSortOrder;
+
+  protected BaseWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema,
+                              SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                              int[] equalityFieldIds, Schema equalityDeleteRowSchema,
+                              SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                              SortOrder positionDeleteSortOrder) {

Review comment:
       I am not sure a position delete is necessarily sorted. I thought it can be unsorted too, even though we will probably ensure it is sorted while writing from Spark.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677135066



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/SparkWriterFactory.java
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.data.BaseWriterFactory;
+import org.apache.iceberg.io.DeleteSchemaUtil;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.data.SparkAvroWriter;
+import org.apache.iceberg.spark.data.SparkOrcWriter;
+import org.apache.iceberg.spark.data.SparkParquetWriters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+import static org.apache.iceberg.MetadataColumns.DELETE_FILE_ROW_FIELD_NAME;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.DELETE_DEFAULT_FILE_FORMAT;
+
+class SparkWriterFactory extends BaseWriterFactory<InternalRow> {
+  private StructType dataSparkType;
+  private StructType equalityDeleteSparkType;
+  private StructType positionDeleteSparkType;
+
+  SparkWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema, StructType dataSparkType,
+                     SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                     int[] equalityFieldIds, Schema equalityDeleteRowSchema, StructType equalityDeleteSparkType,
+                     SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                     StructType positionDeleteSparkType, SortOrder positionDeleteSortOrder) {
+
+    super(table, dataFileFormat, dataSchema, dataSortOrder, deleteFileFormat, equalityFieldIds, equalityDeleteRowSchema,
+        equalityDeleteSortOrder, positionDeleteRowSchema, positionDeleteSortOrder);
+
+    this.dataSparkType = dataSparkType;
+    this.equalityDeleteSparkType = equalityDeleteSparkType;
+    this.positionDeleteSparkType = positionDeleteSparkType;
+  }
+
+  static Builder builderFor(Table table) {
+    return new Builder(table);
+  }
+
+  @Override
+  protected void configureDataWrite(Avro.DataWriteBuilder builder) {
+    builder.createWriterFunc(ignored -> new SparkAvroWriter(dataSparkType()));
+  }
+
+  @Override
+  protected void configureEqualityDelete(Avro.DeleteWriteBuilder builder) {
+    builder.createWriterFunc(ignored -> new SparkAvroWriter(equalityDeleteSparkType()));
+  }
+
+  @Override
+  protected void configurePositionDelete(Avro.DeleteWriteBuilder builder) {
+    boolean withRow = positionDeleteSparkType().getFieldIndex(DELETE_FILE_ROW_FIELD_NAME).isDefined();
+    if (withRow) {
+      // SparkAvroWriter accepts just the Spark type of the row ignoring the path and pos
+      StructField rowField = positionDeleteSparkType().apply(DELETE_FILE_ROW_FIELD_NAME);
+      StructType positionDeleteRowSparkType = (StructType) rowField.dataType();
+      builder.createWriterFunc(ignored -> new SparkAvroWriter(positionDeleteRowSparkType));
+    }
+  }
+
+  @Override
+  protected void configureDataWrite(Parquet.DataWriteBuilder builder) {
+    builder.createWriterFunc(msgType -> SparkParquetWriters.buildWriter(dataSparkType(), msgType));
+  }
+
+  @Override
+  protected void configureEqualityDelete(Parquet.DeleteWriteBuilder builder) {
+    builder.createWriterFunc(msgType -> SparkParquetWriters.buildWriter(equalityDeleteSparkType(), msgType));
+  }
+
+  @Override
+  protected void configurePositionDelete(Parquet.DeleteWriteBuilder builder) {
+    builder.createWriterFunc(msgType -> SparkParquetWriters.buildWriter(positionDeleteSparkType(), msgType));
+    builder.transformPaths(path -> UTF8String.fromString(path.toString()));
+  }
+
+  @Override
+  protected void configureDataWrite(ORC.DataWriteBuilder builder) {
+    builder.createWriterFunc(SparkOrcWriter::new);
+  }
+
+  private StructType dataSparkType() {
+    if (dataSparkType == null) {
+      Preconditions.checkNotNull(dataSchema(), "Data schema must not be null");
+      this.dataSparkType = SparkSchemaUtil.convert(dataSchema());
+    }
+
+    return dataSparkType;
+  }
+
+  private StructType equalityDeleteSparkType() {
+    if (equalityDeleteSparkType == null) {
+      Preconditions.checkNotNull(equalityDeleteRowSchema(), "Equality delete schema must not be null");
+      this.equalityDeleteSparkType = SparkSchemaUtil.convert(equalityDeleteRowSchema());
+    }
+
+    return equalityDeleteSparkType;
+  }
+
+  private StructType positionDeleteSparkType() {
+    if (positionDeleteSparkType == null) {
+      // wrap the optional row schema into the position delete schema that contains path and position
+      Schema positionDeleteSchema = DeleteSchemaUtil.posDeleteSchema(positionDeleteRowSchema());
+      this.positionDeleteSparkType = SparkSchemaUtil.convert(positionDeleteSchema);
+    }
+
+    return positionDeleteSparkType;
+  }
+
+  static class Builder {
+    private final Table table;
+    private FileFormat dataFileFormat;
+    private Schema dataSchema;
+    private StructType dataSparkType;
+    private SortOrder dataSortOrder;
+    private FileFormat deleteFileFormat;
+    private int[] equalityFieldIds;
+    private Schema equalityDeleteRowSchema;
+    private StructType equalityDeleteSparkType;
+    private SortOrder equalityDeleteSortOrder;
+    private Schema positionDeleteRowSchema;
+    private StructType positionDeleteSparkType;
+    private SortOrder positionDeleteSortOrder;
+
+    Builder(Table table) {
+      this.table = table;
+
+      Map<String, String> properties = table.properties();
+
+      String dataFileFormatName = properties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT);
+      this.dataFileFormat = FileFormat.valueOf(dataFileFormatName.toUpperCase(Locale.ENGLISH));
+
+      String deleteFileFormatName = properties.getOrDefault(DELETE_DEFAULT_FILE_FORMAT, dataFileFormatName);
+      this.deleteFileFormat = FileFormat.valueOf(deleteFileFormatName.toUpperCase(Locale.ENGLISH));
+    }
+
+    Builder dataFileFormat(FileFormat newDataFileFormat) {
+      this.dataFileFormat = newDataFileFormat;
+      return this;
+    }
+
+    Builder dataSchema(Schema newDataSchema) {
+      this.dataSchema = newDataSchema;
+      return this;
+    }
+
+    Builder dataSparkType(StructType newDataSparkType) {
+      this.dataSparkType = newDataSparkType;
+      return this;
+    }
+
+    Builder dataSortOrder(SortOrder newDataSortOrder) {
+      this.dataSortOrder = newDataSortOrder;
+      return this;
+    }
+
+    Builder deleteFileFormat(FileFormat newDeleteFileFormat) {
+      this.deleteFileFormat = newDeleteFileFormat;
+      return this;
+    }
+
+    Builder equalityFieldIds(int[] newEqualityFieldIds) {
+      this.equalityFieldIds = newEqualityFieldIds;
+      return this;
+    }
+
+    Builder equalityDeleteRowSchema(Schema newEqualityDeleteRowSchema) {
+      this.equalityDeleteRowSchema = newEqualityDeleteRowSchema;
+      return this;
+    }
+
+    Builder equalityDeleteSparkType(StructType newEqualityDeleteSparkType) {
+      this.equalityDeleteSparkType = newEqualityDeleteSparkType;
+      return this;
+    }
+
+    Builder equalityDeleteSortOrder(SortOrder newEqualityDeleteSortOrder) {
+      this.equalityDeleteSortOrder = newEqualityDeleteSortOrder;
+      return this;
+    }
+
+    Builder positionDeleteRowSchema(Schema newPositionDeleteRowSchema) {
+      this.positionDeleteRowSchema = newPositionDeleteRowSchema;
+      return this;
+    }
+
+    Builder positionDeleteSparkType(StructType newPositionDeleteSparkType) {

Review comment:
       Yeah, correct. One of the reasons I did not introduce a common builder is because we still need some engine specific methods. Let me know if there are any ideas.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677130500



##########
File path: data/src/main/java/org/apache/iceberg/data/BaseWriterFactory.java
##########
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriterFactory;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+/**
+ * A base writer factory to be extended by query engine integrations.
+ */
+public abstract class BaseWriterFactory<T> implements WriterFactory<T> {
+  private final Table table;
+  private final FileFormat dataFileFormat;
+  private final Schema dataSchema;
+  private final SortOrder dataSortOrder;
+  private final FileFormat deleteFileFormat;
+  private final int[] equalityFieldIds;
+  private final Schema equalityDeleteRowSchema;
+  private final SortOrder equalityDeleteSortOrder;
+  private final Schema positionDeleteRowSchema;
+  private final SortOrder positionDeleteSortOrder;
+
+  protected BaseWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema,

Review comment:
       This is an abstract class that should be extended by query engine integrations. There is `SparkWriterFactory` that extends this class and that one has a builder.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r680262805



##########
File path: core/src/main/java/org/apache/iceberg/MetadataColumns.java
##########
@@ -44,6 +44,7 @@ private MetadataColumns() {
       Integer.MAX_VALUE - 101, "file_path", Types.StringType.get(), "Path of a file in which a deleted row is stored");
   public static final NestedField DELETE_FILE_POS = NestedField.required(
       Integer.MAX_VALUE - 102, "pos", Types.LongType.get(), "Ordinal position of a deleted row in the data file");
+  public static final String DELETE_FILE_ROW_FIELD_NAME = "row";

Review comment:
       I don't think so




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677923867



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {
+
+  /**
+   * Creates a new {@link DataWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written data belongs to
+   * @param partition the partition written data belongs to or null if the spec is unpartitioned
+   * @return the constructed data writer
+   */
+  DataWriter<T> newDataWriter(EncryptedOutputFile file, PartitionSpec spec, StructLike partition);
+
+  /**
+   * Creates a new {@link EqualityDeleteWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written deletes belong to
+   * @param partition the partition written deletes belong to or null if the spec is unpartitioned
+   * @return the constructed equality delete writer
+   */
+  EqualityDeleteWriter<T> newEqualityDeleteWriter(EncryptedOutputFile file, PartitionSpec spec, StructLike partition);
+
+  /**
+   * Creates a new {@link PositionDeleteWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written deletes belong to
+   * @param partition the partition written deletes belong to or null if the spec is unpartitioned
+   * @return the constructed position delete writer
+   */
+  PositionDeleteWriter<T> newPositionDeleteWriter(EncryptedOutputFile file, PartitionSpec spec, StructLike partition);

Review comment:
       Nit:  Does the above javadoc format correct ?  I did the javadoc format in my Intellij IDE, it will format like  this: 
   
   ```java
     /**
      * Creates a new {@link PositionDeleteWriter}.
      *
      * @param file      the output file
      * @param spec      the partition spec written deletes belong to
      * @param partition the partition written deletes belong to or null if the spec is unpartitioned
      * @return the constructed position delete writer
      */
     PositionDeleteWriter<T> newPositionDeleteWriter(EncryptedOutputFile file, PartitionSpec spec, StructLike partition);
   ```




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677935084



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {
+
+  /**
+   * Creates a new {@link DataWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written data belongs to

Review comment:
       OK,  @jackye1995 's comment  answered my question perfectly.  Make sense !




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677025324



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {

Review comment:
       I am planning to use `WriterFactory` in subsequent PRs when adding new TaskWriters.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi merged pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi merged pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873


   


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r679486825



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {
+
+  /**
+   * Creates a new {@link DataWriter}.
+   *
+   * @param file the output file
+   * @param spec the partition spec written data belongs to

Review comment:
       To sum up, the spec for deletes should match the data spec of rows we reference.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677024412



##########
File path: core/src/main/java/org/apache/iceberg/avro/Avro.java
##########
@@ -321,6 +321,8 @@ public DataWriteBuilder withSortOrder(SortOrder newSortOrder) {
 
     public <T> DataWriter<T> build() throws IOException {
       Preconditions.checkArgument(spec != null, "Cannot create data writer without spec");
+      Preconditions.checkArgument(spec.isUnpartitioned() || partition != null,

Review comment:
       Extra validation I originally had in the write factory impl but I think it makes sense to move it here.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677780139



##########
File path: data/src/main/java/org/apache/iceberg/data/BaseWriterFactory.java
##########
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriterFactory;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+/**
+ * A base writer factory to be extended by query engine integrations.
+ */
+public abstract class BaseWriterFactory<T> implements WriterFactory<T> {
+  private final Table table;
+  private final FileFormat dataFileFormat;
+  private final Schema dataSchema;
+  private final SortOrder dataSortOrder;
+  private final FileFormat deleteFileFormat;
+  private final int[] equalityFieldIds;
+  private final Schema equalityDeleteRowSchema;
+  private final SortOrder equalityDeleteSortOrder;
+  private final Schema positionDeleteRowSchema;
+  private final SortOrder positionDeleteSortOrder;
+
+  protected BaseWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema,
+                              SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                              int[] equalityFieldIds, Schema equalityDeleteRowSchema,
+                              SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                              SortOrder positionDeleteSortOrder) {

Review comment:
       why is `positionDeleteSortOrder` a configurable input? Shouldn't it always be sorted based on the spec definition?




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677134323



##########
File path: data/src/test/java/org/apache/iceberg/io/TestWriterFactory.java
##########
@@ -0,0 +1,448 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.TableTestBase;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.IcebergGenerics;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataReader;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.StructLikeSet;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.MetadataColumns.DELETE_FILE_PATH;
+import static org.apache.iceberg.MetadataColumns.DELETE_FILE_POS;
+import static org.apache.iceberg.MetadataColumns.DELETE_FILE_ROW_FIELD_NAME;
+
+@RunWith(Parameterized.class)
+public abstract class TestWriterFactory<T> extends TableTestBase {
+  @Parameterized.Parameters(name = "FileFormat={0}, Partitioned={1}")
+  public static Object[] parameters() {
+    return new Object[][] {
+        new Object[]{FileFormat.AVRO, false},
+        new Object[]{FileFormat.AVRO, true},
+        new Object[]{FileFormat.PARQUET, false},
+        new Object[]{FileFormat.PARQUET, true},
+        new Object[]{FileFormat.ORC, false},
+        new Object[]{FileFormat.ORC, true}
+    };
+  }
+
+  private static final int TABLE_FORMAT_VERSION = 2;
+
+  private final FileFormat fileFormat;
+  private final boolean partitioned;
+  private final List<T> dataRows;
+
+  private StructLike partition = null;
+  private OutputFileFactory fileFactory = null;
+
+  public TestWriterFactory(FileFormat fileFormat, boolean partitioned) {
+    super(TABLE_FORMAT_VERSION);
+    this.fileFormat = fileFormat;
+    this.partitioned = partitioned;
+    this.dataRows = ImmutableList.of(
+        toRow(1, "aaa"),
+        toRow(2, "aaa"),
+        toRow(3, "aaa"),
+        toRow(4, "aaa"),
+        toRow(5, "aaa")
+    );
+  }
+
+  protected abstract WriterFactory<T> newWriterFactory(Schema dataSchema, List<Integer> equalityFieldIds,
+                                                       Schema equalityDeleteRowSchema, Schema positionDeleteRowSchema);
+
+  protected abstract T toRow(Integer id, String data);
+
+  protected abstract StructLikeSet toSet(Iterable<T> records);
+
+  protected FileFormat format() {
+    return fileFormat;
+  }
+
+  @Before
+  public void setupTable() throws Exception {
+    this.tableDir = temp.newFolder();
+    Assert.assertTrue(tableDir.delete()); // created during table creation
+
+    this.metadataDir = new File(tableDir, "metadata");
+
+    if (partitioned) {
+      this.table = create(SCHEMA, SPEC);
+      this.partition = initPartitionKey();
+    } else {
+      this.table = create(SCHEMA, PartitionSpec.unpartitioned());
+      this.partition = null;
+    }
+
+    this.fileFactory = OutputFileFactory.builderFor(table, 1, 1).format(fileFormat).build();
+  }
+
+  @Test
+  public void testDataWriter() throws IOException {
+    WriterFactory<T> writerFactory = newWriterFactory(table.schema());
+
+    DataFile dataFile = writeData(writerFactory, dataRows, table.spec(), partition);
+
+    table.newRowDelta()
+        .addRows(dataFile)
+        .commit();
+
+    Assert.assertEquals("Records should match", toSet(dataRows), actualRowSet("*"));
+  }
+
+  @Test
+  public void testEqualityDeleteWriter() throws IOException {
+    Assume.assumeFalse("ORC delete files are not supported", fileFormat == FileFormat.ORC);
+
+    List<Integer> equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId());
+    Schema equalityDeleteRowSchema = table.schema().select("id");
+    WriterFactory<T> writerFactory = newWriterFactory(table.schema(), equalityFieldIds, equalityDeleteRowSchema);
+
+    // write a data file
+    DataFile dataFile = writeData(writerFactory, dataRows, table.spec(), partition);
+
+    // commit the written data file
+    table.newRowDelta()
+        .addRows(dataFile)
+        .commit();
+
+    // write an equality delete file
+    List<T> deletes = ImmutableList.of(
+        toRow(1, "aaa"),
+        toRow(3, "bbb"),
+        toRow(5, "ccc")
+    );
+    DeleteFile deleteFile = writeEqualityDeletes(writerFactory, deletes, table.spec(), partition);
+
+    // verify the written delete file
+    GenericRecord deleteRecord = GenericRecord.create(equalityDeleteRowSchema);
+    List<Record> expectedDeletes = ImmutableList.of(
+        deleteRecord.copy("id", 1),
+        deleteRecord.copy("id", 3),
+        deleteRecord.copy("id", 5)
+    );
+    InputFile inputDeleteFile = table.io().newInputFile(deleteFile.path().toString());
+    List<Record> actualDeletes = readFile(equalityDeleteRowSchema, inputDeleteFile);
+    Assert.assertEquals("Delete records must match", expectedDeletes, actualDeletes);
+
+    // commit the written delete file
+    table.newRowDelta()
+        .addDeletes(deleteFile)
+        .commit();
+
+    // verify the delete file is applied correctly
+    List<T> expectedRows = ImmutableList.of(
+        toRow(2, "aaa"),
+        toRow(4, "aaa")
+    );
+    Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*"));
+  }
+
+  @Test
+  public void testEqualityDeleteWriterWithMultipleSpecs() throws IOException {
+    Assume.assumeFalse("ORC delete files are not supported", fileFormat == FileFormat.ORC);
+    Assume.assumeFalse("Table must start unpartitioned", partitioned);
+
+    List<Integer> equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId());
+    Schema equalityDeleteRowSchema = table.schema().select("id");
+    WriterFactory<T> writerFactory = newWriterFactory(table.schema(), equalityFieldIds, equalityDeleteRowSchema);
+
+    // write an unpartitioned data file
+    DataFile firstDataFile = writeData(writerFactory, dataRows, table.spec(), partition);
+    Assert.assertEquals("First data file must be unpartitioned", 0, firstDataFile.partition().size());
+
+    List<T> deletes = ImmutableList.of(
+        toRow(1, "aaa"),
+        toRow(2, "aaa"),
+        toRow(3, "aaa"),
+        toRow(4, "aaa")
+    );
+
+    // write an unpartitioned delete file
+    DeleteFile firstDeleteFile = writeEqualityDeletes(writerFactory, deletes, table.spec(), partition);
+    Assert.assertEquals("First delete file must be unpartitioned", 0, firstDeleteFile.partition().size());
+
+    // commit the first data and delete files
+    table.newAppend()
+        .appendFile(firstDataFile)
+        .commit();
+    table.newRowDelta()
+        .addDeletes(firstDeleteFile)
+        .commit();
+
+    // evolve the spec
+    table.updateSpec()
+        .addField("data")
+        .commit();
+
+    partition = initPartitionKey();
+
+    // write a partitioned data file
+    DataFile secondDataFile = writeData(writerFactory, dataRows, table.spec(), partition);
+    Assert.assertEquals("Second data file must be partitioned", 1, secondDataFile.partition().size());
+
+    // write a partitioned delete file
+    DeleteFile secondDeleteFile = writeEqualityDeletes(writerFactory, deletes, table.spec(), partition);
+    Assert.assertEquals("Second delete file must be artitioned", 1, secondDeleteFile.partition().size());
+
+    // commit the second data and delete files
+    table.newAppend()
+        .appendFile(secondDataFile)
+        .commit();
+    table.newRowDelta()
+        .addDeletes(secondDeleteFile)
+        .commit();
+
+    // verify both delete files are applied correctly
+    List<T> expectedRows = ImmutableList.of(
+        toRow(5, "aaa"),
+        toRow(5, "aaa")
+    );
+    Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*"));
+  }
+
+  @Test
+  public void testPositionDeleteWriter() throws IOException {
+    Assume.assumeFalse("ORC delete files are not supported", fileFormat == FileFormat.ORC);
+
+    WriterFactory<T> writerFactory = newWriterFactory(table.schema());
+
+    // write a data file
+    DataFile dataFile = writeData(writerFactory, dataRows, table.spec(), partition);
+
+    // write a position delete file
+    List<PositionDelete<T>> deletes = ImmutableList.of(
+        new PositionDelete<T>().set(dataFile.path(), 0L, null),
+        new PositionDelete<T>().set(dataFile.path(), 2L, null),
+        new PositionDelete<T>().set(dataFile.path(), 4L, null)
+    );
+    Pair<DeleteFile, CharSequenceSet> result = writePositionDeletes(writerFactory, deletes, table.spec(), partition);
+    DeleteFile deleteFile = result.first();
+    CharSequenceSet referencedDataFiles = result.second();
+
+    // verify the written delete file
+    GenericRecord deleteRecord = GenericRecord.create(DeleteSchemaUtil.pathPosSchema());
+    List<Record> expectedDeletes = ImmutableList.of(
+        deleteRecord.copy(DELETE_FILE_PATH.name(), dataFile.path(), DELETE_FILE_POS.name(), 0L),
+        deleteRecord.copy(DELETE_FILE_PATH.name(), dataFile.path(), DELETE_FILE_POS.name(), 2L),
+        deleteRecord.copy(DELETE_FILE_PATH.name(), dataFile.path(), DELETE_FILE_POS.name(), 4L)
+    );
+    InputFile inputDeleteFile = table.io().newInputFile(deleteFile.path().toString());
+    List<Record> actualDeletes = readFile(DeleteSchemaUtil.pathPosSchema(), inputDeleteFile);
+    Assert.assertEquals("Delete records must match", expectedDeletes, actualDeletes);
+
+    // commit the data and delete files
+    table.newRowDelta()
+        .addRows(dataFile)
+        .addDeletes(deleteFile)
+        .validateDataFilesExist(referencedDataFiles)
+        .validateDeletedFiles()
+        .commit();
+
+    // verify the delete file is applied correctly
+    List<T> expectedRows = ImmutableList.of(
+        toRow(2, "aaa"),
+        toRow(4, "aaa")
+    );
+    Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*"));
+  }
+
+  @Test
+  public void testPositionDeleteWriterWithRow() throws IOException {
+    Assume.assumeFalse("ORC delete files are not supported", fileFormat == FileFormat.ORC);

Review comment:
       I think we just did not have the proper ORC support when delete writers were implemented.
   I am not aware of any blockers and would love to see ORC delete writers being implemented.
   Anything I missed, @openinx @rdblue?




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677024987



##########
File path: core/src/main/java/org/apache/iceberg/io/WriterFactory.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+
+/**
+ * A factory for creating data and delete writers.
+ */
+public interface WriterFactory<T> {

Review comment:
       The assumption behind this API is that we may only change the spec and partition within a single transactions and other configs such as data/delete file format, sort order are static and known at the factory creation time.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#issuecomment-887110945


   cc @pvary @RussellSpitzer @rdblue @openinx @jun-he @yyanyy @jackye1995 @flyrain @kbendick @karuppayya


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677028329



##########
File path: core/src/main/java/org/apache/iceberg/MetadataColumns.java
##########
@@ -44,6 +44,7 @@ private MetadataColumns() {
       Integer.MAX_VALUE - 101, "file_path", Types.StringType.get(), "Path of a file in which a deleted row is stored");
   public static final NestedField DELETE_FILE_POS = NestedField.required(
       Integer.MAX_VALUE - 102, "pos", Types.LongType.get(), "Ordinal position of a deleted row in the data file");
+  public static final String DELETE_FILE_ROW_FIELD_NAME = "row";

Review comment:
       @rdblue, did we not add the name on purpose?




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677918320



##########
File path: data/src/main/java/org/apache/iceberg/data/BaseWriterFactory.java
##########
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.data;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetricsConfig;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.encryption.EncryptionKeyMetadata;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.WriterFactory;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+/**
+ * A base writer factory to be extended by query engine integrations.
+ */
+public abstract class BaseWriterFactory<T> implements WriterFactory<T> {
+  private final Table table;
+  private final FileFormat dataFileFormat;
+  private final Schema dataSchema;
+  private final SortOrder dataSortOrder;
+  private final FileFormat deleteFileFormat;
+  private final int[] equalityFieldIds;
+  private final Schema equalityDeleteRowSchema;
+  private final SortOrder equalityDeleteSortOrder;
+  private final Schema positionDeleteRowSchema;
+  private final SortOrder positionDeleteSortOrder;
+
+  protected BaseWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema,
+                              SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                              int[] equalityFieldIds, Schema equalityDeleteRowSchema,
+                              SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                              SortOrder positionDeleteSortOrder) {

Review comment:
       @jackye1995 is correct,  the positional delete files are always sorted by the pair <file_path, row_offset>, pls take a look at this PR: https://github.com/apache/iceberg/pull/1858/files#diff-179b5fea5d3aef7c16dd6104c17e1dc53ac9067f13695f3469d21e96c323eb97R126.  So for positional delete files we don't need an extra `SortOrder` now.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677118513



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/SparkWriterFactory.java
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortOrder;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.data.BaseWriterFactory;
+import org.apache.iceberg.io.DeleteSchemaUtil;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.data.SparkAvroWriter;
+import org.apache.iceberg.spark.data.SparkOrcWriter;
+import org.apache.iceberg.spark.data.SparkParquetWriters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+import static org.apache.iceberg.MetadataColumns.DELETE_FILE_ROW_FIELD_NAME;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.DELETE_DEFAULT_FILE_FORMAT;
+
+class SparkWriterFactory extends BaseWriterFactory<InternalRow> {
+  private StructType dataSparkType;
+  private StructType equalityDeleteSparkType;
+  private StructType positionDeleteSparkType;
+
+  SparkWriterFactory(Table table, FileFormat dataFileFormat, Schema dataSchema, StructType dataSparkType,
+                     SortOrder dataSortOrder, FileFormat deleteFileFormat,
+                     int[] equalityFieldIds, Schema equalityDeleteRowSchema, StructType equalityDeleteSparkType,
+                     SortOrder equalityDeleteSortOrder, Schema positionDeleteRowSchema,
+                     StructType positionDeleteSparkType, SortOrder positionDeleteSortOrder) {
+
+    super(table, dataFileFormat, dataSchema, dataSortOrder, deleteFileFormat, equalityFieldIds, equalityDeleteRowSchema,
+        equalityDeleteSortOrder, positionDeleteRowSchema, positionDeleteSortOrder);
+
+    this.dataSparkType = dataSparkType;
+    this.equalityDeleteSparkType = equalityDeleteSparkType;
+    this.positionDeleteSparkType = positionDeleteSparkType;
+  }
+
+  static Builder builderFor(Table table) {
+    return new Builder(table);
+  }
+
+  @Override
+  protected void configureDataWrite(Avro.DataWriteBuilder builder) {
+    builder.createWriterFunc(ignored -> new SparkAvroWriter(dataSparkType()));
+  }
+
+  @Override
+  protected void configureEqualityDelete(Avro.DeleteWriteBuilder builder) {
+    builder.createWriterFunc(ignored -> new SparkAvroWriter(equalityDeleteSparkType()));
+  }
+
+  @Override
+  protected void configurePositionDelete(Avro.DeleteWriteBuilder builder) {
+    boolean withRow = positionDeleteSparkType().getFieldIndex(DELETE_FILE_ROW_FIELD_NAME).isDefined();
+    if (withRow) {
+      // SparkAvroWriter accepts just the Spark type of the row ignoring the path and pos
+      StructField rowField = positionDeleteSparkType().apply(DELETE_FILE_ROW_FIELD_NAME);
+      StructType positionDeleteRowSparkType = (StructType) rowField.dataType();
+      builder.createWriterFunc(ignored -> new SparkAvroWriter(positionDeleteRowSparkType));
+    }
+  }
+
+  @Override
+  protected void configureDataWrite(Parquet.DataWriteBuilder builder) {
+    builder.createWriterFunc(msgType -> SparkParquetWriters.buildWriter(dataSparkType(), msgType));
+  }
+
+  @Override
+  protected void configureEqualityDelete(Parquet.DeleteWriteBuilder builder) {
+    builder.createWriterFunc(msgType -> SparkParquetWriters.buildWriter(equalityDeleteSparkType(), msgType));
+  }
+
+  @Override
+  protected void configurePositionDelete(Parquet.DeleteWriteBuilder builder) {
+    builder.createWriterFunc(msgType -> SparkParquetWriters.buildWriter(positionDeleteSparkType(), msgType));
+    builder.transformPaths(path -> UTF8String.fromString(path.toString()));
+  }
+
+  @Override
+  protected void configureDataWrite(ORC.DataWriteBuilder builder) {
+    builder.createWriterFunc(SparkOrcWriter::new);
+  }
+
+  private StructType dataSparkType() {
+    if (dataSparkType == null) {
+      Preconditions.checkNotNull(dataSchema(), "Data schema must not be null");
+      this.dataSparkType = SparkSchemaUtil.convert(dataSchema());
+    }
+
+    return dataSparkType;
+  }
+
+  private StructType equalityDeleteSparkType() {
+    if (equalityDeleteSparkType == null) {
+      Preconditions.checkNotNull(equalityDeleteRowSchema(), "Equality delete schema must not be null");
+      this.equalityDeleteSparkType = SparkSchemaUtil.convert(equalityDeleteRowSchema());
+    }
+
+    return equalityDeleteSparkType;
+  }
+
+  private StructType positionDeleteSparkType() {
+    if (positionDeleteSparkType == null) {
+      // wrap the optional row schema into the position delete schema that contains path and position
+      Schema positionDeleteSchema = DeleteSchemaUtil.posDeleteSchema(positionDeleteRowSchema());
+      this.positionDeleteSparkType = SparkSchemaUtil.convert(positionDeleteSchema);
+    }
+
+    return positionDeleteSparkType;
+  }
+
+  static class Builder {
+    private final Table table;
+    private FileFormat dataFileFormat;
+    private Schema dataSchema;
+    private StructType dataSparkType;
+    private SortOrder dataSortOrder;
+    private FileFormat deleteFileFormat;
+    private int[] equalityFieldIds;
+    private Schema equalityDeleteRowSchema;
+    private StructType equalityDeleteSparkType;
+    private SortOrder equalityDeleteSortOrder;
+    private Schema positionDeleteRowSchema;
+    private StructType positionDeleteSparkType;
+    private SortOrder positionDeleteSortOrder;
+
+    Builder(Table table) {
+      this.table = table;
+
+      Map<String, String> properties = table.properties();
+
+      String dataFileFormatName = properties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT);
+      this.dataFileFormat = FileFormat.valueOf(dataFileFormatName.toUpperCase(Locale.ENGLISH));
+
+      String deleteFileFormatName = properties.getOrDefault(DELETE_DEFAULT_FILE_FORMAT, dataFileFormatName);
+      this.deleteFileFormat = FileFormat.valueOf(deleteFileFormatName.toUpperCase(Locale.ENGLISH));
+    }
+
+    Builder dataFileFormat(FileFormat newDataFileFormat) {
+      this.dataFileFormat = newDataFileFormat;
+      return this;
+    }
+
+    Builder dataSchema(Schema newDataSchema) {
+      this.dataSchema = newDataSchema;
+      return this;
+    }
+
+    Builder dataSparkType(StructType newDataSparkType) {
+      this.dataSparkType = newDataSparkType;
+      return this;
+    }
+
+    Builder dataSortOrder(SortOrder newDataSortOrder) {
+      this.dataSortOrder = newDataSortOrder;
+      return this;
+    }
+
+    Builder deleteFileFormat(FileFormat newDeleteFileFormat) {
+      this.deleteFileFormat = newDeleteFileFormat;
+      return this;
+    }
+
+    Builder equalityFieldIds(int[] newEqualityFieldIds) {
+      this.equalityFieldIds = newEqualityFieldIds;
+      return this;
+    }
+
+    Builder equalityDeleteRowSchema(Schema newEqualityDeleteRowSchema) {
+      this.equalityDeleteRowSchema = newEqualityDeleteRowSchema;
+      return this;
+    }
+
+    Builder equalityDeleteSparkType(StructType newEqualityDeleteSparkType) {
+      this.equalityDeleteSparkType = newEqualityDeleteSparkType;
+      return this;
+    }
+
+    Builder equalityDeleteSortOrder(SortOrder newEqualityDeleteSortOrder) {
+      this.equalityDeleteSortOrder = newEqualityDeleteSortOrder;
+      return this;
+    }
+
+    Builder positionDeleteRowSchema(Schema newPositionDeleteRowSchema) {
+      this.positionDeleteRowSchema = newPositionDeleteRowSchema;
+      return this;
+    }
+
+    Builder positionDeleteSparkType(StructType newPositionDeleteSparkType) {

Review comment:
       So for Hive we suppose to add our own method? 




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2873: Core: Add WriterFactory

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2873:
URL: https://github.com/apache/iceberg/pull/2873#discussion_r677260920



##########
File path: data/src/test/java/org/apache/iceberg/io/TestWriterFactory.java
##########
@@ -0,0 +1,448 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.TableTestBase;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.IcebergGenerics;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataReader;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
+import org.apache.iceberg.deletes.EqualityDeleteWriter;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.StructLikeSet;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.MetadataColumns.DELETE_FILE_PATH;
+import static org.apache.iceberg.MetadataColumns.DELETE_FILE_POS;
+import static org.apache.iceberg.MetadataColumns.DELETE_FILE_ROW_FIELD_NAME;
+
+@RunWith(Parameterized.class)
+public abstract class TestWriterFactory<T> extends TableTestBase {
+  @Parameterized.Parameters(name = "FileFormat={0}, Partitioned={1}")
+  public static Object[] parameters() {
+    return new Object[][] {
+        new Object[]{FileFormat.AVRO, false},
+        new Object[]{FileFormat.AVRO, true},
+        new Object[]{FileFormat.PARQUET, false},
+        new Object[]{FileFormat.PARQUET, true},
+        new Object[]{FileFormat.ORC, false},
+        new Object[]{FileFormat.ORC, true}
+    };
+  }
+
+  private static final int TABLE_FORMAT_VERSION = 2;
+
+  private final FileFormat fileFormat;
+  private final boolean partitioned;
+  private final List<T> dataRows;
+
+  private StructLike partition = null;
+  private OutputFileFactory fileFactory = null;
+
+  public TestWriterFactory(FileFormat fileFormat, boolean partitioned) {
+    super(TABLE_FORMAT_VERSION);
+    this.fileFormat = fileFormat;
+    this.partitioned = partitioned;
+    this.dataRows = ImmutableList.of(
+        toRow(1, "aaa"),
+        toRow(2, "aaa"),
+        toRow(3, "aaa"),
+        toRow(4, "aaa"),
+        toRow(5, "aaa")
+    );
+  }
+
+  protected abstract WriterFactory<T> newWriterFactory(Schema dataSchema, List<Integer> equalityFieldIds,
+                                                       Schema equalityDeleteRowSchema, Schema positionDeleteRowSchema);
+
+  protected abstract T toRow(Integer id, String data);
+
+  protected abstract StructLikeSet toSet(Iterable<T> records);
+
+  protected FileFormat format() {
+    return fileFormat;
+  }
+
+  @Before
+  public void setupTable() throws Exception {
+    this.tableDir = temp.newFolder();
+    Assert.assertTrue(tableDir.delete()); // created during table creation
+
+    this.metadataDir = new File(tableDir, "metadata");
+
+    if (partitioned) {
+      this.table = create(SCHEMA, SPEC);
+      this.partition = initPartitionKey();
+    } else {
+      this.table = create(SCHEMA, PartitionSpec.unpartitioned());
+      this.partition = null;
+    }
+
+    this.fileFactory = OutputFileFactory.builderFor(table, 1, 1).format(fileFormat).build();
+  }
+
+  @Test
+  public void testDataWriter() throws IOException {
+    WriterFactory<T> writerFactory = newWriterFactory(table.schema());
+
+    DataFile dataFile = writeData(writerFactory, dataRows, table.spec(), partition);
+
+    table.newRowDelta()
+        .addRows(dataFile)
+        .commit();
+
+    Assert.assertEquals("Records should match", toSet(dataRows), actualRowSet("*"));
+  }
+
+  @Test
+  public void testEqualityDeleteWriter() throws IOException {
+    Assume.assumeFalse("ORC delete files are not supported", fileFormat == FileFormat.ORC);
+
+    List<Integer> equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId());
+    Schema equalityDeleteRowSchema = table.schema().select("id");
+    WriterFactory<T> writerFactory = newWriterFactory(table.schema(), equalityFieldIds, equalityDeleteRowSchema);
+
+    // write a data file
+    DataFile dataFile = writeData(writerFactory, dataRows, table.spec(), partition);
+
+    // commit the written data file
+    table.newRowDelta()
+        .addRows(dataFile)
+        .commit();
+
+    // write an equality delete file
+    List<T> deletes = ImmutableList.of(
+        toRow(1, "aaa"),
+        toRow(3, "bbb"),
+        toRow(5, "ccc")
+    );
+    DeleteFile deleteFile = writeEqualityDeletes(writerFactory, deletes, table.spec(), partition);
+
+    // verify the written delete file
+    GenericRecord deleteRecord = GenericRecord.create(equalityDeleteRowSchema);
+    List<Record> expectedDeletes = ImmutableList.of(
+        deleteRecord.copy("id", 1),
+        deleteRecord.copy("id", 3),
+        deleteRecord.copy("id", 5)
+    );
+    InputFile inputDeleteFile = table.io().newInputFile(deleteFile.path().toString());
+    List<Record> actualDeletes = readFile(equalityDeleteRowSchema, inputDeleteFile);
+    Assert.assertEquals("Delete records must match", expectedDeletes, actualDeletes);
+
+    // commit the written delete file
+    table.newRowDelta()
+        .addDeletes(deleteFile)
+        .commit();
+
+    // verify the delete file is applied correctly
+    List<T> expectedRows = ImmutableList.of(
+        toRow(2, "aaa"),
+        toRow(4, "aaa")
+    );
+    Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*"));
+  }
+
+  @Test
+  public void testEqualityDeleteWriterWithMultipleSpecs() throws IOException {

Review comment:
       Thanks for adding this new unit test, looks good to me !




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org