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 2020/12/02 09:37:24 UTC

[GitHub] [iceberg] openinx opened a new pull request #1858: Core: Add SortedPosDeleteWriter

openinx opened a new pull request #1858:
URL: https://github.com/apache/iceberg/pull/1858


   This is a separate issue (from [here](https://github.com/apache/iceberg/pull/1818/files#diff-179b5fea5d3aef7c16dd6104c17e1dc53ac9067f13695f3469d21e96c323eb97R39)) to implement the writer to write the sorted pos-deletes. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
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 #1858: Core: Add SortedPosDeleteWriter

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



##########
File path: data/src/test/java/org/apache/iceberg/io/TestGenericSortedPosDeleteWriter.java
##########
@@ -0,0 +1,315 @@
+/*
+ * 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.Locale;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableTestBase;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.data.GenericAppenderFactory;
+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.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.Lists;
+import org.apache.iceberg.util.StructLikeSet;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestGenericSortedPosDeleteWriter extends TableTestBase {
+  private static final int FORMAT_V2 = 2;
+
+  private final FileFormat format;
+
+  private OutputFileFactory fileFactory;
+  private Record gRecord;
+
+  @Parameterized.Parameters(name = "FileFormat={0}")
+  public static Object[] parameters() {
+    return new Object[][] {
+        new Object[] {"avro"},
+        new Object[] {"parquet"},
+    };
+  }
+
+  public TestGenericSortedPosDeleteWriter(String fileFormat) {
+    super(FORMAT_V2);
+    this.format = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
+  }
+
+  @Before
+  public void setupTable() throws IOException {
+    this.tableDir = temp.newFolder();
+    Assert.assertTrue(tableDir.delete());
+
+    this.metadataDir = new File(tableDir, "metadata");
+    this.table = create(SCHEMA, PartitionSpec.unpartitioned());
+    this.gRecord = GenericRecord.create(SCHEMA);
+
+    this.fileFactory = new OutputFileFactory(table.spec(), format, table.locationProvider(), table.io(),
+        table.encryption(), 1, 1);
+
+    table.updateProperties()
+        .defaultFormat(format)
+        .commit();
+  }
+
+  private EncryptedOutputFile createEncryptedOutputFile() {
+    return fileFactory.newOutputFile();
+  }
+
+  private DataFile prepareDataFile(FileAppenderFactory<Record> appenderFactory, List<Record> rowSet)
+      throws IOException {
+    DataWriter<Record> writer = appenderFactory.newDataWriter(createEncryptedOutputFile(), format, null);
+    try (DataWriter<Record> closeableWriter = writer) {
+      for (Record record : rowSet) {
+        closeableWriter.add(record);
+      }
+    }
+
+    return writer.toDataFile();
+  }
+
+  private Record createRow(Integer id, String data) {
+    Record row = gRecord.copy();
+    row.setField("id", id);
+    row.setField("data", data);
+    return row;
+  }
+
+  private StructLikeSet expectedRowSet(Iterable<Record> records) {
+    StructLikeSet set = StructLikeSet.create(table.schema().asStruct());
+    records.forEach(set::add);
+    return set;
+  }
+
+  private StructLikeSet actualRowSet(String... columns) throws IOException {
+    StructLikeSet set = StructLikeSet.create(table.schema().asStruct());
+    try (CloseableIterable<Record> reader = IcebergGenerics.read(table).select(columns).build()) {
+      reader.forEach(set::add);
+    }
+    return set;
+  }
+
+  @Test
+  public void testSortedPosDelete() throws IOException {
+    List<Record> rowSet = Lists.newArrayList(
+        createRow(0, "aaa"),
+        createRow(1, "bbb"),
+        createRow(2, "ccc"),
+        createRow(3, "ddd"),
+        createRow(4, "eee")
+    );
+
+    FileAppenderFactory<Record> appenderFactory = new GenericAppenderFactory(table.schema(), table.spec(),
+        null, null, null);
+    DataFile dataFile = prepareDataFile(appenderFactory, rowSet);
+
+    SortedPosDeleteWriter<Record> writer = new SortedPosDeleteWriter<>(appenderFactory, fileFactory, format, null, 100);
+    try (SortedPosDeleteWriter<Record> closeableWriter = writer) {
+      for (int index = 0; index < rowSet.size(); index += 2) {

Review comment:
       I think if we delete them in natural order, sorting them or not in delete writer will result in the correct order. Do we want to initialize the index as 4 and decrement the counter to test the sorting logic?  

##########
File path: core/src/main/java/org/apache/iceberg/io/SortedPosDeleteWriter.java
##########
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Comparators;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.CharSequenceWrapper;
+
+class SortedPosDeleteWriter<T> implements Closeable {
+  private static final long DEFAULT_RECORDS_NUM_THRESHOLD = 100_000L;
+
+  private final Map<CharSequenceWrapper, List<PosValue<T>>> posDeletes = Maps.newHashMap();
+  private final List<DeleteFile> completedFiles = Lists.newArrayList();
+  private final Set<CharSequence> referencedDataFiles = CharSequenceSet.empty();
+  private final PosValueComparator<T> posValueComparator = new PosValueComparator<>();
+  private final CharSequenceWrapper wrapper = CharSequenceWrapper.wrap(null);
+
+  private final FileAppenderFactory<T> appenderFactory;
+  private final OutputFileFactory fileFactory;
+  private final FileFormat format;
+  private final PartitionKey partition;
+  private final long recordsNumThreshold;
+
+  private int records = 0;
+
+  SortedPosDeleteWriter(FileAppenderFactory<T> appenderFactory,
+                        OutputFileFactory fileFactory,
+                        FileFormat format,
+                        PartitionKey partition,
+                        long recordsNumThreshold) {
+    this.appenderFactory = appenderFactory;
+    this.fileFactory = fileFactory;
+    this.format = format;
+    this.partition = partition;
+    this.recordsNumThreshold = recordsNumThreshold;
+  }
+
+  SortedPosDeleteWriter(FileAppenderFactory<T> appenderFactory,
+                        OutputFileFactory fileFactory,
+                        FileFormat format,
+                        PartitionKey partition) {
+    this(appenderFactory, fileFactory, format, partition, DEFAULT_RECORDS_NUM_THRESHOLD);
+  }
+
+  public void delete(CharSequence path, long pos) {
+    delete(path, pos, null);
+  }
+
+  public void delete(CharSequence path, long pos, T row) {
+    posDeletes.compute(CharSequenceWrapper.wrap(path), (k, v) -> {
+      if (v == null) {
+        return Lists.newArrayList(PosValue.of(pos, row));
+      } else {
+        v.add(PosValue.of(pos, row));
+        return v;
+      }
+    });
+
+    records += 1;
+
+    // TODO Flush buffer based on the policy that checking whether whole heap memory size exceed the threshold.
+    if (records >= recordsNumThreshold) {
+      flushBuffer();
+    }
+  }
+
+  public List<DeleteFile> complete() throws IOException {
+    close();
+
+    return completedFiles;
+  }
+
+  public Set<CharSequence> referencedDataFiles() {
+    return referencedDataFiles;
+  }
+
+  @Override
+  public void close() throws IOException {
+    flushBuffer();
+  }
+
+  private void flushBuffer() {
+    if (posDeletes.isEmpty()) {
+      return;
+    }
+
+    // Create a new output file.
+    EncryptedOutputFile outputFile;
+    if (partition == null) {
+      outputFile = fileFactory.newOutputFile();
+    } else {
+      outputFile = fileFactory.newOutputFile(partition);
+    }
+
+    PositionDeleteWriter<T> writer = appenderFactory.newPosDeleteWriter(outputFile, format, partition);
+    try (PositionDeleteWriter<T> closeableWriter = writer) {
+      // Sort all the paths.
+      CharSequence[] paths = new CharSequence[posDeletes.size()];
+      int index = 0;
+      for (CharSequenceWrapper charSequenceWrapper : posDeletes.keySet()) {
+        paths[index] = charSequenceWrapper.get();
+        index += 1;
+      }
+      Arrays.sort(paths, Comparators.charSequences());
+
+      // Write all the sorted <path, pos, row> triples.
+      for (CharSequence path : paths) {
+        List<PosValue<T>> positions = posDeletes.get(wrapper.set(path));
+        positions.sort(posValueComparator);

Review comment:
       Nit: could probably be `positions.sort(Comparator.comparingLong(PosValue::pos))`

##########
File path: core/src/main/java/org/apache/iceberg/io/SortedPosDeleteWriter.java
##########
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Comparators;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.CharSequenceWrapper;
+
+class SortedPosDeleteWriter<T> implements Closeable {
+  private static final long DEFAULT_RECORDS_NUM_THRESHOLD = 100_000L;
+
+  private final Map<CharSequenceWrapper, List<PosValue<T>>> posDeletes = Maps.newHashMap();
+  private final List<DeleteFile> completedFiles = Lists.newArrayList();
+  private final Set<CharSequence> referencedDataFiles = CharSequenceSet.empty();
+  private final PosValueComparator<T> posValueComparator = new PosValueComparator<>();
+  private final CharSequenceWrapper wrapper = CharSequenceWrapper.wrap(null);
+
+  private final FileAppenderFactory<T> appenderFactory;
+  private final OutputFileFactory fileFactory;
+  private final FileFormat format;
+  private final PartitionKey partition;
+  private final long recordsNumThreshold;
+
+  private int records = 0;
+
+  SortedPosDeleteWriter(FileAppenderFactory<T> appenderFactory,
+                        OutputFileFactory fileFactory,
+                        FileFormat format,
+                        PartitionKey partition,
+                        long recordsNumThreshold) {
+    this.appenderFactory = appenderFactory;
+    this.fileFactory = fileFactory;
+    this.format = format;
+    this.partition = partition;
+    this.recordsNumThreshold = recordsNumThreshold;
+  }
+
+  SortedPosDeleteWriter(FileAppenderFactory<T> appenderFactory,
+                        OutputFileFactory fileFactory,
+                        FileFormat format,
+                        PartitionKey partition) {
+    this(appenderFactory, fileFactory, format, partition, DEFAULT_RECORDS_NUM_THRESHOLD);
+  }
+
+  public void delete(CharSequence path, long pos) {
+    delete(path, pos, null);
+  }
+
+  public void delete(CharSequence path, long pos, T row) {
+    posDeletes.compute(CharSequenceWrapper.wrap(path), (k, v) -> {

Review comment:
       Nit: `wrapper.set`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
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 #1858: Core: Add SortedPosDeleteWriter

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



##########
File path: core/src/main/java/org/apache/iceberg/io/SortedPosDeleteWriter.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Multimap;
+import org.apache.iceberg.relocated.com.google.common.collect.Multimaps;
+import org.apache.iceberg.types.Comparators;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.CharSequenceWrapper;
+
+class SortedPosDeleteWriter<T> implements Closeable {
+  private static final long DEFAULT_RECORDS_NUM_THRESHOLD = 100_000L;
+
+  private final Multimap<CharSequenceWrapper, PosValue<T>> posDeletes =
+      Multimaps.newMultimap(Maps.newHashMap(), Lists::newArrayList);
+  private final List<DeleteFile> completedFiles = Lists.newArrayList();
+  private final Set<CharSequence> referencedDataFiles = CharSequenceSet.empty();
+  private final CharSequenceWrapper wrapper = CharSequenceWrapper.wrap(null);
+
+  private final FileAppenderFactory<T> appenderFactory;
+  private final OutputFileFactory fileFactory;
+  private final FileFormat format;
+  private final PartitionKey partition;
+  private final long recordsNumThreshold;
+
+  private int records = 0;
+
+  SortedPosDeleteWriter(FileAppenderFactory<T> appenderFactory,
+                        OutputFileFactory fileFactory,
+                        FileFormat format,
+                        PartitionKey partition,
+                        long recordsNumThreshold) {
+    this.appenderFactory = appenderFactory;
+    this.fileFactory = fileFactory;
+    this.format = format;
+    this.partition = partition;
+    this.recordsNumThreshold = recordsNumThreshold;
+  }
+
+  SortedPosDeleteWriter(FileAppenderFactory<T> appenderFactory,
+                        OutputFileFactory fileFactory,
+                        FileFormat format,
+                        PartitionKey partition) {
+    this(appenderFactory, fileFactory, format, partition, DEFAULT_RECORDS_NUM_THRESHOLD);
+  }
+
+  public void delete(CharSequence path, long pos) {
+    delete(path, pos, null);
+  }
+
+  public void delete(CharSequence path, long pos, T row) {
+    posDeletes.put(CharSequenceWrapper.wrap(path), PosValue.of(pos, row));

Review comment:
       Sounds great, it will avoid creating so many small `CharSequenceWrapper` objects because in most cases we have already an existing `path` in this in-memory map. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
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 #1858: Core: Add SortedPosDeleteWriter

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



##########
File path: core/src/main/java/org/apache/iceberg/io/SortedPosDeleteWriter.java
##########
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Comparators;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.CharSequenceWrapper;
+
+class SortedPosDeleteWriter<T> implements Closeable {
+  private static final long DEFAULT_RECORDS_NUM_THRESHOLD = 100_000L;
+
+  private final Map<CharSequenceWrapper, List<PosValue<T>>> posDeletes = Maps.newHashMap();
+  private final List<DeleteFile> completedFiles = Lists.newArrayList();
+  private final Set<CharSequence> referencedDataFiles = CharSequenceSet.empty();
+  private final PosValueComparator<T> posValueComparator = new PosValueComparator<>();
+  private final CharSequenceWrapper wrapper = CharSequenceWrapper.wrap(null);
+
+  private final FileAppenderFactory<T> appenderFactory;
+  private final OutputFileFactory fileFactory;
+  private final FileFormat format;
+  private final PartitionKey partition;
+  private final long recordsNumThreshold;
+
+  private int records = 0;
+
+  SortedPosDeleteWriter(FileAppenderFactory<T> appenderFactory,
+                        OutputFileFactory fileFactory,
+                        FileFormat format,
+                        PartitionKey partition,
+                        long recordsNumThreshold) {
+    this.appenderFactory = appenderFactory;
+    this.fileFactory = fileFactory;
+    this.format = format;
+    this.partition = partition;
+    this.recordsNumThreshold = recordsNumThreshold;
+  }
+
+  SortedPosDeleteWriter(FileAppenderFactory<T> appenderFactory,
+                        OutputFileFactory fileFactory,
+                        FileFormat format,
+                        PartitionKey partition) {
+    this(appenderFactory, fileFactory, format, partition, DEFAULT_RECORDS_NUM_THRESHOLD);
+  }
+
+  public void delete(CharSequence path, long pos) {
+    delete(path, pos, null);
+  }
+
+  public void delete(CharSequence path, long pos, T row) {
+    posDeletes.compute(CharSequenceWrapper.wrap(path), (k, v) -> {

Review comment:
       We could not use `wrapper.set` here  because we will put this item into `map` and if not then other paths also use `wrapper.set` to compare `CharSequence` then the key of map will be messed up.  It's safe to create a new `CharSequenceWrapper` 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.

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 merged pull request #1858: Core: Add SortedPosDeleteWriter

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


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
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 pull request #1858: Core: Add SortedPosDeleteWriter

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


   @openinx, I'm planning on reviewing these PRs over the weekend. Thanks for getting all of this done!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
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 #1858: Core: Add SortedPosDeleteWriter

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



##########
File path: core/src/main/java/org/apache/iceberg/io/SortedPosDeleteWriter.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Multimap;
+import org.apache.iceberg.relocated.com.google.common.collect.Multimaps;
+import org.apache.iceberg.types.Comparators;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.CharSequenceWrapper;
+
+class SortedPosDeleteWriter<T> implements Closeable {
+  private static final long DEFAULT_RECORDS_NUM_THRESHOLD = 100_000L;
+
+  private final Multimap<CharSequenceWrapper, PosValue<T>> posDeletes =
+      Multimaps.newMultimap(Maps.newHashMap(), Lists::newArrayList);
+  private final List<DeleteFile> completedFiles = Lists.newArrayList();
+  private final Set<CharSequence> referencedDataFiles = CharSequenceSet.empty();
+  private final CharSequenceWrapper wrapper = CharSequenceWrapper.wrap(null);
+
+  private final FileAppenderFactory<T> appenderFactory;
+  private final OutputFileFactory fileFactory;
+  private final FileFormat format;
+  private final PartitionKey partition;
+  private final long recordsNumThreshold;
+
+  private int records = 0;
+
+  SortedPosDeleteWriter(FileAppenderFactory<T> appenderFactory,
+                        OutputFileFactory fileFactory,
+                        FileFormat format,
+                        PartitionKey partition,
+                        long recordsNumThreshold) {
+    this.appenderFactory = appenderFactory;
+    this.fileFactory = fileFactory;
+    this.format = format;
+    this.partition = partition;
+    this.recordsNumThreshold = recordsNumThreshold;
+  }
+
+  SortedPosDeleteWriter(FileAppenderFactory<T> appenderFactory,
+                        OutputFileFactory fileFactory,
+                        FileFormat format,
+                        PartitionKey partition) {
+    this(appenderFactory, fileFactory, format, partition, DEFAULT_RECORDS_NUM_THRESHOLD);
+  }
+
+  public void delete(CharSequence path, long pos) {
+    delete(path, pos, null);
+  }
+
+  public void delete(CharSequence path, long pos, T row) {
+    posDeletes.put(CharSequenceWrapper.wrap(path), PosValue.of(pos, row));

Review comment:
       Using a multimap requires this to create a new `CharSequenceWrapper` for each delete, even though we expect a very small set of keys. I think it would be better to use a regular map so that a new wrapper is created for only the first delete in each file:
   
   ```java
     List<PosValue<T>> values = posDeletes.get(wrapper.set(path));
     if (values != null) {
       values.add(PosValue.of(pos, row));
     } else {
       posDeletes.add(Lists.newArrayList(PosValue.of(pos, row)));
     }
     ...
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
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 pull request #1858: Core: Add SortedPosDeleteWriter

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


   Thanks for the fixes, @openinx! I merged 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.

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 #1858: Core: Add SortedPosDeleteWriter

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



##########
File path: core/src/main/java/org/apache/iceberg/io/SortedPosDeleteWriter.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Multimap;
+import org.apache.iceberg.relocated.com.google.common.collect.Multimaps;
+import org.apache.iceberg.types.Comparators;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.CharSequenceWrapper;
+
+class SortedPosDeleteWriter<T> implements Closeable {
+  private static final long DEFAULT_RECORDS_NUM_THRESHOLD = 100_000L;
+
+  private final Multimap<CharSequenceWrapper, PosValue<T>> posDeletes =
+      Multimaps.newMultimap(Maps.newHashMap(), Lists::newArrayList);
+  private final List<DeleteFile> completedFiles = Lists.newArrayList();
+  private final Set<CharSequence> referencedDataFiles = CharSequenceSet.empty();
+  private final CharSequenceWrapper wrapper = CharSequenceWrapper.wrap(null);
+
+  private final FileAppenderFactory<T> appenderFactory;
+  private final OutputFileFactory fileFactory;
+  private final FileFormat format;
+  private final PartitionKey partition;
+  private final long recordsNumThreshold;
+
+  private int records = 0;
+
+  SortedPosDeleteWriter(FileAppenderFactory<T> appenderFactory,
+                        OutputFileFactory fileFactory,
+                        FileFormat format,
+                        PartitionKey partition,
+                        long recordsNumThreshold) {
+    this.appenderFactory = appenderFactory;
+    this.fileFactory = fileFactory;
+    this.format = format;
+    this.partition = partition;
+    this.recordsNumThreshold = recordsNumThreshold;
+  }
+
+  SortedPosDeleteWriter(FileAppenderFactory<T> appenderFactory,
+                        OutputFileFactory fileFactory,
+                        FileFormat format,
+                        PartitionKey partition) {
+    this(appenderFactory, fileFactory, format, partition, DEFAULT_RECORDS_NUM_THRESHOLD);
+  }
+
+  public void delete(CharSequence path, long pos) {
+    delete(path, pos, null);
+  }
+
+  public void delete(CharSequence path, long pos, T row) {
+    posDeletes.put(CharSequenceWrapper.wrap(path), PosValue.of(pos, row));
+    records += 1;
+
+    // TODO Flush buffer based on the policy that checking whether whole heap memory size exceed the threshold.
+    if (records >= recordsNumThreshold) {
+      flushBuffer();
+    }
+  }
+
+  public List<DeleteFile> complete() throws IOException {
+    close();
+
+    return completedFiles;
+  }
+
+  public Set<CharSequence> referencedDataFiles() {
+    return referencedDataFiles;
+  }
+
+  @Override
+  public void close() throws IOException {
+    flushBuffer();
+  }
+
+  private void flushBuffer() {
+    if (posDeletes.isEmpty()) {
+      return;
+    }
+
+    // Create a new output file.
+    EncryptedOutputFile outputFile;
+    if (partition == null) {
+      outputFile = fileFactory.newOutputFile();
+    } else {
+      outputFile = fileFactory.newOutputFile(partition);
+    }
+
+    PositionDeleteWriter<T> writer = appenderFactory.newPosDeleteWriter(outputFile, format, partition);
+    try (PositionDeleteWriter<T> closeableWriter = writer) {
+      // Sort all the paths.
+      List<CharSequence> paths = Lists.newArrayListWithCapacity(posDeletes.keySet().size());
+      for (CharSequenceWrapper charSequenceWrapper : posDeletes.keySet()) {
+        paths.add(charSequenceWrapper.get());
+      }
+      paths.sort(Comparators.charSequences());
+
+      // Write all the sorted <path, pos, row> triples.
+      for (CharSequence path : paths) {
+        List<PosValue<T>> positions = (List<PosValue<T>>) posDeletes.get(wrapper.set(path));
+        positions.sort(Comparator.comparingLong(PosValue::pos));
+
+        positions.forEach(posValue -> closeableWriter.delete(path, posValue.pos(), posValue.row()));
+      }
+    } catch (IOException e) {
+      throw new UncheckedIOException("Failed to write the sorted path/pos pairs to pos-delete file: " +
+          outputFile.encryptingOutputFile().location(), e);
+    }
+
+    // Clear the buffered pos-deletions.
+    posDeletes.clear();
+    records = 0;
+
+    // Add the referenced data files.
+    referencedDataFiles.addAll(writer.referencedDataFiles());
+
+    // Add the completed delete files.
+    completedFiles.add(writer.toDeleteFile());
+  }
+
+  private static class PosValue<R> {

Review comment:
       Minor: This is called `PosValue`, but the values is always used as "row" so it would make more sense to call it `PosRow` or `PosAndRow`. That way references use `posRow.row()` rather than `posValue.row()`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
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 #1858: Core: Add SortedPosDeleteWriter

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



##########
File path: data/src/test/java/org/apache/iceberg/io/TestGenericSortedPosDeleteWriter.java
##########
@@ -0,0 +1,315 @@
+/*
+ * 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.Locale;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableTestBase;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.data.GenericAppenderFactory;
+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.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.Lists;
+import org.apache.iceberg.util.StructLikeSet;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestGenericSortedPosDeleteWriter extends TableTestBase {
+  private static final int FORMAT_V2 = 2;
+
+  private final FileFormat format;
+
+  private OutputFileFactory fileFactory;
+  private Record gRecord;
+
+  @Parameterized.Parameters(name = "FileFormat={0}")
+  public static Object[] parameters() {
+    return new Object[][] {
+        new Object[] {"avro"},
+        new Object[] {"parquet"},
+    };
+  }
+
+  public TestGenericSortedPosDeleteWriter(String fileFormat) {
+    super(FORMAT_V2);
+    this.format = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
+  }
+
+  @Before
+  public void setupTable() throws IOException {
+    this.tableDir = temp.newFolder();
+    Assert.assertTrue(tableDir.delete());
+
+    this.metadataDir = new File(tableDir, "metadata");
+    this.table = create(SCHEMA, PartitionSpec.unpartitioned());
+    this.gRecord = GenericRecord.create(SCHEMA);
+
+    this.fileFactory = new OutputFileFactory(table.spec(), format, table.locationProvider(), table.io(),
+        table.encryption(), 1, 1);
+
+    table.updateProperties()
+        .defaultFormat(format)
+        .commit();
+  }
+
+  private EncryptedOutputFile createEncryptedOutputFile() {
+    return fileFactory.newOutputFile();
+  }
+
+  private DataFile prepareDataFile(FileAppenderFactory<Record> appenderFactory, List<Record> rowSet)
+      throws IOException {
+    DataWriter<Record> writer = appenderFactory.newDataWriter(createEncryptedOutputFile(), format, null);
+    try (DataWriter<Record> closeableWriter = writer) {
+      for (Record record : rowSet) {
+        closeableWriter.add(record);
+      }
+    }
+
+    return writer.toDataFile();
+  }
+
+  private Record createRow(Integer id, String data) {
+    Record row = gRecord.copy();
+    row.setField("id", id);
+    row.setField("data", data);
+    return row;
+  }
+
+  private StructLikeSet expectedRowSet(Iterable<Record> records) {
+    StructLikeSet set = StructLikeSet.create(table.schema().asStruct());
+    records.forEach(set::add);
+    return set;
+  }
+
+  private StructLikeSet actualRowSet(String... columns) throws IOException {
+    StructLikeSet set = StructLikeSet.create(table.schema().asStruct());
+    try (CloseableIterable<Record> reader = IcebergGenerics.read(table).select(columns).build()) {
+      reader.forEach(set::add);
+    }
+    return set;
+  }
+
+  @Test
+  public void testSortedPosDelete() throws IOException {
+    List<Record> rowSet = Lists.newArrayList(
+        createRow(0, "aaa"),
+        createRow(1, "bbb"),
+        createRow(2, "ccc"),
+        createRow(3, "ddd"),
+        createRow(4, "eee")
+    );
+
+    FileAppenderFactory<Record> appenderFactory = new GenericAppenderFactory(table.schema(), table.spec(),
+        null, null, null);
+    DataFile dataFile = prepareDataFile(appenderFactory, rowSet);
+
+    SortedPosDeleteWriter<Record> writer = new SortedPosDeleteWriter<>(appenderFactory, fileFactory, format, null, 100);
+    try (SortedPosDeleteWriter<Record> closeableWriter = writer) {
+      for (int index = rowSet.size() - 1; index >= 0; index -= 2) {
+        closeableWriter.delete(dataFile.path(), index);
+      }
+    }
+
+    List<DeleteFile> deleteFiles = writer.complete();
+    Assert.assertEquals(1, deleteFiles.size());
+    DeleteFile deleteFile = deleteFiles.get(0);
+
+    // Check whether the path-pos pairs are sorted as expected.
+    Schema pathPosSchema = DeleteSchemaUtil.pathPosSchema();
+    Record record = GenericRecord.create(pathPosSchema);
+    List<Record> expectedDeletes = Lists.newArrayList(
+        record.copy("file_path", dataFile.path(), "pos", 0L),
+        record.copy("file_path", dataFile.path(), "pos", 2L),
+        record.copy("file_path", dataFile.path(), "pos", 4L)
+    );
+    Assert.assertEquals(expectedDeletes, readRecordsAsList(pathPosSchema, deleteFile.path()));
+
+    table.newRowDelta()
+        .addRows(dataFile)
+        .addDeletes(deleteFiles.get(0))
+        .validateDataFilesExist(writer.referencedDataFiles())
+        .validateDeletedFiles()
+        .commit();
+
+    List<Record> expectedData = Lists.newArrayList(
+        createRow(1, "bbb"),
+        createRow(3, "ddd")
+    );
+    Assert.assertEquals("Should have the expected records", expectedRowSet(expectedData), actualRowSet("*"));
+  }
+
+  @Test
+  public void testSortedPosDeleteWithRow() throws IOException {
+    List<Record> rowSet = Lists.newArrayList(
+        createRow(0, "aaa"),
+        createRow(1, "bbb"),
+        createRow(2, "ccc"),
+        createRow(3, "ddd"),
+        createRow(4, "eee")
+    );
+
+    FileAppenderFactory<Record> appenderFactory = new GenericAppenderFactory(table.schema(), table.spec(),
+        null, null, table.schema());
+    DataFile dataFile = prepareDataFile(appenderFactory, rowSet);
+
+    SortedPosDeleteWriter<Record> writer = new SortedPosDeleteWriter<>(appenderFactory, fileFactory, format, null, 100);
+    try (SortedPosDeleteWriter<Record> closeableWriter = writer) {
+      for (int index = rowSet.size() - 1; index >= 0; index -= 2) {
+        closeableWriter.delete(dataFile.path(), index, rowSet.get(index)); // Write deletes with row.
+      }
+    }
+
+    List<DeleteFile> deleteFiles = writer.complete();
+    Assert.assertEquals(1, deleteFiles.size());
+    DeleteFile deleteFile = deleteFiles.get(0);
+
+    // Check whether the path-pos pairs are sorted as expected.
+    Schema pathPosSchema = DeleteSchemaUtil.posDeleteSchema(table.schema());
+    Record record = GenericRecord.create(pathPosSchema);
+    List<Record> expectedDeletes = Lists.newArrayList(
+        record.copy("file_path", dataFile.path(), "pos", 0L, "row", createRow(0, "aaa")),
+        record.copy("file_path", dataFile.path(), "pos", 2L, "row", createRow(2, "ccc")),
+        record.copy("file_path", dataFile.path(), "pos", 4L, "row", createRow(4, "eee"))
+    );
+    Assert.assertEquals(expectedDeletes, readRecordsAsList(pathPosSchema, deleteFile.path()));
+
+    table.newRowDelta()
+        .addRows(dataFile)
+        .addDeletes(deleteFiles.get(0))
+        .validateDataFilesExist(writer.referencedDataFiles())
+        .validateDeletedFiles()
+        .commit();
+
+    List<Record> expectedData = Lists.newArrayList(
+        createRow(1, "bbb"),
+        createRow(3, "ddd")
+    );
+    Assert.assertEquals("Should have the expected records", expectedRowSet(expectedData), actualRowSet("*"));
+  }
+
+  @Test
+  public void testMultipleFlush() throws IOException {
+    FileAppenderFactory<Record> appenderFactory = new GenericAppenderFactory(table.schema(), table.spec(),
+        null, null, null);
+
+    // It will produce 5 record lists, each list will write into a separate data file:
+    // The 1th file has: <0  , val-0>   , <1  , val-1>   , ... , <99 , val-99>
+    // The 2th file has: <100, val-100> , <101, val-101> , ... , <199, val-199>
+    // The 3th file has: <200, val-200> , <201, val-201> , ... , <299, val-299>
+    // The 4th file has: <300, val-300> , <301, val-301> , ... , <399, val-399>
+    // The 5th file has: <400, val-400> , <401, val-401> , ... , <499, val-499>

Review comment:
       Thanks, this really helps when reading the test.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
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 #1858: Core: Add SortedPosDeleteWriter

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



##########
File path: core/src/main/java/org/apache/iceberg/io/SortedPosDeleteWriter.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Multimap;
+import org.apache.iceberg.relocated.com.google.common.collect.Multimaps;
+import org.apache.iceberg.types.Comparators;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.CharSequenceWrapper;
+
+class SortedPosDeleteWriter<T> implements Closeable {
+  private static final long DEFAULT_RECORDS_NUM_THRESHOLD = 100_000L;
+
+  private final Multimap<CharSequenceWrapper, PosValue<T>> posDeletes =
+      Multimaps.newMultimap(Maps.newHashMap(), Lists::newArrayList);
+  private final List<DeleteFile> completedFiles = Lists.newArrayList();
+  private final Set<CharSequence> referencedDataFiles = CharSequenceSet.empty();
+  private final CharSequenceWrapper wrapper = CharSequenceWrapper.wrap(null);
+
+  private final FileAppenderFactory<T> appenderFactory;
+  private final OutputFileFactory fileFactory;
+  private final FileFormat format;
+  private final PartitionKey partition;
+  private final long recordsNumThreshold;
+
+  private int records = 0;
+
+  SortedPosDeleteWriter(FileAppenderFactory<T> appenderFactory,
+                        OutputFileFactory fileFactory,
+                        FileFormat format,
+                        PartitionKey partition,
+                        long recordsNumThreshold) {
+    this.appenderFactory = appenderFactory;
+    this.fileFactory = fileFactory;
+    this.format = format;
+    this.partition = partition;
+    this.recordsNumThreshold = recordsNumThreshold;
+  }
+
+  SortedPosDeleteWriter(FileAppenderFactory<T> appenderFactory,
+                        OutputFileFactory fileFactory,
+                        FileFormat format,
+                        PartitionKey partition) {
+    this(appenderFactory, fileFactory, format, partition, DEFAULT_RECORDS_NUM_THRESHOLD);
+  }
+
+  public void delete(CharSequence path, long pos) {
+    delete(path, pos, null);
+  }
+
+  public void delete(CharSequence path, long pos, T row) {
+    posDeletes.put(CharSequenceWrapper.wrap(path), PosValue.of(pos, row));

Review comment:
       Using a multimap requires this to create a new `CharSequenceWrapper` for each delete, even though we expect a very small set of keys. I think it would be better to use a regular map so that a new wrapper is created for only the first delete in each file:
   
   ```java
     List<PosValue<T>> values = posDeletes.get(wrapper.set(path));
     if (values != null) {
       values.add(PosValue.of(pos, row));
     } else {
       posDeletes.put(CharSequenceWrapper.wrap(path), Lists.newArrayList(PosValue.of(pos, row)));
     }
     ...
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
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 #1858: Core: Add SortedPosDeleteWriter

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



##########
File path: core/src/main/java/org/apache/iceberg/io/SortedPosDeleteWriter.java
##########
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Comparators;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.CharSequenceWrapper;
+
+class SortedPosDeleteWriter<T> implements Closeable {
+  private static final long DEFAULT_RECORDS_NUM_THRESHOLD = 100_000L;
+
+  private final Map<CharSequenceWrapper, List<PosValue<T>>> posDeletes = Maps.newHashMap();
+  private final List<DeleteFile> completedFiles = Lists.newArrayList();
+  private final Set<CharSequence> referencedDataFiles = CharSequenceSet.empty();
+  private final PosValueComparator<T> posValueComparator = new PosValueComparator<>();
+  private final CharSequenceWrapper wrapper = CharSequenceWrapper.wrap(null);
+
+  private final FileAppenderFactory<T> appenderFactory;
+  private final OutputFileFactory fileFactory;
+  private final FileFormat format;
+  private final PartitionKey partition;
+  private final long recordsNumThreshold;
+
+  private int records = 0;
+
+  SortedPosDeleteWriter(FileAppenderFactory<T> appenderFactory,
+                        OutputFileFactory fileFactory,
+                        FileFormat format,
+                        PartitionKey partition,
+                        long recordsNumThreshold) {
+    this.appenderFactory = appenderFactory;
+    this.fileFactory = fileFactory;
+    this.format = format;
+    this.partition = partition;
+    this.recordsNumThreshold = recordsNumThreshold;
+  }
+
+  SortedPosDeleteWriter(FileAppenderFactory<T> appenderFactory,
+                        OutputFileFactory fileFactory,
+                        FileFormat format,
+                        PartitionKey partition) {
+    this(appenderFactory, fileFactory, format, partition, DEFAULT_RECORDS_NUM_THRESHOLD);
+  }
+
+  public void delete(CharSequence path, long pos) {
+    delete(path, pos, null);
+  }
+
+  public void delete(CharSequence path, long pos, T row) {
+    posDeletes.compute(CharSequenceWrapper.wrap(path), (k, v) -> {
+      if (v == null) {
+        return Lists.newArrayList(PosValue.of(pos, row));
+      } else {
+        v.add(PosValue.of(pos, row));
+        return v;
+      }
+    });
+
+    records += 1;
+
+    // TODO Flush buffer based on the policy that checking whether whole heap memory size exceed the threshold.
+    if (records >= recordsNumThreshold) {
+      flushBuffer();
+    }
+  }
+
+  public List<DeleteFile> complete() throws IOException {
+    close();
+
+    return completedFiles;
+  }
+
+  public Set<CharSequence> referencedDataFiles() {
+    return referencedDataFiles;
+  }
+
+  @Override
+  public void close() throws IOException {
+    flushBuffer();
+  }
+
+  private void flushBuffer() {
+    if (posDeletes.isEmpty()) {
+      return;
+    }
+
+    // Create a new output file.
+    EncryptedOutputFile outputFile;
+    if (partition == null) {
+      outputFile = fileFactory.newOutputFile();
+    } else {
+      outputFile = fileFactory.newOutputFile(partition);
+    }
+
+    PositionDeleteWriter<T> writer = appenderFactory.newPosDeleteWriter(outputFile, format, partition);
+    try (PositionDeleteWriter<T> closeableWriter = writer) {
+      // Sort all the paths.
+      CharSequence[] paths = new CharSequence[posDeletes.size()];
+      int index = 0;
+      for (CharSequenceWrapper charSequenceWrapper : posDeletes.keySet()) {
+        paths[index] = charSequenceWrapper.get();
+        index += 1;
+      }
+      Arrays.sort(paths, Comparators.charSequences());
+
+      // Write all the sorted <path, pos, row> triples.
+      for (CharSequence path : paths) {
+        List<PosValue<T>> positions = posDeletes.get(wrapper.set(path));
+        positions.sort(posValueComparator);

Review comment:
       That sounds good. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
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 pull request #1858: Core: Add SortedPosDeleteWriter

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


   Overall, looks great! I noted a few things, but I think we should be able to get this in with just a couple fixes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
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 #1858: Core: Add SortedPosDeleteWriter

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



##########
File path: data/src/test/java/org/apache/iceberg/io/TestGenericSortedPosDeleteWriter.java
##########
@@ -0,0 +1,315 @@
+/*
+ * 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.Locale;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableTestBase;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.data.GenericAppenderFactory;
+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.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.Lists;
+import org.apache.iceberg.util.StructLikeSet;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestGenericSortedPosDeleteWriter extends TableTestBase {
+  private static final int FORMAT_V2 = 2;
+
+  private final FileFormat format;
+
+  private OutputFileFactory fileFactory;
+  private Record gRecord;
+
+  @Parameterized.Parameters(name = "FileFormat={0}")
+  public static Object[] parameters() {
+    return new Object[][] {
+        new Object[] {"avro"},
+        new Object[] {"parquet"},
+    };
+  }
+
+  public TestGenericSortedPosDeleteWriter(String fileFormat) {
+    super(FORMAT_V2);
+    this.format = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
+  }
+
+  @Before
+  public void setupTable() throws IOException {
+    this.tableDir = temp.newFolder();
+    Assert.assertTrue(tableDir.delete());
+
+    this.metadataDir = new File(tableDir, "metadata");
+    this.table = create(SCHEMA, PartitionSpec.unpartitioned());
+    this.gRecord = GenericRecord.create(SCHEMA);
+
+    this.fileFactory = new OutputFileFactory(table.spec(), format, table.locationProvider(), table.io(),
+        table.encryption(), 1, 1);
+
+    table.updateProperties()
+        .defaultFormat(format)
+        .commit();
+  }
+
+  private EncryptedOutputFile createEncryptedOutputFile() {
+    return fileFactory.newOutputFile();
+  }
+
+  private DataFile prepareDataFile(FileAppenderFactory<Record> appenderFactory, List<Record> rowSet)
+      throws IOException {
+    DataWriter<Record> writer = appenderFactory.newDataWriter(createEncryptedOutputFile(), format, null);
+    try (DataWriter<Record> closeableWriter = writer) {
+      for (Record record : rowSet) {
+        closeableWriter.add(record);
+      }
+    }
+
+    return writer.toDataFile();
+  }
+
+  private Record createRow(Integer id, String data) {
+    Record row = gRecord.copy();
+    row.setField("id", id);
+    row.setField("data", data);
+    return row;
+  }
+
+  private StructLikeSet expectedRowSet(Iterable<Record> records) {
+    StructLikeSet set = StructLikeSet.create(table.schema().asStruct());
+    records.forEach(set::add);
+    return set;
+  }
+
+  private StructLikeSet actualRowSet(String... columns) throws IOException {
+    StructLikeSet set = StructLikeSet.create(table.schema().asStruct());
+    try (CloseableIterable<Record> reader = IcebergGenerics.read(table).select(columns).build()) {
+      reader.forEach(set::add);
+    }
+    return set;
+  }
+
+  @Test
+  public void testSortedPosDelete() throws IOException {
+    List<Record> rowSet = Lists.newArrayList(
+        createRow(0, "aaa"),
+        createRow(1, "bbb"),
+        createRow(2, "ccc"),
+        createRow(3, "ddd"),
+        createRow(4, "eee")
+    );
+
+    FileAppenderFactory<Record> appenderFactory = new GenericAppenderFactory(table.schema(), table.spec(),
+        null, null, null);
+    DataFile dataFile = prepareDataFile(appenderFactory, rowSet);
+
+    SortedPosDeleteWriter<Record> writer = new SortedPosDeleteWriter<>(appenderFactory, fileFactory, format, null, 100);
+    try (SortedPosDeleteWriter<Record> closeableWriter = writer) {
+      for (int index = rowSet.size() - 1; index >= 0; index -= 2) {
+        closeableWriter.delete(dataFile.path(), index);
+      }
+    }
+
+    List<DeleteFile> deleteFiles = writer.complete();
+    Assert.assertEquals(1, deleteFiles.size());
+    DeleteFile deleteFile = deleteFiles.get(0);
+
+    // Check whether the path-pos pairs are sorted as expected.
+    Schema pathPosSchema = DeleteSchemaUtil.pathPosSchema();
+    Record record = GenericRecord.create(pathPosSchema);
+    List<Record> expectedDeletes = Lists.newArrayList(
+        record.copy("file_path", dataFile.path(), "pos", 0L),
+        record.copy("file_path", dataFile.path(), "pos", 2L),
+        record.copy("file_path", dataFile.path(), "pos", 4L)
+    );
+    Assert.assertEquals(expectedDeletes, readRecordsAsList(pathPosSchema, deleteFile.path()));
+
+    table.newRowDelta()
+        .addRows(dataFile)
+        .addDeletes(deleteFiles.get(0))
+        .validateDataFilesExist(writer.referencedDataFiles())
+        .validateDeletedFiles()
+        .commit();
+
+    List<Record> expectedData = Lists.newArrayList(
+        createRow(1, "bbb"),
+        createRow(3, "ddd")
+    );
+    Assert.assertEquals("Should have the expected records", expectedRowSet(expectedData), actualRowSet("*"));
+  }
+
+  @Test
+  public void testSortedPosDeleteWithRow() throws IOException {
+    List<Record> rowSet = Lists.newArrayList(
+        createRow(0, "aaa"),
+        createRow(1, "bbb"),
+        createRow(2, "ccc"),
+        createRow(3, "ddd"),
+        createRow(4, "eee")
+    );
+
+    FileAppenderFactory<Record> appenderFactory = new GenericAppenderFactory(table.schema(), table.spec(),
+        null, null, table.schema());
+    DataFile dataFile = prepareDataFile(appenderFactory, rowSet);
+
+    SortedPosDeleteWriter<Record> writer = new SortedPosDeleteWriter<>(appenderFactory, fileFactory, format, null, 100);
+    try (SortedPosDeleteWriter<Record> closeableWriter = writer) {
+      for (int index = rowSet.size() - 1; index >= 0; index -= 2) {
+        closeableWriter.delete(dataFile.path(), index, rowSet.get(index)); // Write deletes with row.
+      }
+    }
+
+    List<DeleteFile> deleteFiles = writer.complete();
+    Assert.assertEquals(1, deleteFiles.size());
+    DeleteFile deleteFile = deleteFiles.get(0);
+
+    // Check whether the path-pos pairs are sorted as expected.
+    Schema pathPosSchema = DeleteSchemaUtil.posDeleteSchema(table.schema());
+    Record record = GenericRecord.create(pathPosSchema);
+    List<Record> expectedDeletes = Lists.newArrayList(
+        record.copy("file_path", dataFile.path(), "pos", 0L, "row", createRow(0, "aaa")),
+        record.copy("file_path", dataFile.path(), "pos", 2L, "row", createRow(2, "ccc")),
+        record.copy("file_path", dataFile.path(), "pos", 4L, "row", createRow(4, "eee"))
+    );
+    Assert.assertEquals(expectedDeletes, readRecordsAsList(pathPosSchema, deleteFile.path()));
+
+    table.newRowDelta()
+        .addRows(dataFile)
+        .addDeletes(deleteFiles.get(0))
+        .validateDataFilesExist(writer.referencedDataFiles())
+        .validateDeletedFiles()
+        .commit();
+
+    List<Record> expectedData = Lists.newArrayList(
+        createRow(1, "bbb"),
+        createRow(3, "ddd")
+    );
+    Assert.assertEquals("Should have the expected records", expectedRowSet(expectedData), actualRowSet("*"));
+  }
+
+  @Test
+  public void testMultipleFlush() throws IOException {
+    FileAppenderFactory<Record> appenderFactory = new GenericAppenderFactory(table.schema(), table.spec(),
+        null, null, null);
+
+    // It will produce 5 record lists, each list will write into a separate data file:
+    // The 1th file has: <0  , val-0>   , <1  , val-1>   , ... , <99 , val-99>
+    // The 2th file has: <100, val-100> , <101, val-101> , ... , <199, val-199>
+    // The 3th file has: <200, val-200> , <201, val-201> , ... , <299, val-299>
+    // The 4th file has: <300, val-300> , <301, val-301> , ... , <399, val-399>
+    // The 5th file has: <400, val-400> , <401, val-401> , ... , <499, val-499>
+    List<DataFile> dataFiles = Lists.newArrayList();
+    for (int fileIndex = 0; fileIndex < 5; fileIndex++) {
+      List<Record> recordList = Lists.newLinkedList();
+      for (int recordIndex = 0; recordIndex < 100; recordIndex++) {
+        int id = fileIndex * 100 + recordIndex;
+        recordList.add(createRow(id, String.format("val-%s", id)));
+      }
+
+      // Write the records and generate the data file.
+      dataFiles.add(prepareDataFile(appenderFactory, recordList));
+    }
+
+    // Commit those data files to iceberg table.
+    RowDelta rowDelta = table.newRowDelta();
+    dataFiles.forEach(rowDelta::addRows);
+    rowDelta.commit();
+
+    SortedPosDeleteWriter<Record> writer = new SortedPosDeleteWriter<>(appenderFactory, fileFactory, format, null, 50);
+    try (SortedPosDeleteWriter<Record> closeableWriter = writer) {
+      for (int pos = 0; pos < 100; pos++) { // Delete in DESC order.

Review comment:
       This is ascending order.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
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 #1858: Core: Add SortedPosDeleteWriter

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



##########
File path: data/src/test/java/org/apache/iceberg/io/TestGenericSortedPosDeleteWriter.java
##########
@@ -0,0 +1,315 @@
+/*
+ * 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.Locale;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableTestBase;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.data.GenericAppenderFactory;
+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.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.Lists;
+import org.apache.iceberg.util.StructLikeSet;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestGenericSortedPosDeleteWriter extends TableTestBase {
+  private static final int FORMAT_V2 = 2;
+
+  private final FileFormat format;
+
+  private OutputFileFactory fileFactory;
+  private Record gRecord;
+
+  @Parameterized.Parameters(name = "FileFormat={0}")
+  public static Object[] parameters() {
+    return new Object[][] {
+        new Object[] {"avro"},
+        new Object[] {"parquet"},
+    };
+  }
+
+  public TestGenericSortedPosDeleteWriter(String fileFormat) {
+    super(FORMAT_V2);
+    this.format = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
+  }
+
+  @Before
+  public void setupTable() throws IOException {
+    this.tableDir = temp.newFolder();
+    Assert.assertTrue(tableDir.delete());
+
+    this.metadataDir = new File(tableDir, "metadata");
+    this.table = create(SCHEMA, PartitionSpec.unpartitioned());
+    this.gRecord = GenericRecord.create(SCHEMA);
+
+    this.fileFactory = new OutputFileFactory(table.spec(), format, table.locationProvider(), table.io(),
+        table.encryption(), 1, 1);
+
+    table.updateProperties()
+        .defaultFormat(format)
+        .commit();
+  }
+
+  private EncryptedOutputFile createEncryptedOutputFile() {
+    return fileFactory.newOutputFile();
+  }
+
+  private DataFile prepareDataFile(FileAppenderFactory<Record> appenderFactory, List<Record> rowSet)
+      throws IOException {
+    DataWriter<Record> writer = appenderFactory.newDataWriter(createEncryptedOutputFile(), format, null);
+    try (DataWriter<Record> closeableWriter = writer) {
+      for (Record record : rowSet) {
+        closeableWriter.add(record);
+      }
+    }
+
+    return writer.toDataFile();
+  }
+
+  private Record createRow(Integer id, String data) {
+    Record row = gRecord.copy();
+    row.setField("id", id);
+    row.setField("data", data);
+    return row;
+  }
+
+  private StructLikeSet expectedRowSet(Iterable<Record> records) {
+    StructLikeSet set = StructLikeSet.create(table.schema().asStruct());
+    records.forEach(set::add);
+    return set;
+  }
+
+  private StructLikeSet actualRowSet(String... columns) throws IOException {
+    StructLikeSet set = StructLikeSet.create(table.schema().asStruct());
+    try (CloseableIterable<Record> reader = IcebergGenerics.read(table).select(columns).build()) {
+      reader.forEach(set::add);
+    }
+    return set;
+  }
+
+  @Test
+  public void testSortedPosDelete() throws IOException {

Review comment:
       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.

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 #1858: Core: Add SortedPosDeleteWriter

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



##########
File path: core/src/main/java/org/apache/iceberg/io/SortedPosDeleteWriter.java
##########
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Comparators;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.CharSequenceWrapper;
+
+class SortedPosDeleteWriter<T> implements Closeable {
+  private static final long DEFAULT_RECORDS_NUM_THRESHOLD = 100_000L;
+
+  private final Map<CharSequenceWrapper, List<PosValue<T>>> posDeletes = Maps.newHashMap();
+  private final List<DeleteFile> completedFiles = Lists.newArrayList();
+  private final Set<CharSequence> referencedDataFiles = CharSequenceSet.empty();
+  private final PosValueComparator<T> posValueComparator = new PosValueComparator<>();
+  private final CharSequenceWrapper wrapper = CharSequenceWrapper.wrap(null);
+
+  private final FileAppenderFactory<T> appenderFactory;
+  private final OutputFileFactory fileFactory;
+  private final FileFormat format;
+  private final PartitionKey partition;
+  private final long recordsNumThreshold;
+
+  private int records = 0;
+
+  SortedPosDeleteWriter(FileAppenderFactory<T> appenderFactory,
+                        OutputFileFactory fileFactory,
+                        FileFormat format,
+                        PartitionKey partition,
+                        long recordsNumThreshold) {
+    this.appenderFactory = appenderFactory;
+    this.fileFactory = fileFactory;
+    this.format = format;
+    this.partition = partition;
+    this.recordsNumThreshold = recordsNumThreshold;
+  }
+
+  SortedPosDeleteWriter(FileAppenderFactory<T> appenderFactory,
+                        OutputFileFactory fileFactory,
+                        FileFormat format,
+                        PartitionKey partition) {
+    this(appenderFactory, fileFactory, format, partition, DEFAULT_RECORDS_NUM_THRESHOLD);
+  }
+
+  public void delete(CharSequence path, long pos) {
+    delete(path, pos, null);
+  }
+
+  public void delete(CharSequence path, long pos, T row) {
+    posDeletes.compute(CharSequenceWrapper.wrap(path), (k, v) -> {

Review comment:
       You are right, forgot that we may put the key into map too. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
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 #1858: Core: Add SortedPosDeleteWriter

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



##########
File path: data/src/test/java/org/apache/iceberg/io/TestGenericSortedPosDeleteWriter.java
##########
@@ -0,0 +1,315 @@
+/*
+ * 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.Locale;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableTestBase;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.data.GenericAppenderFactory;
+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.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.Lists;
+import org.apache.iceberg.util.StructLikeSet;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestGenericSortedPosDeleteWriter extends TableTestBase {
+  private static final int FORMAT_V2 = 2;
+
+  private final FileFormat format;
+
+  private OutputFileFactory fileFactory;
+  private Record gRecord;
+
+  @Parameterized.Parameters(name = "FileFormat={0}")
+  public static Object[] parameters() {
+    return new Object[][] {
+        new Object[] {"avro"},
+        new Object[] {"parquet"},
+    };
+  }
+
+  public TestGenericSortedPosDeleteWriter(String fileFormat) {
+    super(FORMAT_V2);
+    this.format = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
+  }
+
+  @Before
+  public void setupTable() throws IOException {
+    this.tableDir = temp.newFolder();
+    Assert.assertTrue(tableDir.delete());
+
+    this.metadataDir = new File(tableDir, "metadata");
+    this.table = create(SCHEMA, PartitionSpec.unpartitioned());
+    this.gRecord = GenericRecord.create(SCHEMA);
+
+    this.fileFactory = new OutputFileFactory(table.spec(), format, table.locationProvider(), table.io(),
+        table.encryption(), 1, 1);
+
+    table.updateProperties()
+        .defaultFormat(format)
+        .commit();
+  }
+
+  private EncryptedOutputFile createEncryptedOutputFile() {
+    return fileFactory.newOutputFile();
+  }
+
+  private DataFile prepareDataFile(FileAppenderFactory<Record> appenderFactory, List<Record> rowSet)
+      throws IOException {
+    DataWriter<Record> writer = appenderFactory.newDataWriter(createEncryptedOutputFile(), format, null);
+    try (DataWriter<Record> closeableWriter = writer) {
+      for (Record record : rowSet) {
+        closeableWriter.add(record);
+      }
+    }
+
+    return writer.toDataFile();
+  }
+
+  private Record createRow(Integer id, String data) {
+    Record row = gRecord.copy();
+    row.setField("id", id);
+    row.setField("data", data);
+    return row;
+  }
+
+  private StructLikeSet expectedRowSet(Iterable<Record> records) {
+    StructLikeSet set = StructLikeSet.create(table.schema().asStruct());
+    records.forEach(set::add);
+    return set;
+  }
+
+  private StructLikeSet actualRowSet(String... columns) throws IOException {
+    StructLikeSet set = StructLikeSet.create(table.schema().asStruct());
+    try (CloseableIterable<Record> reader = IcebergGenerics.read(table).select(columns).build()) {
+      reader.forEach(set::add);
+    }
+    return set;
+  }
+
+  @Test
+  public void testSortedPosDelete() throws IOException {

Review comment:
       Can you also add a test for passing a `null` row when the schema is provided to the appender factory? That should fail because when rows are passed, every row must be passed to ensure the stats are correct.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
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 #1858: Core: Add SortedPosDeleteWriter

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



##########
File path: core/src/main/java/org/apache/iceberg/io/SortedPosDeleteWriter.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.io;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.deletes.PositionDeleteWriter;
+import org.apache.iceberg.encryption.EncryptedOutputFile;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Multimap;
+import org.apache.iceberg.relocated.com.google.common.collect.Multimaps;
+import org.apache.iceberg.types.Comparators;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.CharSequenceWrapper;
+
+class SortedPosDeleteWriter<T> implements Closeable {
+  private static final long DEFAULT_RECORDS_NUM_THRESHOLD = 100_000L;
+
+  private final Multimap<CharSequenceWrapper, PosValue<T>> posDeletes =
+      Multimaps.newMultimap(Maps.newHashMap(), Lists::newArrayList);
+  private final List<DeleteFile> completedFiles = Lists.newArrayList();
+  private final Set<CharSequence> referencedDataFiles = CharSequenceSet.empty();
+  private final CharSequenceWrapper wrapper = CharSequenceWrapper.wrap(null);
+
+  private final FileAppenderFactory<T> appenderFactory;
+  private final OutputFileFactory fileFactory;
+  private final FileFormat format;
+  private final PartitionKey partition;
+  private final long recordsNumThreshold;
+
+  private int records = 0;
+
+  SortedPosDeleteWriter(FileAppenderFactory<T> appenderFactory,
+                        OutputFileFactory fileFactory,
+                        FileFormat format,
+                        PartitionKey partition,
+                        long recordsNumThreshold) {
+    this.appenderFactory = appenderFactory;
+    this.fileFactory = fileFactory;
+    this.format = format;
+    this.partition = partition;
+    this.recordsNumThreshold = recordsNumThreshold;
+  }
+
+  SortedPosDeleteWriter(FileAppenderFactory<T> appenderFactory,
+                        OutputFileFactory fileFactory,
+                        FileFormat format,
+                        PartitionKey partition) {
+    this(appenderFactory, fileFactory, format, partition, DEFAULT_RECORDS_NUM_THRESHOLD);
+  }
+
+  public void delete(CharSequence path, long pos) {
+    delete(path, pos, null);
+  }
+
+  public void delete(CharSequence path, long pos, T row) {
+    posDeletes.put(CharSequenceWrapper.wrap(path), PosValue.of(pos, row));
+    records += 1;
+
+    // TODO Flush buffer based on the policy that checking whether whole heap memory size exceed the threshold.
+    if (records >= recordsNumThreshold) {
+      flushBuffer();
+    }
+  }
+
+  public List<DeleteFile> complete() throws IOException {
+    close();
+
+    return completedFiles;
+  }
+
+  public Set<CharSequence> referencedDataFiles() {
+    return referencedDataFiles;
+  }
+
+  @Override
+  public void close() throws IOException {
+    flushBuffer();
+  }
+
+  private void flushBuffer() {

Review comment:
       There isn't really a buffer in this class, so this name doesn't make sense. How about using `flushBufferedDeletes` or simply `flushDeletes`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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