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 2022/08/20 00:58:32 UTC

[GitHub] [iceberg] wypoon opened a new pull request, #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

wypoon opened a new pull request, #4588:
URL: https://github.com/apache/iceberg/pull/4588

   This is an extension of #4395.
   Here we add a custom metric for the number of delete rows that have been applied in a scan of a format v2 table.
   
   We introduce a counter in `BatchDataReader` and `RowDataReader`, that is incremented when a delete is applied. This counter is passed into `DeleteFilter`, and in the cases where we construct a `PositionDeleteIndex`, is passed into the implementation of the `PositionDeleteIndex`. In all the read paths, the counter is incremented whenever a delete is applied. When Spark calls `currentMetricsValues()` on a `PartitionReader`, which is a subclass of either `BatchDataReader` or `RowDataReader`, we get the current value of the counter and return that.
   
   Tested manually by creating a format v2 table using each of Parquet, ORC, and Avro files, deleting and updating rows in the tables, and reading from the table. The expected number of deletes show up in the Spark UI.
   Also extended existing unit tests (`DeleteReadTests`) to count the number of deletes applied during the scan.
   
   <img width="529" alt="Screen Shot 2022-04-19 at 8 59 31 PM" src="https://user-images.githubusercontent.com/3925490/164147620-7085eafd-304d-45d1-aa53-0c6029638d48.png">
   


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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r876497934


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeleteCount.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.source;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+import org.apache.spark.sql.vectorized.ColumnarBatch;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestSparkReaderDeleteCount extends SparkDeleteReadTestsBase {
+
+  private long deleteCount;
+
+  public TestSparkReaderDeleteCount(String format, boolean vectorized) {
+    super(format, vectorized);
+  }
+
+  @Parameterized.Parameters(name = "format = {0}, vectorized = {1}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] {"parquet", true},
+        new Object[] {"orc", false},
+        new Object[] {"avro", false}
+    };
+  }
+
+  @Override
+  protected boolean countDeletes() {
+    return true;
+  }
+
+  private void setDeleteCount(long count) {
+    deleteCount = count;
+  }
+
+  @Override
+  protected long deleteCount() {
+    return deleteCount;
+  }
+
+  @Override
+  public StructLikeSet rowSet(String name, Table table, String... columns) throws IOException {
+    Schema schema = table.schema().select(columns);
+    StructType sparkSchema = SparkSchemaUtil.convert(schema);
+    Types.StructType type = schema.asStruct();
+    StructLikeSet set = StructLikeSet.create(type);
+
+    SparkScanBuilder scanBuilder = new SparkScanBuilder(spark, table, CaseInsensitiveStringMap.empty());

Review Comment:
   Let me look into that approach.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r856698666


##########
data/src/main/java/org/apache/iceberg/data/DeleteFilter.java:
##########
@@ -67,22 +72,26 @@
   private final List<DeleteFile> eqDeletes;
   private final Schema requiredSchema;
   private final Accessor<StructLike> posAccessor;
+  private final DeleteCounter counter;
 
   private PositionDeleteIndex deleteRowPositions = null;
-  private Predicate<T> eqDeleteRows = null;
 
-  protected DeleteFilter(String filePath, List<DeleteFile> deletes, Schema tableSchema, Schema requestedSchema) {
+  protected DeleteFilter(String filePath, List<DeleteFile> deletes, Schema tableSchema, Schema requestedSchema,
+      DeleteCounter counter) {
     this.setFilterThreshold = DEFAULT_SET_FILTER_THRESHOLD;
     this.filePath = filePath;
+    this.counter = counter;
 
     ImmutableList.Builder<DeleteFile> posDeleteBuilder = ImmutableList.builder();
     ImmutableList.Builder<DeleteFile> eqDeleteBuilder = ImmutableList.builder();
     for (DeleteFile delete : deletes) {
       switch (delete.content()) {
         case POSITION_DELETES:
+          LOG.trace("adding position delete file {} to filter", delete.path());

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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r889373525


##########
data/src/main/java/org/apache/iceberg/data/DeleteFilter.java:
##########
@@ -67,22 +72,26 @@
   private final List<DeleteFile> eqDeletes;
   private final Schema requiredSchema;
   private final Accessor<StructLike> posAccessor;
+  private final DeleteCounter counter;
 
   private PositionDeleteIndex deleteRowPositions = null;
-  private Predicate<T> eqDeleteRows = null;
 
-  protected DeleteFilter(String filePath, List<DeleteFile> deletes, Schema tableSchema, Schema requestedSchema) {
+  protected DeleteFilter(String filePath, List<DeleteFile> deletes, Schema tableSchema, Schema requestedSchema,
+      DeleteCounter counter) {

Review Comment:
   Yes, we need to pass the `DeleteCounter` in. The `DeleteCounter` is created in either `RowDataReader` or `BatchDataReader` and in their `open(FileScanTask)` method,  an instance of their `SparkDeleteFilter` static nested class is created for that `FileScanTask` with the `DeleteCounter`. We need a single counter in either `RowDataReader` or `BatchDataReader` which is used to aggregate the delete count over all the `FileScanTask`s.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r889369323


##########
core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java:
##########
@@ -47,4 +47,8 @@ public boolean isDeleted(long position) {
   public boolean isEmpty() {
     return roaring64Bitmap.isEmpty();
   }
+
+  public long numberOfPositionsDeleted() {

Review Comment:
   Ok, I can put `numberOfPositionsDeleted()` in the `PositionDeleteIndex` interface.



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r888142184


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java:
##########
@@ -152,9 +220,53 @@ public StructLikeSet rowSet(String name, Table table, String... columns) {
       set.add(rowWrapper.wrap(row));
     });
 
+    extractDeleteCount();
     return set;
   }
 
+  private void extractDeleteCount() {
+    // Get the executionId of the query we just executed
+    List<Long> executed = listener.executed();
+    long lastExecuted = executed.get(executed.size() - 1);
+    // Ensure that the execution end was registered

Review Comment:
   Also can you please add this to the Num Splits PR so we can have a few tests in there as well? This will be really useful for all of our future metric additions



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

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

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


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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r905344633


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java:
##########
@@ -196,14 +207,32 @@ public boolean supportColumnarReads(InputPartition partition) {
   }
 
   private static class RowReader extends RowDataReader implements PartitionReader<InternalRow> {
+    private long numSplits;
+
     RowReader(ReadTask task) {
       super(task.task, task.table(), task.expectedSchema(), task.isCaseSensitive());
+      numSplits = task.task.files().size();
+      LOG.debug("Reading {} file split(s) for table {} using RowReader", numSplits, task.table().name());
+    }

Review Comment:
   How about moving `numSplits` to class `BaseDataReader`? This logic can be shared by all subclass of BaseDataReader in that case.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r878938339


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeleteCount.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.source;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+import org.apache.spark.sql.vectorized.ColumnarBatch;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestSparkReaderDeleteCount extends SparkDeleteReadTestsBase {
+
+  private long deleteCount;
+
+  public TestSparkReaderDeleteCount(String format, boolean vectorized) {
+    super(format, vectorized);
+  }
+
+  @Parameterized.Parameters(name = "format = {0}, vectorized = {1}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] {"parquet", true},

Review Comment:
   This is no longer applicable as I have reverted splitting out testing delete count into a separate test class.



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r877638371


##########
core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java:
##########
@@ -40,7 +46,11 @@ public void delete(long posStart, long posEnd) {
 
   @Override
   public boolean isDeleted(long position) {
-    return roaring64Bitmap.contains(position);
+    boolean posIsDeleted = roaring64Bitmap.contains(position);
+    if (counter != null && posIsDeleted) {
+      counter.increment();
+    }

Review Comment:
   I think having "isDeleted" have a side effect ruins the book keeping simplicity for me. I'd much rather if we were to do this counter approach we would the "delete" methods rather than the "isDeleted" method. The delete method already has a side effect and we would expect it to be the place where the count would happen as opposed to "isDeleted" which we have to add a JavaDoc note to for illustrating the side effect behavior. 
   
   I still am not really sold on passing through the counter here, it just seems to me like we are complicating the internals of this class and we don't really need to since like we've already discussed, the value we are looking for is already calculated even without us passing through  an object. I'm also not a big fan of behaviors being dependent on objects being null or not.
   
   But if other reviewers feel differently I could go along with this approach as well.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r956614885


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java:
##########
@@ -181,8 +181,11 @@ Pair<int[], Integer> buildPosDelRowIdMapping(PositionDeleteIndex deletedRowPosit
         if (!deletedRowPositions.isDeleted(originalRowId + rowStartPosInBatch)) {
           posDelRowIdMapping[currentRowId] = originalRowId;
           currentRowId++;
-        } else if (hasIsDeletedColumn) {
-          isDeleted[originalRowId] = true;
+        } else {
+          if (hasIsDeletedColumn) {
+            isDeleted[originalRowId] = true;
+          }
+          deletes.incrementDeleteCount();

Review Comment:
   Sure. I have added blank lines after non-final if blocks.



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

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

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


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


[GitHub] [iceberg] flyrain commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
flyrain commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1235036501

   LGTM. Thanks @wypoon.


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

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

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


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


[GitHub] [iceberg] wypoon commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1236225915

   Thanks @pvary!


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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r950768444


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java:
##########
@@ -183,6 +187,8 @@ Pair<int[], Integer> buildPosDelRowIdMapping(PositionDeleteIndex deletedRowPosit
           currentRowId++;
         } else if (hasIsDeletedColumn) {
           isDeleted[originalRowId] = true;
+        } else {
+          deletes.incrementDeleteCount();
         }

Review Comment:
   @flyrain thank you for explaining how reads with `hasIsDeletedColumn` occur. 
   I didn't really understand that before, but based on @RussellSpitzer's comment I cited, I figured it was fine not to count deletes for such reads. I have done this consistently for both vectorized and non-vectorized reads, as you realize. This is deliberate.
   
   Now that I understand the use case for `hasIsDeletedColumn`, I agree that there is merit to the argument for considering "number of row deletes applied" metric to apply regardless of `hasIsDeletedColumn`. On the other hand, for the queries where `hasIsDeletedColumn` is true, you are able to infer the number of row deletes applied from the result. I don't have a strong opinion on what to do in the `hasIsDeletedColumn` scenario. But the current implementation is consistent. @RussellSpitzer do you want to weigh in? 
   
   If we want to count the deletes always, we'll need to modify `Deletes#markDeleted` (we'll need to pass in the `DeleteCounter`) as well as the changes you mention for `ColumnarBatchReader`.
   



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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r856665447


##########
core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java:
##########
@@ -40,7 +46,11 @@ public void delete(long posStart, long posEnd) {
 
   @Override
   public boolean isDeleted(long position) {
-    return roaring64Bitmap.contains(position);
+    boolean posIsDeleted = roaring64Bitmap.contains(position);
+    if (counter != null && posIsDeleted) {
+      counter.increment();
+    }

Review Comment:
   Also, is there any concern that this makes calls to `isDeleted` not idempotent?



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r855338338


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java:
##########
@@ -47,24 +48,34 @@
 import org.apache.spark.rdd.InputFileBlockHolder;
 import org.apache.spark.sql.catalyst.InternalRow;
 import org.apache.spark.sql.vectorized.ColumnarBatch;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 class BatchDataReader extends BaseDataReader<ColumnarBatch> {
+  private static final Logger LOG = LoggerFactory.getLogger(BatchDataReader.class);
   private final Schema expectedSchema;
   private final String nameMapping;
   private final boolean caseSensitive;
   private final int batchSize;
+  private final DeleteCounter counter;
 
   BatchDataReader(CombinedScanTask task, Table table, Schema expectedSchema, boolean caseSensitive, int size) {
     super(table, task);
     this.expectedSchema = expectedSchema;
     this.nameMapping = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING);
     this.caseSensitive = caseSensitive;
     this.batchSize = size;
+    this.counter = new DeleteCounter();
+  }
+
+  protected DeleteCounter counter() {
+    return counter;
   }
 
   @Override
   CloseableIterator<ColumnarBatch> open(FileScanTask task) {
     DataFile file = task.file();
+    LOG.debug("opening data file {}", file.path());

Review Comment:
   Start with a capital 



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r856475347


##########
data/src/main/java/org/apache/iceberg/data/DeleteFilter.java:
##########
@@ -172,7 +185,8 @@ public CloseableIterable<T> findEqualityDeleteRows(CloseableIterable<T> records)
     Filter<T> deletedRowsFilter = new Filter<T>() {
       @Override
       protected boolean shouldKeep(T item) {
-        return deletedRows.test(item);
+        boolean keep = deletedRows.test(item); // true means row is deleted

Review Comment:
   Reverted.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r856688857


##########
core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java:
##########
@@ -40,7 +46,11 @@ public void delete(long posStart, long posEnd) {
 
   @Override
   public boolean isDeleted(long position) {
-    return roaring64Bitmap.contains(position);
+    boolean posIsDeleted = roaring64Bitmap.contains(position);
+    if (counter != null && posIsDeleted) {
+      counter.increment();
+    }

Review Comment:
   There are existing tests that have no need for counting deletes, that call some static methods in `Deletes`. I kept those forms of the methods and had them call new forms that take a `DeleteCounter`, passing a null `DeleteCounter`. That is the reason why a `DeleteCounter` field might be null. The way the code is now, if we decide to do without `DeleteCounter` and simply use an `AtomicLong`, then it can be updated very easily. I do understand your suggestion for a `NullDeleteCounter`.
   
   You raise a good point about `PositionDeleteIndex#isDeleted` not being idempotent now. I have checked where it is called, and it is not a concern. We should document this point though, to prevent it being a problem in future code. Before I ended up introducing a delete counter into the `PositionDeleteIndex` implementation and incrementing it when `isDeleted` is called, I considered other approaches, but there was one read path in particular where this seemed to the best solution.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r856705589


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskNumDeletes.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.source.metrics;
+
+import org.apache.spark.sql.connector.metric.CustomTaskMetric;
+
+public class TaskNumDeletes implements CustomTaskMetric {
+  private long value;

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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r888050255


##########
data/src/main/java/org/apache/iceberg/data/DeleteFilter.java:
##########
@@ -96,6 +105,10 @@ protected DeleteFilter(String filePath, List<DeleteFile> deletes, Schema tableSc
     this.posAccessor = requiredSchema.accessorForField(MetadataColumns.ROW_POSITION.fieldId());
   }
 
+  protected DeleteFilter(String filePath, List<DeleteFile> deletes, Schema tableSchema, Schema requestedSchema) {

Review Comment:
   Is this just for legacy compatibility? Do we have any users of this here? as a protected method I would think we would be free to drop it if we wanted to.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r887367193


##########
data/src/main/java/org/apache/iceberg/data/DeleteFilter.java:
##########
@@ -54,35 +56,42 @@
 import org.apache.iceberg.util.Filter;
 import org.apache.iceberg.util.StructLikeSet;
 import org.apache.iceberg.util.StructProjection;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public abstract class DeleteFilter<T> {
-  private static final long DEFAULT_SET_FILTER_THRESHOLD = 100_000L;
+  private static final Logger LOG = LoggerFactory.getLogger(DeleteFilter.class);
+  public static final long DEFAULT_STREAM_FILTER_THRESHOLD = 100_000L;

Review Comment:
   I will revert the last commit from this PR. If this PR can be merged without that, I'll open a separate PR once it is merged, consisting of that change.



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r887303132


##########
data/src/main/java/org/apache/iceberg/data/DeleteFilter.java:
##########
@@ -54,35 +56,42 @@
 import org.apache.iceberg.util.Filter;
 import org.apache.iceberg.util.StructLikeSet;
 import org.apache.iceberg.util.StructProjection;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public abstract class DeleteFilter<T> {
-  private static final long DEFAULT_SET_FILTER_THRESHOLD = 100_000L;
+  private static final Logger LOG = LoggerFactory.getLogger(DeleteFilter.class);
+  public static final long DEFAULT_STREAM_FILTER_THRESHOLD = 100_000L;

Review Comment:
   Can we extract all of these set/streamFilter changes out into another PR?



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

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

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


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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r950762704


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java:
##########
@@ -183,6 +187,8 @@ Pair<int[], Integer> buildPosDelRowIdMapping(PositionDeleteIndex deletedRowPosit
           currentRowId++;
         } else if (hasIsDeletedColumn) {
           isDeleted[originalRowId] = true;
+        } else {
+          deletes.incrementDeleteCount();
         }

Review Comment:
   It is triggered by either projection or filtering. Here are two examples:
   ```
   select name, _deleted from student
   ```
   It returns both deleted and un-delete rows.
   ```
   select name from student where _deleted=true
   ```
   It returns deleted rows.
   
   The way I saw it is that the delete count metric is orthogonal to the metadata column `_deleted`. It reports how many row-level deletes applied as its label indicates, `public static final String DISPLAY_STRING = "number of row deletes applied"`. I don’t have strong option on it though. What do you think? BTW, the non-vectorized read also didn’t report the metric in case of `_deleted` column, see line 255 in class `DeleteFilter`.



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

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

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


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


[GitHub] [iceberg] wypoon commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1234948098

   Ok, local build succeeds, but I guess the CI must rebase on master; even though there are no conflicts, the build fails, because of a new class added by #5578 and #5682.


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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r956355988


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java:
##########
@@ -130,16 +140,19 @@ protected Table createTable(String name, Schema schema, PartitionSpec spec) {
     TableOperations ops = ((BaseTable) table).operations();
     TableMetadata meta = ops.current();
     ops.commit(meta, meta.upgradeToFormatVersion(2));
-    if (vectorized) {
+    table.updateProperties().set(TableProperties.DEFAULT_FILE_FORMAT, format).commit();
+    if (format.equals("parquet") && vectorized) {
       table
           .updateProperties()
           .set(TableProperties.PARQUET_VECTORIZATION_ENABLED, "true")
           .set(
               TableProperties.PARQUET_BATCH_SIZE,
               "4") // split 7 records to two batches to cover more code paths
           .commit();
-    } else {
+    } else if (format.equals("parquet")) { // in this case, non-vectorized
       table.updateProperties().set(TableProperties.PARQUET_VECTORIZATION_ENABLED, "false").commit();
+    } else if (format.equals("orc")) { // we only have non-vectorized for orc in our parameters
+      table.updateProperties().set(TableProperties.ORC_VECTORIZATION_ENABLED, "false").commit();

Review Comment:
   Thanks for the suggestion. I have adopted a variant of it.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r950626188


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java:
##########
@@ -130,7 +140,8 @@ protected Table createTable(String name, Schema schema, PartitionSpec spec) {
     TableOperations ops = ((BaseTable) table).operations();
     TableMetadata meta = ops.current();
     ops.commit(meta, meta.upgradeToFormatVersion(2));
-    if (vectorized) {
+    table.updateProperties().set(TableProperties.DEFAULT_FILE_FORMAT, format).commit();
+    if (format.equals("parquet") && vectorized) {

Review Comment:
   I could have been mistaken, but when I started working on this, vectorized read was only supported for parquet. For this reason, the parameters for the testsuite only has vectorized=true for format=parquet. In any case, the parameters give a fair coverage of both file formats and vectorized/non-vectorized.
   
   I think I understand your concern. If I understand it correctly, your concern is with `TableProperties.ORC_VECTORIZATION_ENABLED` which currently defaults to false. If format=orc, we want to set `TableProperties.ORC_VECTORIZATION_ENABLED` explicitly in case the default changes in the future.
   
   I'll fix this.



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

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

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


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


[GitHub] [iceberg] wypoon commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1206672889

   I have ported the changes to Spark 3.3. The Spark 3.3 changes are identical to the Spark 3.2 changes (the affected files are identical at this point).


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

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

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


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


[GitHub] [iceberg] wypoon commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1235812314

   @flyrain have you sorted out your credentials? can you or another committer merge this now? Thanks.


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

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

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


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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r905365893


##########
core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java:
##########
@@ -47,4 +47,8 @@ public boolean isDeleted(long position) {
   public boolean isEmpty() {
     return roaring64Bitmap.isEmpty();
   }
+
+  public long numberOfPositionsDeleted() {

Review Comment:
   Looking a bit more, we don't need this due to https://github.com/apache/iceberg/pull/4588/files#r905365603.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r950623070


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java:
##########
@@ -183,6 +187,8 @@ Pair<int[], Integer> buildPosDelRowIdMapping(PositionDeleteIndex deletedRowPosit
           currentRowId++;
         } else if (hasIsDeletedColumn) {
           isDeleted[originalRowId] = true;
+        } else {
+          deletes.incrementDeleteCount();
         }

Review Comment:
   So, following the explanation by @RussellSpitzer: 
   
   > there is just now a second read path with _is_deleted metadata column which actually returns deleted rows. When we follow that path I'm not sure it's important for us to count the deleted rows since we'll be returning them anyway, I could go either way. 
   
   I decided that for this read path with the _id_deleted metadata column, we will not perform the count. That is what has been implemented here.
   
   Pardon my ignorance: for my benefit, how does one perform such a read (where `hasIsDeletedColumn` is true)?



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r950622138


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java:
##########
@@ -170,6 +173,7 @@ Pair<int[], Integer> posDelRowIdMapping() {
      * @return the mapping array and the new num of rows in a batch, null if no row is deleted
      */
     Pair<int[], Integer> buildPosDelRowIdMapping(PositionDeleteIndex deletedRowPositions) {
+      LOG.debug("Building row id mapping from positional deletes");

Review Comment:
   I'm fine with removing the debug logging for the method calls in `ColumnBatchReader`. I'll do so.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r952039537


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java:
##########
@@ -261,7 +268,10 @@ protected InputFile getInputFile(String location) {
 
     @Override
     protected void markRowDeleted(InternalRow row) {
-      row.setBoolean(columnIsDeletedPosition(), true);
+      if (!row.getBoolean(columnIsDeletedPosition())) {
+        row.setBoolean(columnIsDeletedPosition(), true);
+        counter().increment();
+      }

Review Comment:
   `markRowDeleted` can be called as the `Consumer` in `Deletes.markDeleted(CloseableIterable<T>, Predicate<T>, Consumer<T>)` via `DeleteFilter#applyPosDeletes(CloseableIterable<T>)` and via `DeleteFilter#applyEqDeletes(CloseableIterable<T>)`.
   I assume we want to avoid counting twice when the same row is deleted via a positional delete and then via an equality delete.
   Thus we check to see if the row is already marked deleted, and if it is not, then we mark it and increment the count.
   This count is tested in `TestSparkReaderDeletes#testMixedPosAndEqDeletesWithDeletedColumn`.



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r876420067


##########
core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java:
##########
@@ -40,7 +46,11 @@ public void delete(long posStart, long posEnd) {
 
   @Override
   public boolean isDeleted(long position) {
-    return roaring64Bitmap.contains(position);
+    boolean posIsDeleted = roaring64Bitmap.contains(position);
+    if (counter != null && posIsDeleted) {
+      counter.increment();
+    }

Review Comment:
   Shouldn't we be able to determine the number of deleted rows here by checking the number of entries in the bitmap? So shouldn't we be able to just count in the "delete methods"? Also do we have to pass through the counter here? Seems like we could just get this value out of the index when we are done building it instead of passing through the counter into the index?



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r888122799


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java:
##########
@@ -152,9 +220,53 @@ public StructLikeSet rowSet(String name, Table table, String... columns) {
       set.add(rowWrapper.wrap(row));
     });
 
+    extractDeleteCount();
     return set;
   }
 
+  private void extractDeleteCount() {
+    // Get the executionId of the query we just executed
+    List<Long> executed = listener.executed();
+    long lastExecuted = executed.get(executed.size() - 1);
+    // Ensure that the execution end was registered

Review Comment:
   Oh this is great, Looking into the apis you dug up I think there may be an even easier way to do this. Sorry I didn't realize you could do this before
   
   Looking into sharedState you can do something like
   ```scala
   // Some operation
   scala> spark.range(1, 100).withColumn("x", col("id")).withColumnRenamed("id", "y").writeTo("local.default.test").append
   
   // UI Metrics
   scala> spark.sharedState.statusStore.executionsList.last.metrics
   res41: Seq[org.apache.spark.sql.execution.ui.SQLPlanMetric] = List(SQLPlanMetric(number of output rows,25,sum), SQLPlanMetric(duration,24,timing))
   
   // Description
   scala> spark.sharedState.statusStore.executionsList.last.description
   res42: String = append at <console>:23
   
   // Execution id
   scala> spark.sharedState.statusStore.executionsList.last.executionId
   res47: Long = 6
   ```
   
   So this block could be like
   ```java
     private long lastExecutedMetric(String metricName) {
       SQLExecutionUIData lastExecution = spark.sharedState().statusStore().executionsList().last();
       Option<SQLPlanMetric> sqlPlanMetric = lastExecution.metrics().find(metric -> metric.name().equals(metricName));
       Assert.assertTrue("Metric xxxx not present in last execution", sqlPlanMetric.isDefined());
       long metricId = sqlPlanMetric.get().accumulatorId();
   
       // Refresh metricValues, they will remain null until the execution is complete and metrics are aggregated
       int attempts = 3;
       while (lastExecution.metricValues() == null && attempts > 0) {
         try {
           Thread.sleep(100);
           attempts -= 1;
         } catch (InterruptedException e) {
           throw new RuntimeException(e);
         }
         lastExecution = spark.sharedState().statusStore().execution(lastExecution.executionId()).get();
       }
       Assert.assertNotNull(String.format("Metric %s was never finalized", metricName), lastExecution.metricValues());
       return Long.parseLong(lastExecution.metricValues().get(metricId).get());
     }
     ```
     
     Then we just have
     ```java
     @Override
     protected void checkDeleteCount(long expectedDeletes) {
       Assert.assertEquals("Unexpected number of deletes recorded", expectedDeletes,
           lastExecutedMetric(NumDeletes.DISPLAY_STRING));
     }
     ```
     
     And we can drop the other state in the test class.
   



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

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

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


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


[GitHub] [iceberg] pvary commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
pvary commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r960236343


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/SparkSQLExecutionHelper.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.source;
+
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.execution.ui.SQLAppStatusStore;
+import org.apache.spark.sql.execution.ui.SQLExecutionUIData;
+import org.apache.spark.sql.execution.ui.SQLPlanMetric;
+import org.junit.Assert;
+import scala.Option;
+
+public class SparkSQLExecutionHelper {
+
+  private SparkSQLExecutionHelper() {}
+
+  /**
+   * Finds the value of a specified metric for the last SQL query that was executed. Metric values
+   * are stored in the `SQLAppStatusStore` as strings.
+   *
+   * @param spark SparkSession used to run the SQL query
+   * @param metricName name of the metric
+   * @return value of the metric
+   */
+  public static String lastExecutedMetricValue(SparkSession spark, String metricName) {
+    SQLAppStatusStore statusStore = spark.sharedState().statusStore();
+    SQLExecutionUIData lastExecution = statusStore.executionsList().last();
+    Option<SQLPlanMetric> sqlPlanMetric =
+        lastExecution.metrics().find(metric -> metric.name().equals(metricName));
+    Assert.assertTrue(
+        String.format("Metric '%s' not found in last execution", metricName),
+        sqlPlanMetric.isDefined());
+    long metricId = sqlPlanMetric.get().accumulatorId();
+
+    // Refresh metricValues, they will remain null until the execution is complete and metrics are
+    // aggregated
+    int attempts = 3;
+    while (lastExecution.metricValues() == null && attempts > 0) {
+      try {
+        Thread.sleep(100);
+        attempts -= 1;

Review Comment:
   Nit: Why not `attempts--`  or `--attempts`?



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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r956502145


##########
core/src/main/java/org/apache/iceberg/deletes/DeleteCounter.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.deletes;
+
+/** A counter to be used to count deletes as they are applied. */
+public class DeleteCounter {
+
+  private long count = 0L;
+
+  /** Increment the counter by one. */
+  public void increment() {
+    count++;
+  }
+
+  /** Return the current value of the counter. */
+  public long get() {
+    return count;
+  }

Review Comment:
   I can’t recall if this was covered or not already, but is there a reason we can’t use `Counter` from the `metrics` package in api?
   
   I think this work might have been done before the `Counter` class from api existed, but there’s also very likely a reason to stick with this simpler class (for Spark serde or something).
   
   https://github.com/apache/iceberg/blob/master/api/src/main/java/org/apache/iceberg/metrics/Counter.java



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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r956508165


##########
data/src/test/java/org/apache/iceberg/data/DeleteReadTests.java:
##########
@@ -297,11 +322,14 @@ public void testEqualityDeletesSpanningMultipleDataFiles() throws IOException {
             table, Files.localOutput(temp.newFile()), Row.of(0), dataDeletes, deleteRowSchema);
 
     table.newRowDelta().addDeletes(eqDeletes).commit();
+    // At ths point, the table has (7 - 3) + (8 - 4) = 8 rows. 7 rows in all are deleted.
 
     StructLikeSet expected = rowSetWithoutIds(table, records, 29, 89, 122, 144);
     StructLikeSet actual = rowSet(tableName, table, "*");
 
     Assert.assertEquals("Table should contain expected rows", expected, actual);
+    // 3 deletes in the first data file and 4 deletes in the second data file

Review Comment:
   Nit / Non-blocking: this comment seems redundant given we’ve mentioned the 7 deletes previously. But it’s not incorrect so I’m not opposed to keeping it.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r938042980


##########
core/src/main/java/org/apache/iceberg/deletes/Deletes.java:
##########
@@ -90,27 +103,45 @@ public static StructLikeSet toEqualitySet(CloseableIterable<StructLike> eqDelete
   }
 
   public static <T extends StructLike> PositionDeleteIndex toPositionIndex(CharSequence dataLocation,
-                                                                           List<CloseableIterable<T>> deleteFiles) {
+                                                                           List<CloseableIterable<T>> deleteFiles,
+                                                                           DeleteCounter counter) {
     DataFileFilter<T> locationFilter = new DataFileFilter<>(dataLocation);
     List<CloseableIterable<Long>> positions = Lists.transform(deleteFiles, deletes ->
         CloseableIterable.transform(locationFilter.filter(deletes), row -> (Long) POSITION_ACCESSOR.get(row)));
-    return toPositionIndex(CloseableIterable.concat(positions));
+    return toPositionIndex(CloseableIterable.concat(positions), counter);
   }
 
-  public static PositionDeleteIndex toPositionIndex(CloseableIterable<Long> posDeletes) {
+  public static PositionDeleteIndex toPositionIndex(CloseableIterable<Long> posDeletes, DeleteCounter counter) {
     try (CloseableIterable<Long> deletes = posDeletes) {
-      PositionDeleteIndex positionDeleteIndex = new BitmapPositionDeleteIndex();
+      BitmapPositionDeleteIndex positionDeleteIndex = new BitmapPositionDeleteIndex();
       deletes.forEach(positionDeleteIndex::delete);
+      counter.increment(positionDeleteIndex.numberOfPositionsDeleted());

Review Comment:
   Thanks for pointing this out. I have fixed this.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r937370595


##########
core/src/main/java/org/apache/iceberg/deletes/Deletes.java:
##########
@@ -202,17 +233,24 @@ boolean isDeleted(T row) {
   }
 
   private static class PositionStreamDeleteFilter<T> extends PositionStreamDeleteIterable<T> {
-    private PositionStreamDeleteFilter(CloseableIterable<T> rows, Function<T, Long> rowToPosition,
-                                       CloseableIterable<Long> deletePositions) {
+    private final DeleteCounter counter;
+
+    PositionStreamDeleteFilter(CloseableIterable<T> rows, Function<T, Long> rowToPosition,
+                               CloseableIterable<Long> deletePositions, DeleteCounter counter) {
       super(rows, rowToPosition, deletePositions);
+      this.counter = counter;
     }
 
     @Override
     protected CloseableIterator<T> applyDelete(CloseableIterator<T> items) {
       return new FilterIterator<T>(items) {
         @Override
         protected boolean shouldKeep(T item) {
-          return !isDeleted(item);
+          boolean deleted = isDeleted(item);
+          if (deleted) {
+            counter.increment();
+          }
+          return !deleted;

Review Comment:
   I ran `IcebergSourceParquetPosDeleteBenchmark` before and after this change, and the numbers for `readIceberg` show hardly any difference.



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r855340336


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java:
##########
@@ -75,6 +86,7 @@ CloseableIterator<ColumnarBatch> open(FileScanTask task) {
     InputFile location = getInputFile(task);
     Preconditions.checkNotNull(location, "Could not find InputFile associated with FileScanTask");
     if (task.file().format() == FileFormat.PARQUET) {
+      LOG.debug("data file is parquet; calling deleteFilter to get SparkDeleteFilter ...");

Review Comment:
   Capital? But maybe remove this one, not sure it's adding a lot of value since you could just add the line in the `deleteFilter` code



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r855352508


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskNumDeletes.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.source.metrics;
+
+import org.apache.spark.sql.connector.metric.CustomTaskMetric;
+
+public class TaskNumDeletes implements CustomTaskMetric {
+  private long value;

Review Comment:
   nit: final



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r855386771


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumDeletes.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.source.metrics;
+
+import java.text.NumberFormat;
+import org.apache.spark.sql.connector.metric.CustomMetric;
+
+public class NumDeletes implements CustomMetric {
+
+  @Override
+  public String name() {
+    return "numDeletes";
+  }
+
+  @Override
+  public String description() {
+    return "number of row deletes applied";

Review Comment:
   I use a lowercase "number" to be consistent with other (standard) metrics appearing in the Spark UI (e.g., "number of output rows"). It would look odd otherwise.



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r855349710


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumDeletes.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.source.metrics;
+
+import java.text.NumberFormat;
+import org.apache.spark.sql.connector.metric.CustomMetric;
+
+public class NumDeletes implements CustomMetric {
+
+  @Override
+  public String name() {
+    return "numDeletes";
+  }
+
+  @Override
+  public String description() {
+    return "number of row deletes applied";

Review Comment:
   Capital first



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r855348221


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java:
##########
@@ -193,15 +204,41 @@ public boolean supportColumnarReads(InputPartition partition) {
     }
   }
 
+  static long numFilesToScan(CombinedScanTask scanTask) {
+    long fileCount = 0L;
+    for (FileScanTask file : scanTask.files()) {
+      fileCount += 1L;
+    }
+    return fileCount;

Review Comment:
   why not just scanTask.files().size?



##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java:
##########
@@ -193,15 +204,41 @@ public boolean supportColumnarReads(InputPartition partition) {
     }
   }
 
+  static long numFilesToScan(CombinedScanTask scanTask) {
+    long fileCount = 0L;
+    for (FileScanTask file : scanTask.files()) {
+      fileCount += 1L;
+    }
+    return fileCount;

Review Comment:
   How about scanTask.files().size?



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r889370489


##########
core/src/main/java/org/apache/iceberg/deletes/Deletes.java:
##########
@@ -84,27 +89,46 @@ public static StructLikeSet toEqualitySet(CloseableIterable<StructLike> eqDelete
   }
 
   public static <T extends StructLike> PositionDeleteIndex toPositionIndex(CharSequence dataLocation,
-                                                                           List<CloseableIterable<T>> deleteFiles) {
+                                                                           List<CloseableIterable<T>> deleteFiles,
+                                                                           DeleteCounter counter) {
     DataFileFilter<T> locationFilter = new DataFileFilter<>(dataLocation);
     List<CloseableIterable<Long>> positions = Lists.transform(deleteFiles, deletes ->
         CloseableIterable.transform(locationFilter.filter(deletes), row -> (Long) POSITION_ACCESSOR.get(row)));
-    return toPositionIndex(CloseableIterable.concat(positions));
+    return toPositionIndex(CloseableIterable.concat(positions), counter);
   }
 
-  public static PositionDeleteIndex toPositionIndex(CloseableIterable<Long> posDeletes) {
+  public static PositionDeleteIndex toPositionIndex(CloseableIterable<Long> posDeletes, DeleteCounter counter) {
     try (CloseableIterable<Long> deletes = posDeletes) {
       PositionDeleteIndex positionDeleteIndex = new BitmapPositionDeleteIndex();
       deletes.forEach(positionDeleteIndex::delete);
+      counter.increment(((BitmapPositionDeleteIndex) positionDeleteIndex).numberOfPositionsDeleted());
       return positionDeleteIndex;
     } catch (IOException e) {
       throw new UncheckedIOException("Failed to close position delete source", e);
     }
   }
 
+  public static <T extends StructLike> PositionDeleteIndex toPositionIndex(CharSequence dataLocation,

Review Comment:
   Yes, there are there to support existing API calls.



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r888047897


##########
core/src/main/java/org/apache/iceberg/deletes/Deletes.java:
##########
@@ -84,27 +89,46 @@ public static StructLikeSet toEqualitySet(CloseableIterable<StructLike> eqDelete
   }
 
   public static <T extends StructLike> PositionDeleteIndex toPositionIndex(CharSequence dataLocation,
-                                                                           List<CloseableIterable<T>> deleteFiles) {
+                                                                           List<CloseableIterable<T>> deleteFiles,
+                                                                           DeleteCounter counter) {
     DataFileFilter<T> locationFilter = new DataFileFilter<>(dataLocation);
     List<CloseableIterable<Long>> positions = Lists.transform(deleteFiles, deletes ->
         CloseableIterable.transform(locationFilter.filter(deletes), row -> (Long) POSITION_ACCESSOR.get(row)));
-    return toPositionIndex(CloseableIterable.concat(positions));
+    return toPositionIndex(CloseableIterable.concat(positions), counter);
   }
 
-  public static PositionDeleteIndex toPositionIndex(CloseableIterable<Long> posDeletes) {
+  public static PositionDeleteIndex toPositionIndex(CloseableIterable<Long> posDeletes, DeleteCounter counter) {
     try (CloseableIterable<Long> deletes = posDeletes) {
       PositionDeleteIndex positionDeleteIndex = new BitmapPositionDeleteIndex();
       deletes.forEach(positionDeleteIndex::delete);
+      counter.increment(((BitmapPositionDeleteIndex) positionDeleteIndex).numberOfPositionsDeleted());
       return positionDeleteIndex;
     } catch (IOException e) {
       throw new UncheckedIOException("Failed to close position delete source", e);
     }
   }
 
+  public static <T extends StructLike> PositionDeleteIndex toPositionIndex(CharSequence dataLocation,

Review Comment:
   Dow we need these polymorphisms? Or is this just to keep apis? 



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r890649671


##########
core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java:
##########
@@ -47,4 +47,8 @@ public boolean isDeleted(long position) {
   public boolean isEmpty() {
     return roaring64Bitmap.isEmpty();
   }
+
+  public long numberOfPositionsDeleted() {

Review Comment:
   Hmm, turns out there is a `TestSparkParquetReadMetadataColumns.CustomizedPositionDeleteIndex` that implements `PositionDeleteIndex`. I'll leave the interface alone then, since I don't want to change unrelated tests.



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

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

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


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


[GitHub] [iceberg] wypoon commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1205525395

   @RussellSpitzer @flyrain I have rebased on master. @flyrain I have fixed the bug you pointed out; thank you for pointing it out.
   Please review this again. I kept the changes in Spark 3.2, since I was rebasing. I can port the changes to Spark 3.3 in this same PR (and am inclined to do so), if the changes look good to you.


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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r956509199


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java:
##########
@@ -49,6 +53,7 @@ protected Stream<ContentFile<?>> referencedFiles(FileScanTask task) {
   @Override
   protected CloseableIterator<ColumnarBatch> open(FileScanTask task) {
     String filePath = task.file().path().toString();
+    LOG.debug("Opening data file {}", filePath);

Review Comment:
   Nit: This debug log could arguably be removed, but I wouldn’t block on this comment. It could be good to have more verbose debug logs.



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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r956503339


##########
core/src/main/java/org/apache/iceberg/deletes/DeleteCounter.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.deletes;
+
+/** A counter to be used to count deletes as they are applied. */
+public class DeleteCounter {
+
+  private long count = 0L;
+
+  /** Increment the counter by one. */
+  public void increment() {
+    count++;
+  }
+
+  /** Return the current value of the counter. */
+  public long get() {
+    return count;
+  }

Review Comment:
   I’m not sure how the entire `MetricsContext` integrates with Spark and executors etc.
   
   If we don’t have existing work that uses `MetricsContext` for custom Spark metrics (particularly task level metrics), possibly we can update this code in a later PR.
   
   I know Steven handled a lot of the metrics context adjustments for Flink and @nastra did a lot of the work for that in general.



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

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

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


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


[GitHub] [iceberg] pvary commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
pvary commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r960232409


##########
core/src/main/java/org/apache/iceberg/deletes/Deletes.java:
##########
@@ -74,9 +75,24 @@ public static <T> CloseableIterable<T> markDeleted(
         });
   }
 
+  /**
+   * Returns the remaining rows (the ones that are not deleted), while counting the deleted ones.

Review Comment:
   Please add the description for the parameters 



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r956615382


##########
data/src/test/java/org/apache/iceberg/data/DeleteReadTests.java:
##########
@@ -282,6 +306,7 @@ public void testEqualityDeletesSpanningMultipleDataFiles() throws IOException {
         FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), Row.of(0), records);
 
     table.newAppend().appendFile(dataFile).commit();
+    // At this point, the table has 7 + 8 = 15 rows, of which all but one are in duplicate.

Review Comment:
   Perhaps I'm idiosyncratic. I put the comment after the statement because the "at this point" refers to the state after the statement has executed and not before. (If the comment were short enough, I would have put it on the same line as the statement, after it. That would probably be more natural.)
   
   Nevertheless, I have now put the comments before the statements.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r961198548


##########
core/src/main/java/org/apache/iceberg/deletes/Deletes.java:
##########
@@ -62,6 +62,7 @@ public static <T> CloseableIterable<T> filter(
     return equalityFilter.filter(rows);
   }
 
+  /** Returns the same rows that are input, while marking the deleted ones. */

Review Comment:
   Fixed.



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r855334416


##########
data/src/main/java/org/apache/iceberg/data/DeleteFilter.java:
##########
@@ -67,22 +72,26 @@
   private final List<DeleteFile> eqDeletes;
   private final Schema requiredSchema;
   private final Accessor<StructLike> posAccessor;
+  private final DeleteCounter counter;
 
   private PositionDeleteIndex deleteRowPositions = null;
-  private Predicate<T> eqDeleteRows = null;
 
-  protected DeleteFilter(String filePath, List<DeleteFile> deletes, Schema tableSchema, Schema requestedSchema) {
+  protected DeleteFilter(String filePath, List<DeleteFile> deletes, Schema tableSchema, Schema requestedSchema,
+      DeleteCounter counter) {
     this.setFilterThreshold = DEFAULT_SET_FILTER_THRESHOLD;
     this.filePath = filePath;
+    this.counter = counter;
 
     ImmutableList.Builder<DeleteFile> posDeleteBuilder = ImmutableList.builder();
     ImmutableList.Builder<DeleteFile> eqDeleteBuilder = ImmutableList.builder();
     for (DeleteFile delete : deletes) {
       switch (delete.content()) {
         case POSITION_DELETES:
+          LOG.trace("adding position delete file {} to filter", delete.path());

Review Comment:
   nit: Start with a capital letter



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r855343011


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java:
##########
@@ -126,7 +138,8 @@ CloseableIterator<ColumnarBatch> open(FileScanTask task) {
   }
 
   private SparkDeleteFilter deleteFilter(FileScanTask task) {
-    return task.deletes().isEmpty() ? null : new SparkDeleteFilter(task, table().schema(), expectedSchema);
+    LOG.debug("is task.deletes() empty? {}", task.deletes().isEmpty());

Review Comment:
   I would probably just break up the ternary operator below and log when creating the Delete filter?



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r855351510


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumSplits.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.source.metrics;
+
+import java.text.NumberFormat;
+import org.apache.spark.sql.connector.metric.CustomMetric;
+
+public class NumSplits implements CustomMetric {

Review Comment:
   Also should be CustomSumMetric
   https://spark.apache.org/docs/latest/api/java/org/apache/spark/sql/connector/metric/CustomSumMetric.html



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r855346754


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java:
##########
@@ -77,6 +77,11 @@ class SparkCopyOnWriteScan extends SparkScan implements SupportsRuntimeFiltering
     }
   }
 
+  @Override
+  public String description() {
+    return "IcebergCopyOnWriteScan " + super.description();

Review Comment:
   Probably separate this into another PR



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r856700831


##########
data/src/main/java/org/apache/iceberg/data/DeleteFilter.java:
##########
@@ -226,17 +261,18 @@ private CloseableIterable<T> applyPosDeletes(CloseableIterable<T> records) {
 
     // if there are fewer deletes than a reasonable number to keep in memory, use a set
     if (posDeletes.stream().mapToLong(DeleteFile::recordCount).sum() < setFilterThreshold) {
-      return Deletes.filter(records, this::pos, Deletes.toPositionIndex(filePath, deletes));
+      return Deletes.filter(records, this::pos, Deletes.toPositionIndex(filePath, deletes, counter));
     }
 
-    return Deletes.streamingFilter(records, this::pos, Deletes.deletePositions(filePath, deletes));
+    return Deletes.streamingFilter(records, this::pos, Deletes.deletePositions(filePath, deletes), counter);
   }
 
   private CloseableIterable<Record> openPosDeletes(DeleteFile file) {
     return openDeletes(file, POS_DELETE_SCHEMA);
   }
 
   private CloseableIterable<Record> openDeletes(DeleteFile deleteFile, Schema deleteSchema) {
+    LOG.trace("opening delete file {}", deleteFile.path());

Review Comment:
   Done.



##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java:
##########
@@ -47,24 +48,34 @@
 import org.apache.spark.rdd.InputFileBlockHolder;
 import org.apache.spark.sql.catalyst.InternalRow;
 import org.apache.spark.sql.vectorized.ColumnarBatch;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 class BatchDataReader extends BaseDataReader<ColumnarBatch> {
+  private static final Logger LOG = LoggerFactory.getLogger(BatchDataReader.class);
   private final Schema expectedSchema;
   private final String nameMapping;
   private final boolean caseSensitive;
   private final int batchSize;
+  private final DeleteCounter counter;
 
   BatchDataReader(CombinedScanTask task, Table table, Schema expectedSchema, boolean caseSensitive, int size) {
     super(table, task);
     this.expectedSchema = expectedSchema;
     this.nameMapping = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING);
     this.caseSensitive = caseSensitive;
     this.batchSize = size;
+    this.counter = new DeleteCounter();
+  }
+
+  protected DeleteCounter counter() {
+    return counter;
   }
 
   @Override
   CloseableIterator<ColumnarBatch> open(FileScanTask task) {
     DataFile file = task.file();
+    LOG.debug("opening data file {}", file.path());

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

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

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


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


[GitHub] [iceberg] wypoon commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1134010849

   @kbendick and @RussellSpitzer thank you for your reviews and suggestions.
   I have removed `DeleteCounter` from `BitmapPositionDeleteIndex`. I have implemented the suggestion to extract the custom metric from Spark after running a DataFrame query (although this turned out to be trickier than expected due to async issues (that did not surface locally but surfaced in the CI)). I think the PR is much better now.


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

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

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


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


[GitHub] [iceberg] wypoon commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1204340488

   I'm going to rebase on master again and conform to the new spotless code style so that this can merge.


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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r951818287


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java:
##########
@@ -170,6 +173,7 @@ Pair<int[], Integer> posDelRowIdMapping() {
      * @return the mapping array and the new num of rows in a batch, null if no row is deleted
      */
     Pair<int[], Integer> buildPosDelRowIdMapping(PositionDeleteIndex deletedRowPositions) {
+      LOG.debug("Building row id mapping from positional deletes");

Review Comment:
   Removed.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r951818627


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java:
##########
@@ -217,6 +223,7 @@ int[] initEqDeleteRowIdMapping() {
      * @param columnarBatch the {@link ColumnarBatch} to apply the equality delete
      */
     void applyEqDelete(ColumnarBatch columnarBatch) {
+      LOG.debug("Applying equality deletes to row id mapping");

Review Comment:
   Removed.



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

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

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


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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r945957184


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java:
##########
@@ -170,6 +173,7 @@ Pair<int[], Integer> posDelRowIdMapping() {
      * @return the mapping array and the new num of rows in a batch, null if no row is deleted
      */
     Pair<int[], Integer> buildPosDelRowIdMapping(PositionDeleteIndex deletedRowPositions) {
+      LOG.debug("Building row id mapping from positional deletes");

Review Comment:
   Nit: is the log necessary? There would be a lot of messages in a real use case when debug level is on. I doubt how useful it is. 



##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java:
##########
@@ -130,7 +140,8 @@ protected Table createTable(String name, Schema schema, PartitionSpec spec) {
     TableOperations ops = ((BaseTable) table).operations();
     TableMetadata meta = ops.current();
     ops.commit(meta, meta.upgradeToFormatVersion(2));
-    if (vectorized) {
+    table.updateProperties().set(TableProperties.DEFAULT_FILE_FORMAT, format).commit();
+    if (format.equals("parquet") && vectorized) {

Review Comment:
   This is all about parquet format. We may do this 
   ```
    if (format.equals("parquet")) {
      if (vectorized) {
         table
             .updateProperties()
             .set(TableProperties.PARQUET_VECTORIZATION_ENABLED, "true")
             .set(
                 TableProperties.PARQUET_BATCH_SIZE,
                 "4") // split 7 records to two batches to cover more code paths
             .commit();
       } else {
         table.updateProperties().set(TableProperties.PARQUET_VECTORIZATION_ENABLED, "false").commit();
       }
   }
   ```
   Also since we added orc and avro, we may set the vectorized property explicitly like we did for `parquet` other than using the default value. The default value may change in the future, which will break the tests.
   
   



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java:
##########
@@ -183,6 +187,8 @@ Pair<int[], Integer> buildPosDelRowIdMapping(PositionDeleteIndex deletedRowPosit
           currentRowId++;
         } else if (hasIsDeletedColumn) {
           isDeleted[originalRowId] = true;
+        } else {
+          deletes.incrementDeleteCount();
         }

Review Comment:
   We still need the metrics in case of `_deleted` column, right?
   ```
   else {
        if (hasIsDeletedColumn) {
             isDeleted[originalRowId] = true;
        }
       deletes.incrementDeleteCount();
   }
   ```



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java:
##########
@@ -217,6 +223,7 @@ int[] initEqDeleteRowIdMapping() {
      * @param columnarBatch the {@link ColumnarBatch} to apply the equality delete
      */
     void applyEqDelete(ColumnarBatch columnarBatch) {
+      LOG.debug("Applying equality deletes to row id mapping");

Review Comment:
   The same comment here.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java:
##########
@@ -229,6 +236,8 @@ void applyEqDelete(ColumnarBatch columnarBatch) {
           currentRowId++;
         } else if (hasIsDeletedColumn) {
           isDeleted[rowIdMapping[rowId]] = true;
+        } else {
+          deletes.incrementDeleteCount();

Review Comment:
   The same comment here.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java:
##########
@@ -77,6 +78,7 @@
   private final NameMapping nameMapping;
   private final ScanTaskGroup<TaskT> taskGroup;
   private final Iterator<TaskT> tasks;
+  private final DeleteCounter counter;

Review Comment:
   +1 to put it here



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r956615479


##########
data/src/test/java/org/apache/iceberg/data/DeleteReadTests.java:
##########
@@ -297,11 +322,14 @@ public void testEqualityDeletesSpanningMultipleDataFiles() throws IOException {
             table, Files.localOutput(temp.newFile()), Row.of(0), dataDeletes, deleteRowSchema);
 
     table.newRowDelta().addDeletes(eqDeletes).commit();
+    // At ths point, the table has (7 - 3) + (8 - 4) = 8 rows. 7 rows in all are deleted.
 
     StructLikeSet expected = rowSetWithoutIds(table, records, 29, 89, 122, 144);
     StructLikeSet actual = rowSet(tableName, table, "*");
 
     Assert.assertEquals("Table should contain expected rows", expected, actual);
+    // 3 deletes in the first data file and 4 deletes in the second data file

Review Comment:
   Removed this comment as above comment has made it clear how many deletes there are.



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

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

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


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


[GitHub] [iceberg] pvary commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
pvary commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r960235769


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/SparkSQLExecutionHelper.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.source;
+
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.execution.ui.SQLAppStatusStore;
+import org.apache.spark.sql.execution.ui.SQLExecutionUIData;
+import org.apache.spark.sql.execution.ui.SQLPlanMetric;
+import org.junit.Assert;
+import scala.Option;
+
+public class SparkSQLExecutionHelper {
+
+  private SparkSQLExecutionHelper() {}
+
+  /**
+   * Finds the value of a specified metric for the last SQL query that was executed. Metric values
+   * are stored in the `SQLAppStatusStore` as strings.
+   *
+   * @param spark SparkSession used to run the SQL query
+   * @param metricName name of the metric
+   * @return value of the metric
+   */
+  public static String lastExecutedMetricValue(SparkSession spark, String metricName) {
+    SQLAppStatusStore statusStore = spark.sharedState().statusStore();
+    SQLExecutionUIData lastExecution = statusStore.executionsList().last();
+    Option<SQLPlanMetric> sqlPlanMetric =
+        lastExecution.metrics().find(metric -> metric.name().equals(metricName));
+    Assert.assertTrue(
+        String.format("Metric '%s' not found in last execution", metricName),
+        sqlPlanMetric.isDefined());
+    long metricId = sqlPlanMetric.get().accumulatorId();
+
+    // Refresh metricValues, they will remain null until the execution is complete and metrics are
+    // aggregated
+    int attempts = 3;
+    while (lastExecution.metricValues() == null && attempts > 0) {
+      try {
+        Thread.sleep(100);
+        attempts -= 1;
+      } catch (InterruptedException e) {
+        throw new RuntimeException(e);
+      }

Review Comment:
   Nit: newline 



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

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

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


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


[GitHub] [iceberg] pvary commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
pvary commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r960235203


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumSplits.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.source.metrics;
+
+import java.text.NumberFormat;
+import org.apache.spark.sql.connector.metric.CustomMetric;
+
+public class NumSplits implements CustomMetric {
+
+  @Override
+  public String name() {
+    return "numSplits";
+  }
+
+  @Override
+  public String description() {
+    return "number of file splits read";
+  }
+
+  @Override
+  public String aggregateTaskMetrics(long[] taskMetrics) {
+    long sum = initialValue;
+    for (int i = 0; i < taskMetrics.length; i++) {
+      sum += taskMetrics[i];
+    }

Review Comment:
   Nit: newline



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

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

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


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


[GitHub] [iceberg] flyrain commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
flyrain commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1229045750

   Thanks @kbendick!


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

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

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


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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r956518890


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java:
##########
@@ -181,8 +181,11 @@ Pair<int[], Integer> buildPosDelRowIdMapping(PositionDeleteIndex deletedRowPosit
         if (!deletedRowPositions.isDeleted(originalRowId + rowStartPosInBatch)) {
           posDelRowIdMapping[currentRowId] = originalRowId;
           currentRowId++;
-        } else if (hasIsDeletedColumn) {
-          isDeleted[originalRowId] = true;
+        } else {
+          if (hasIsDeletedColumn) {
+            isDeleted[originalRowId] = true;
+          }
+          deletes.incrementDeleteCount();

Review Comment:
   Here is our discussion on this, https://github.com/apache/iceberg/pull/4588#discussion_r945959579. In general, I don't have strong option on either side, but prefer think this metric is orthogonal to metadata column `_deleted`.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r950768444


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java:
##########
@@ -183,6 +187,8 @@ Pair<int[], Integer> buildPosDelRowIdMapping(PositionDeleteIndex deletedRowPosit
           currentRowId++;
         } else if (hasIsDeletedColumn) {
           isDeleted[originalRowId] = true;
+        } else {
+          deletes.incrementDeleteCount();
         }

Review Comment:
   @flyrain thank you for explaining how reads with `hasIsDeletedColumn` occur. 
   I didn't really understand that before, but based on @RussellSpitzer's comment I cited, I figured it was fine not to count deletes for such reads. I have done this consistently for both vectorized and non-vectorized reads, as you realize. This is deliberate.
   
   Now that I understand the use case for `hasIsDeletedColumn`, I agree that there is merit to the argument for considering "number of row deletes applied" metric to apply regardless of `hasIsDeletedColumn`. On the other hand, for the queries where `hasIsDeletedColumn` is true, you are able to infer the number of row deletes applied from the result. I don't have a strong opinion on what to do in the `hasIsDeletedColumn` scenario. But the current implementation is consistent. @RussellSpitzer do you want to weigh in? 
   
   If we want to count the deletes always, we'll need to modify `DeleteFilter#markDeleted` (we'll need to pass in the `DeleteCounter`) as well as the changes you mention for `ColumnarBatchReader`.
   



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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r956508877


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java:
##########
@@ -181,8 +181,11 @@ Pair<int[], Integer> buildPosDelRowIdMapping(PositionDeleteIndex deletedRowPosit
         if (!deletedRowPositions.isDeleted(originalRowId + rowStartPosInBatch)) {
           posDelRowIdMapping[currentRowId] = originalRowId;
           currentRowId++;
-        } else if (hasIsDeletedColumn) {
-          isDeleted[originalRowId] = true;
+        } else {
+          if (hasIsDeletedColumn) {
+            isDeleted[originalRowId] = true;
+          }
+          deletes.incrementDeleteCount();

Review Comment:
   Are we sure that this `else` block is refactored correctly?
   
   Is it possible for `hasIsDeletedColumn` to be false but we still need to increment `deletes.incrementDeleteCount()` (at least in the context of position 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.

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r855344185


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java:
##########
@@ -46,25 +47,34 @@
 import org.apache.iceberg.types.TypeUtil;
 import org.apache.spark.rdd.InputFileBlockHolder;
 import org.apache.spark.sql.catalyst.InternalRow;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 class RowDataReader extends BaseDataReader<InternalRow> {
-
+  private static final Logger LOG = LoggerFactory.getLogger(RowDataReader.class);
   private final Schema tableSchema;
   private final Schema expectedSchema;
   private final String nameMapping;
   private final boolean caseSensitive;
+  private final DeleteCounter counter;
 
   RowDataReader(CombinedScanTask task, Table table, Schema expectedSchema, boolean caseSensitive) {
     super(table, task);
     this.tableSchema = table.schema();
     this.expectedSchema = expectedSchema;
     this.nameMapping = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING);
     this.caseSensitive = caseSensitive;
+    this.counter = new DeleteCounter();
+  }
+
+  protected DeleteCounter counter() {
+    return counter;
   }
 
   @Override
   CloseableIterator<InternalRow> open(FileScanTask task) {
-    SparkDeleteFilter deletes = new SparkDeleteFilter(task, tableSchema, expectedSchema);
+    LOG.debug("calling open, constructing SparkDeleteFilter ...");

Review Comment:
   Capital here, but i'm not sure this is a helpful debug statement either? Probably add a line in the DeleteFilter Constructor instead?



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r856688857


##########
core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java:
##########
@@ -40,7 +46,11 @@ public void delete(long posStart, long posEnd) {
 
   @Override
   public boolean isDeleted(long position) {
-    return roaring64Bitmap.contains(position);
+    boolean posIsDeleted = roaring64Bitmap.contains(position);
+    if (counter != null && posIsDeleted) {
+      counter.increment();
+    }

Review Comment:
   There are existing tests that have no need for counting deletes, that call some static methods in `Deletes`. I kept those forms of the methods and had them call new forms that take a `DeleteCounter`, passing a null `DeleteCounter`. That is the reason why a `DeleteCounter` field might be null. The way the code is now, if we decide to do without `DeleteCounter` and simply use an `AtomicLong`, then it can be updated very easily. I do understand your suggestion for a `NullDeleteCounter`.
   
   You raise a good point about `PositionDeleteIndex#isDeleted` not being idempotent now. I have checked where it is called, and it is not a concern. We should document this point though, to prevent it being a problem in future code. The current use case for `PositionDeleteIndex` is to iterate through row positions and check against a `PositionDeleteIndex` if the position is deleted; in this use case, there is no reason to check a position more than once.
   Before I ended up introducing a delete counter into the `PositionDeleteIndex` implementation and incrementing it when `isDeleted` is called, I considered other approaches, but there was one read path in particular where this seemed to the best solution.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r856697180


##########
core/src/main/java/org/apache/iceberg/deletes/Deletes.java:
##########
@@ -133,11 +158,17 @@ protected EqualitySetDeleteFilter(Function<T, StructLike> extractEq,
 
     @Override
     protected boolean shouldKeep(T row) {
-      return !deletes.contains(extractEqStruct.apply(row));
+      boolean keep = !deletes.contains(extractEqStruct.apply(row));
+      if (!keep) {

Review Comment:
   I have removed these trace messages.



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

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

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


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


[GitHub] [iceberg] wypoon commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1108780017

   @RussellSpitzer I have cleaned up the logging, and reworked the unit tests, on top of the updates to #4395.


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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r907779643


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumSplits.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.source.metrics;
+
+import java.text.NumberFormat;
+import org.apache.spark.sql.connector.metric.CustomMetric;
+
+public class NumSplits implements CustomMetric {
+
+  @Override
+  public String name() {
+    return "numSplits";
+  }
+
+  @Override
+  public String description() {
+    return "number of file splits read";
+  }
+
+  @Override
+  public String aggregateTaskMetrics(long[] taskMetrics) {
+    long sum = initialValue;
+    for (int i = 0; i < taskMetrics.length; i++) {
+      sum += taskMetrics[i];
+    }
+    return NumberFormat.getIntegerInstance().format(sum);
+  }

Review Comment:
   See above.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r888197932


##########
core/src/main/java/org/apache/iceberg/deletes/NonCounter.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.deletes;
+
+/**
+ * A dummy counter that does not increment its count.
+ */
+class NonCounter implements DeleteCounter {

Review Comment:
   @kbendick suggested a no-op version of the `DeleteCounter` instead of using null. There are legacy uses where counting is not needed, but I tend to agree that it doesn't hurt to use a single `DeleteCounter` throughout and that `NonCounter` is probably not needed.



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r888049460


##########
data/src/main/java/org/apache/iceberg/data/DeleteFilter.java:
##########
@@ -67,22 +72,26 @@
   private final List<DeleteFile> eqDeletes;
   private final Schema requiredSchema;
   private final Accessor<StructLike> posAccessor;
+  private final DeleteCounter counter;
 
   private PositionDeleteIndex deleteRowPositions = null;
-  private Predicate<T> eqDeleteRows = null;
 
-  protected DeleteFilter(String filePath, List<DeleteFile> deletes, Schema tableSchema, Schema requestedSchema) {
+  protected DeleteFilter(String filePath, List<DeleteFile> deletes, Schema tableSchema, Schema requestedSchema,
+      DeleteCounter counter) {

Review Comment:
   Do we need to pass through the counter here? Seems like this should just be an internal member of delete filter that is created with the delete filter? Or is this just because we have no other way of getting the counter information back out of the applied delete filter?



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r877629836


##########
core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java:
##########
@@ -40,7 +46,11 @@ public void delete(long posStart, long posEnd) {
 
   @Override
   public boolean isDeleted(long position) {
-    return roaring64Bitmap.contains(position);
+    boolean posIsDeleted = roaring64Bitmap.contains(position);
+    if (counter != null && posIsDeleted) {
+      counter.increment();
+    }

Review Comment:
   You are right that one can get the number of deleted positions in the `PositionDeleteIndex` by `roaring64Bitmap.getLongCardinality()`.
   However, having one counter keeps the bookkeeping simple and easy to reason about.
   There are multiple paths through the code that can be taken, where deletes may be applied, and the unifying thing here is that the same counter is passed into all these places.
   It is more difficult to reason through the counting if I have to account for the `PositionDeleteIndex` cases separately and I would have to keep references to the `PositionDeleteIndex` instances (I haven't looked in depth into this). I think it would be messy.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r961198160


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/SparkSQLExecutionHelper.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.source;
+
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.execution.ui.SQLAppStatusStore;
+import org.apache.spark.sql.execution.ui.SQLExecutionUIData;
+import org.apache.spark.sql.execution.ui.SQLPlanMetric;
+import org.junit.Assert;
+import scala.Option;
+
+public class SparkSQLExecutionHelper {
+
+  private SparkSQLExecutionHelper() {}
+
+  /**
+   * Finds the value of a specified metric for the last SQL query that was executed. Metric values
+   * are stored in the `SQLAppStatusStore` as strings.
+   *
+   * @param spark SparkSession used to run the SQL query
+   * @param metricName name of the metric
+   * @return value of the metric
+   */
+  public static String lastExecutedMetricValue(SparkSession spark, String metricName) {
+    SQLAppStatusStore statusStore = spark.sharedState().statusStore();
+    SQLExecutionUIData lastExecution = statusStore.executionsList().last();
+    Option<SQLPlanMetric> sqlPlanMetric =
+        lastExecution.metrics().find(metric -> metric.name().equals(metricName));
+    Assert.assertTrue(
+        String.format("Metric '%s' not found in last execution", metricName),
+        sqlPlanMetric.isDefined());
+    long metricId = sqlPlanMetric.get().accumulatorId();
+
+    // Refresh metricValues, they will remain null until the execution is complete and metrics are
+    // aggregated
+    int attempts = 3;
+    while (lastExecution.metricValues() == null && attempts > 0) {
+      try {
+        Thread.sleep(100);
+        attempts -= 1;

Review Comment:
   A scala-ism. :-)



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

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

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


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


[GitHub] [iceberg] pvary merged pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
pvary merged PR #4588:
URL: https://github.com/apache/iceberg/pull/4588


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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r889381963


##########
data/src/main/java/org/apache/iceberg/data/DeleteFilter.java:
##########
@@ -212,7 +244,7 @@ public PositionDeleteIndex deletedRowPositions() {
 
     if (deleteRowPositions == null) {
       List<CloseableIterable<Record>> deletes = Lists.transform(posDeletes, this::openPosDeletes);
-      deleteRowPositions = Deletes.toPositionIndex(filePath, deletes);
+      deleteRowPositions = Deletes.toPositionIndex(filePath, deletes, counter);

Review Comment:
   Actually, this eventually calls `Deletes.toPositionIndex(CloseableIterable<Long>, DeleteCounter)` where we do
   ```
         PositionDeleteIndex positionDeleteIndex = new BitmapPositionDeleteIndex();
         deletes.forEach(positionDeleteIndex::delete);
         counter.increment(((BitmapPositionDeleteIndex) positionDeleteIndex).numberOfPositionsDeleted());
         return positionDeleteIndex;
   ```
   That method is called by other callers besides this. I think it is best to do the `counter.increment(positionDeleteIndex.numberOfPositionsDeleted())` right where `PositionDeleteIndex#delete` is called. I think the bookkeeping is easier that way.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r889377177


##########
data/src/main/java/org/apache/iceberg/data/DeleteFilter.java:
##########
@@ -96,6 +105,10 @@ protected DeleteFilter(String filePath, List<DeleteFile> deletes, Schema tableSc
     this.posAccessor = requiredSchema.accessorForField(MetadataColumns.ROW_POSITION.fieldId());
   }
 
+  protected DeleteFilter(String filePath, List<DeleteFile> deletes, Schema tableSchema, Schema requestedSchema) {

Review Comment:
   This constructor without the `DeleteCounter` is called by `GenericDeleteFilter` and `FlinkDeleteFilter`.



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r888037882


##########
core/src/main/java/org/apache/iceberg/deletes/NonCounter.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.deletes;
+
+/**
+ * A dummy counter that does not increment its count.
+ */
+class NonCounter implements DeleteCounter {

Review Comment:
   Not sure why we need this class, Is there a case when we really don't want to count? I would imagine perf wise the difference between counting and not counting is pretty small?



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

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

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


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


[GitHub] [iceberg] flyrain commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
flyrain commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1145199207

   Agreed with @RussellSpitzer , we probably don't need the metrics if a user queries all deleted rows (enabled by #4683). It is more useful if when user query a data file with deletes applied since the number of deletes are hidden from users.
   I'm also worried about the perf implication. Would recommend to benchmark the potential perf loss. 


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

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

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


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


[GitHub] [iceberg] wypoon commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1147706071

   I'm working on rebasing on master, reconciling the conflicts with #4683.


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

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

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


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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r905318402


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumDeletes.java:
##########
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.source.metrics;
+
+import java.text.NumberFormat;
+import org.apache.spark.sql.connector.metric.CustomMetric;
+
+public class NumDeletes implements CustomMetric {
+
+  public static final String DISPLAY_STRING = "number of row deletes applied";
+
+  @Override
+  public String name() {
+    return "numDeletes";
+  }
+
+  @Override
+  public String description() {
+    return DISPLAY_STRING;
+  }
+
+  @Override
+  public String aggregateTaskMetrics(long[] taskMetrics) {
+    long sum = initialValue;
+    for (int i = 0; i < taskMetrics.length; i++) {
+      sum += taskMetrics[i];
+    }
+    return NumberFormat.getIntegerInstance().format(sum);
+  }

Review Comment:
   How about extending from CustomSumMetric? So that you don't have to reimplement this method.



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

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

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


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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r905345545


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java:
##########
@@ -46,30 +47,39 @@
 import org.apache.iceberg.types.TypeUtil;
 import org.apache.spark.rdd.InputFileBlockHolder;
 import org.apache.spark.sql.catalyst.InternalRow;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 class RowDataReader extends BaseDataReader<InternalRow> {
-
+  private static final Logger LOG = LoggerFactory.getLogger(RowDataReader.class);
   private final Schema tableSchema;
   private final Schema expectedSchema;
   private final String nameMapping;
   private final boolean caseSensitive;
+  private final DeleteCounter counter;

Review Comment:
   I'd also suggest to move this to `BaseDataReader`.



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

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

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


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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r905361717


##########
core/src/main/java/org/apache/iceberg/deletes/Deletes.java:
##########
@@ -75,8 +75,21 @@ public static <T> CloseableIterable<T> markDeleted(CloseableIterable<T> rows, Pr
     });
   }
 
-  public static <T> CloseableIterable<T> filterDeleted(CloseableIterable<T> rows, Predicate<T> isDeleted) {
-    return CloseableIterable.filter(rows, isDeleted.negate());
+  // This returns the remaining rows (the ones that are not deleted) while counting the deleted ones

Review Comment:
   Can we make it formal method Java doc? Or we may not need this comment.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r951820279


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java:
##########
@@ -130,7 +140,8 @@ protected Table createTable(String name, Schema schema, PartitionSpec spec) {
     TableOperations ops = ((BaseTable) table).operations();
     TableMetadata meta = ops.current();
     ops.commit(meta, meta.upgradeToFormatVersion(2));
-    if (vectorized) {
+    table.updateProperties().set(TableProperties.DEFAULT_FILE_FORMAT, format).commit();
+    if (format.equals("parquet") && vectorized) {

Review Comment:
   Fixed.



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

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

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


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


[GitHub] [iceberg] pvary commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
pvary commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r960232132


##########
core/src/main/java/org/apache/iceberg/deletes/Deletes.java:
##########
@@ -62,6 +62,7 @@ public static <T> CloseableIterable<T> filter(
     return equalityFilter.filter(rows);
   }
 
+  /** Returns the same rows that are input, while marking the deleted ones. */

Review Comment:
   Maybe this is only me, but when we add a comment,  we can create a full javadoc. I personally hate partial Java doc comments.



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

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

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


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


[GitHub] [iceberg] pvary commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
pvary commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r960235000


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumDeletes.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.source.metrics;
+
+import java.text.NumberFormat;
+import org.apache.spark.sql.connector.metric.CustomMetric;
+
+public class NumDeletes implements CustomMetric {
+
+  public static final String DISPLAY_STRING = "number of row deletes applied";
+
+  @Override
+  public String name() {
+    return "numDeletes";
+  }
+
+  @Override
+  public String description() {
+    return DISPLAY_STRING;
+  }
+
+  @Override
+  public String aggregateTaskMetrics(long[] taskMetrics) {
+    long sum = initialValue;
+    for (int i = 0; i < taskMetrics.length; i++) {
+      sum += taskMetrics[i];
+    }

Review Comment:
   Nit: newline



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

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

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


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


[GitHub] [iceberg] flyrain commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
flyrain commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1233230531

   @wypoon, sorry for the delay, working on my apache authentication issue. I don't have the permission to merge until the issue is resolved. May @RussellSpitzer can help?


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

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

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


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


[GitHub] [iceberg] wypoon commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1104234389

   @rdblue @RussellSpitzer would you please review?


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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r853320291


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

Review Comment:
   This class is not strictly necessary. I could simply use an AtomicLong in its place. (I do not think, however, that the counter needs to be thread-safe, as each task will have its own counter.)



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r855359823


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java:
##########
@@ -139,18 +151,55 @@ protected void dropTable(String name) {
   }
 
   @Override
-  public StructLikeSet rowSet(String name, Table table, String... columns) {
-    Dataset<Row> df = spark.read()
-        .format("iceberg")
-        .load(TableIdentifier.of("default", name).toString())
-        .selectExpr(columns);
+  protected boolean countDeletes() {
+    return true;
+  }
 
-    Types.StructType projection = table.schema().select(columns).asStruct();
-    StructLikeSet set = StructLikeSet.create(projection);
-    df.collectAsList().forEach(row -> {
-      SparkStructLike rowWrapper = new SparkStructLike(projection);
-      set.add(rowWrapper.wrap(row));
-    });
+  private void setDeleteCount(long count) {
+    deleteCount = count;
+  }
+
+  @Override
+  protected long deleteCount() {
+    return deleteCount;
+  }
+
+  @Override
+  public StructLikeSet rowSet(String name, Table table, String... columns) throws IOException {
+    Schema schema = table.schema().select(columns);
+    StructType sparkSchema = SparkSchemaUtil.convert(schema);
+    Types.StructType type = schema.asStruct();
+    StructLikeSet set = StructLikeSet.create(type);
+
+    CloseableIterable<CombinedScanTask> tasks = TableScanUtil.planTasks(
+        table.newScan().planFiles(),
+        TableProperties.METADATA_SPLIT_SIZE_DEFAULT,
+        TableProperties.SPLIT_LOOKBACK_DEFAULT,
+        TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT);
+
+    long delCount = 0L;

Review Comment:
   This feels like we are subverting the intent of this test which is to use the DF reader method to actually read the table. I think we'll need another approach for counting deletes or possibly just explicitly state how many deletes there are in tests that need to check it. At least until we have a method of reading a table with a user readable marker for deleted rows like `_isDeleted`



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

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

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


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


[GitHub] [iceberg] wypoon commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1105468390

   @RussellSpitzer thank you for reviewing!
   This PR is on top of #4395, but since that is not merged, all its changes are here.
   I'll clean up the logging. It is true that their original purpose was for debugging/testing, but I think some of them are still potentially helpful.


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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r855418924


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumDeletes.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.source.metrics;
+
+import java.text.NumberFormat;
+import org.apache.spark.sql.connector.metric.CustomMetric;
+
+public class NumDeletes implements CustomMetric {

Review Comment:
   So we aren't suing CustomSumMetric because it's string output is different than we want? I think that's ok but seems like we should be changing CustomSumMetric in Spark instead of having our own summing class that does the same thing but outputs the number differently.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r855531401


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumDeletes.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.source.metrics;
+
+import java.text.NumberFormat;
+import org.apache.spark.sql.connector.metric.CustomMetric;
+
+public class NumDeletes implements CustomMetric {

Review Comment:
   I can open a Spark PR to change `CustomSumMetric`, but for now I'd just extend `CustomMetric`.
   The thing is, Spark instantiates your `CustomMetric` class and using reflection, calls its `aggregateTaskMetrics` method to get the value to display in the UI. However, even if you extend `CustomSumMetric` and don't override `aggregateTaskMetrics`, Spark still needs to instantiate your `CustomMetric` class, so you can't get away from putting your `CustomMetric` class in the classpath of the Spark History Server. If there was a way to just use `CustomSumMetric` (and live with its deficient output formatting) and not have to provide your `CustomMetric` class to the SHS, I'd accept that, but there isn't. Thus I really don't see any benefit to extending `CustomSumMetric`.
   



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r876412916


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeleteCount.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.source;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+import org.apache.spark.sql.vectorized.ColumnarBatch;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestSparkReaderDeleteCount extends SparkDeleteReadTestsBase {
+
+  private long deleteCount;
+
+  public TestSparkReaderDeleteCount(String format, boolean vectorized) {
+    super(format, vectorized);
+  }
+
+  @Parameterized.Parameters(name = "format = {0}, vectorized = {1}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] {"parquet", true},
+        new Object[] {"orc", false},
+        new Object[] {"avro", false}
+    };
+  }
+
+  @Override
+  protected boolean countDeletes() {
+    return true;
+  }
+
+  private void setDeleteCount(long count) {
+    deleteCount = count;
+  }
+
+  @Override
+  protected long deleteCount() {
+    return deleteCount;
+  }
+
+  @Override
+  public StructLikeSet rowSet(String name, Table table, String... columns) throws IOException {
+    Schema schema = table.schema().select(columns);
+    StructType sparkSchema = SparkSchemaUtil.convert(schema);
+    Types.StructType type = schema.asStruct();
+    StructLikeSet set = StructLikeSet.create(type);
+
+    SparkScanBuilder scanBuilder = new SparkScanBuilder(spark, table, CaseInsensitiveStringMap.empty());

Review Comment:
   Instead of doing this would it be possible for us to use a custom SparkListener to just grab the status at the end of the job and check those values? I think it is a safer approach than reimplementing the read path here.
   
   We could also just modify the SQLAppStatusListener? or something like that



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r876401153


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumDeletes.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.source.metrics;
+
+import java.text.NumberFormat;
+import org.apache.spark.sql.connector.metric.CustomMetric;
+
+public class NumDeletes implements CustomMetric {
+
+  @Override
+  public String name() {
+    return "numDeletes";
+  }
+
+  @Override
+  public String description() {
+    return "number of row deletes applied";

Review Comment:
   that's fine then, I forgot Spark did that



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r876416968


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeleteCount.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.source;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+import org.apache.spark.sql.vectorized.ColumnarBatch;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestSparkReaderDeleteCount extends SparkDeleteReadTestsBase {
+
+  private long deleteCount;
+
+  public TestSparkReaderDeleteCount(String format, boolean vectorized) {
+    super(format, vectorized);
+  }
+
+  @Parameterized.Parameters(name = "format = {0}, vectorized = {1}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] {"parquet", true},

Review Comment:
   I could add that if you think it is beneficial. I was trying to reduce overlap with `TestSparkReaderDeletes`. For testing the delete count, I think the parameters I have added cover all 3 file formats as well as both vectorized and non-vectorized, so I think the existing code paths are covered (to my knowledge). At present, I don't think adding `{"parquet", false}` exercises any additional code path for the delete tracking.



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r888055148


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java:
##########
@@ -195,20 +199,8 @@ int[] initEqDeleteRowIdMapping(int numRowsToRead) {
      */
     void applyEqDelete() {
       Iterator<InternalRow> it = columnarBatch.rowIterator();
-      int rowId = 0;
-      int currentRowId = 0;
-      while (it.hasNext()) {
-        InternalRow row = it.next();
-        if (deletes.eqDeletedRowFilter().test(row)) {
-          // the row is NOT deleted
-          // skip deleted rows by pointing to the next undeleted row Id
-          rowIdMapping[currentRowId] = rowIdMapping[rowId];
-          currentRowId++;
-        }
-
-        rowId++;
-      }
-
+      LOG.debug("Applying equality deletes to row id mapping");
+      int currentRowId = deletes.applyEqDeletes(it, rowIdMapping);

Review Comment:
   Is this just a refactoring?



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

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

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


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


[GitHub] [iceberg] wypoon commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1145113959

   > > > I think we would also really benefit from @flyrain doing a full review on this as well especially now that we have the "markDelete" pathway as well. I assume for that we probably will just skip counting deletes since we don't really care.
   > > 
   > > 
   > > @RussellSpitzer I saw that `Deletes` and `DeleteFilter` have changed in master since I updated this PR (so this PR no longer merges), but I haven't had the time to understand the changes and how to reconcile this PR with them. I don't understand your statement that "we probably will just skip counting deletes since we don't really care." Are you saying that this PR is no longer worthwhile??
   > 
   > No! I think this PR is very important, there is just now a second read path with `_is_deleted` metadata column which actually returns deleted rows. When we follow that path I'm not sure it's important for us to count the deleted rows since we'll be returning them anyway, I could go either way. For the path when we don't actually return deleted rows (the normal read path) we definitely want this metric!
   
   Thanks for the explanation. I haven't followed the changes that recently went in. Your explanation helps put them in context for me. I'll rebase on master and try to address your current feedback.


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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r888020537


##########
core/src/main/java/org/apache/iceberg/deletes/Deletes.java:
##########
@@ -84,27 +89,46 @@ public static StructLikeSet toEqualitySet(CloseableIterable<StructLike> eqDelete
   }
 
   public static <T extends StructLike> PositionDeleteIndex toPositionIndex(CharSequence dataLocation,
-                                                                           List<CloseableIterable<T>> deleteFiles) {
+                                                                           List<CloseableIterable<T>> deleteFiles,
+                                                                           DeleteCounter counter) {
     DataFileFilter<T> locationFilter = new DataFileFilter<>(dataLocation);
     List<CloseableIterable<Long>> positions = Lists.transform(deleteFiles, deletes ->
         CloseableIterable.transform(locationFilter.filter(deletes), row -> (Long) POSITION_ACCESSOR.get(row)));
-    return toPositionIndex(CloseableIterable.concat(positions));
+    return toPositionIndex(CloseableIterable.concat(positions), counter);
   }
 
-  public static PositionDeleteIndex toPositionIndex(CloseableIterable<Long> posDeletes) {
+  public static PositionDeleteIndex toPositionIndex(CloseableIterable<Long> posDeletes, DeleteCounter counter) {
     try (CloseableIterable<Long> deletes = posDeletes) {
       PositionDeleteIndex positionDeleteIndex = new BitmapPositionDeleteIndex();
       deletes.forEach(positionDeleteIndex::delete);
+      counter.increment(((BitmapPositionDeleteIndex) positionDeleteIndex).numberOfPositionsDeleted());

Review Comment:
   we can the skip the cast here if we just keep the type on 102



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1145107100

   > > I think we would also really benefit from @flyrain doing a full review on this as well especially now that we have the "markDelete" pathway as well. I assume for that we probably will just skip counting deletes since we don't really care.
   > 
   > @RussellSpitzer I saw that `Deletes` and `DeleteFilter` have changed in master since I updated this PR (so this PR no longer merges), but I haven't had the time to understand the changes and how to reconcile this PR with them. I don't understand your statement that "we probably will just skip counting deletes since we don't really care." Are you saying that this PR is no longer worthwhile??
   
   No! I think this PR is very important, there is just now a second read path with `_is_deleted` metadata column which actually returns deleted rows. When we follow that path I'm not sure it's important for us to count the deleted rows since we'll be returning them anyway, I could go either way. For the path when we don't actually return deleted rows (the normal read path) we definitely want this metric!


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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r907779387


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumDeletes.java:
##########
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.source.metrics;
+
+import java.text.NumberFormat;
+import org.apache.spark.sql.connector.metric.CustomMetric;
+
+public class NumDeletes implements CustomMetric {
+
+  public static final String DISPLAY_STRING = "number of row deletes applied";
+
+  @Override
+  public String name() {
+    return "numDeletes";
+  }
+
+  @Override
+  public String description() {
+    return DISPLAY_STRING;
+  }
+
+  @Override
+  public String aggregateTaskMetrics(long[] taskMetrics) {
+    long sum = initialValue;
+    for (int i = 0; i < taskMetrics.length; i++) {
+      sum += taskMetrics[i];
+    }
+    return NumberFormat.getIntegerInstance().format(sum);
+  }

Review Comment:
   I already explained this in response to the same suggestion by @RussellSpitzer. `CustomeSumMetric#aggregateTaskMetrics` does not format the String. We want the metric value to appear the same way as other built-in metrics. E.g., 1000 should appear as "1,000" in the US locale, not "1000".



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

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

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


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


[GitHub] [iceberg] wypoon commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1185042222

   I have been on a break. Just returning to this. I'm rebasing on master since there are conflicts. I'll follow up after that.


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

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

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


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


[GitHub] [iceberg] wypoon closed pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon closed pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan
URL: https://github.com/apache/iceberg/pull/4588


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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r890758865


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java:
##########
@@ -139,7 +193,21 @@ protected void dropTable(String name) {
   }
 
   @Override
-  public StructLikeSet rowSet(String name, Table table, String... columns) {
+  protected boolean countDeletes() {

Review Comment:
   I removed the `deleteCount` field and the `setDeleteCount(long)` method, and the `deleteCount()` method now simply calls a `lastExecutedMetricValue(...)` method along the lines you suggest.



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r855416250


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java:
##########
@@ -139,18 +151,55 @@ protected void dropTable(String name) {
   }
 
   @Override
-  public StructLikeSet rowSet(String name, Table table, String... columns) {
-    Dataset<Row> df = spark.read()
-        .format("iceberg")
-        .load(TableIdentifier.of("default", name).toString())
-        .selectExpr(columns);
+  protected boolean countDeletes() {
+    return true;
+  }
 
-    Types.StructType projection = table.schema().select(columns).asStruct();
-    StructLikeSet set = StructLikeSet.create(projection);
-    df.collectAsList().forEach(row -> {
-      SparkStructLike rowWrapper = new SparkStructLike(projection);
-      set.add(rowWrapper.wrap(row));
-    });
+  private void setDeleteCount(long count) {
+    deleteCount = count;
+  }
+
+  @Override
+  protected long deleteCount() {
+    return deleteCount;
+  }
+
+  @Override
+  public StructLikeSet rowSet(String name, Table table, String... columns) throws IOException {
+    Schema schema = table.schema().select(columns);
+    StructType sparkSchema = SparkSchemaUtil.convert(schema);
+    Types.StructType type = schema.asStruct();
+    StructLikeSet set = StructLikeSet.create(type);
+
+    CloseableIterable<CombinedScanTask> tasks = TableScanUtil.planTasks(
+        table.newScan().planFiles(),
+        TableProperties.METADATA_SPLIT_SIZE_DEFAULT,
+        TableProperties.SPLIT_LOOKBACK_DEFAULT,
+        TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT);
+
+    long delCount = 0L;

Review Comment:
   That is the path it takes at this moment, but that's the issue here, we want this test check the output of
   ```    
   Dataset<Row> df = spark.read()
           .format("iceberg")
           .load(TableIdentifier.of("default", name).toString())
           .selectExpr(columns);
   ```
   
   Having this copy the implementation details decouples the test from what actually may be happening we change the code in the future



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r855334798


##########
data/src/main/java/org/apache/iceberg/data/DeleteFilter.java:
##########
@@ -172,7 +185,8 @@ public CloseableIterable<T> findEqualityDeleteRows(CloseableIterable<T> records)
     Filter<T> deletedRowsFilter = new Filter<T>() {
       @Override
       protected boolean shouldKeep(T item) {
-        return deletedRows.test(item);
+        boolean keep = deletedRows.test(item); // true means row is deleted

Review Comment:
   This probably can be reverted?



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r855336294


##########
data/src/main/java/org/apache/iceberg/data/DeleteFilter.java:
##########
@@ -188,31 +202,52 @@ private CloseableIterable<T> applyEqDeletes(CloseableIterable<T> records) {
     Filter<T> remainingRowsFilter = new Filter<T>() {
       @Override
       protected boolean shouldKeep(T item) {
-        return remainingRows.test(item);
+        boolean keep = remainingRows.test(item); // false means row is deleted
+        if (!keep && counter != null) {
+          counter.increment();
+        }
+        return keep;
       }
     };
 
     return remainingRowsFilter.filter(records);
   }
 
-  public Predicate<T> eqDeletedRowFilter() {
-    if (eqDeleteRows == null) {
-      eqDeleteRows = applyEqDeletes().stream()
-          .map(Predicate::negate)
-          .reduce(Predicate::and)
-          .orElse(t -> true);
+  public int applyEqDeletes(Iterator<T> records, int[] mapping) {
+    Predicate<T> remainingRows = applyEqDeletes().stream()
+        .map(Predicate::negate)
+        .reduce(Predicate::and)
+        .orElse(t -> true);
+
+    int rowId = 0;
+    int currentRowId = 0;
+    while (records.hasNext()) {
+      T row = records.next();
+      if (remainingRows.test(row)) {
+        // the row is NOT deleted
+        // skip deleted rows by pointing to the next undeleted row Id
+        mapping[currentRowId] = mapping[rowId];
+        currentRowId++;
+      } else if (counter != null) {
+        counter.increment();
+      }
+
+      rowId++;
     }
-    return eqDeleteRows;
+
+    return currentRowId;
   }
 
   public PositionDeleteIndex deletedRowPositions() {
+    LOG.debug("calling deletedRowPositions()");

Review Comment:
   Probably remove these messages?



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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r956506186


##########
data/src/test/java/org/apache/iceberg/data/DeleteReadTests.java:
##########
@@ -282,6 +306,7 @@ public void testEqualityDeletesSpanningMultipleDataFiles() throws IOException {
         FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), Row.of(0), records);
 
     table.newAppend().appendFile(dataFile).commit();
+    // At this point, the table has 7 + 8 = 15 rows, of which all but one are in duplicate.

Review Comment:
   Nit / non-blocking: it’s a little strange to me to have the whole line comments with empty lines afterwards.
   
   Since this is just a test and it said “at this point”, I feel like we could move it to be directly above the next code block. But it’s just test code so I absolutely wouldn’t block on this.



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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r956507784


##########
data/src/test/java/org/apache/iceberg/data/DeleteReadTests.java:
##########
@@ -297,11 +322,14 @@ public void testEqualityDeletesSpanningMultipleDataFiles() throws IOException {
             table, Files.localOutput(temp.newFile()), Row.of(0), dataDeletes, deleteRowSchema);
 
     table.newRowDelta().addDeletes(eqDeletes).commit();
+    // At ths point, the table has (7 - 3) + (8 - 4) = 8 rows. 7 rows in all are deleted.

Review Comment:
   Nit: the phrasing of this comment is a little hard for me to follow without reading it twice.
   
   Maybe something like `// At ths point, the table has two data files. They have 7 rows and 8 rows with 3 deleted rows and 4 deleted rows respectively.`
   
   But again this is just a test so I wouldn’t block on this.



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

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

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


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


[GitHub] [iceberg] pvary commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
pvary commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r960237480


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumSplits.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.source.metrics;
+
+import java.text.NumberFormat;
+import org.apache.spark.sql.connector.metric.CustomMetric;
+
+public class NumSplits implements CustomMetric {
+
+  @Override
+  public String name() {
+    return "numSplits";
+  }
+
+  @Override
+  public String description() {
+    return "number of file splits read";
+  }
+
+  @Override
+  public String aggregateTaskMetrics(long[] taskMetrics) {
+    long sum = initialValue;
+    for (int i = 0; i < taskMetrics.length; i++) {
+      sum += taskMetrics[i];
+    }

Review Comment:
   Nit: newline 



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

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

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


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


[GitHub] [iceberg] wypoon commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1229007782

   @flyrain thank you for reviewing. I hope this can be merged now.


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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r956525437


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java:
##########
@@ -181,8 +181,11 @@ Pair<int[], Integer> buildPosDelRowIdMapping(PositionDeleteIndex deletedRowPosit
         if (!deletedRowPositions.isDeleted(originalRowId + rowStartPosInBatch)) {
           posDelRowIdMapping[currentRowId] = originalRowId;
           currentRowId++;
-        } else if (hasIsDeletedColumn) {
-          isDeleted[originalRowId] = true;
+        } else {
+          if (hasIsDeletedColumn) {
+            isDeleted[originalRowId] = true;
+          }
+          deletes.incrementDeleteCount();

Review Comment:
   Yes, this is correct.
   The structure of the logic is
   ```
           if (...) {
             // row is not deleted
           } else {
             // row is deleted
             if (hasIsDeletedColumn) { ... }
             deletes.incrementDeleteCount();
           }
   ```
   In the else clause, we increment the counter regardless of `hasIsDeletedColumn` because this is the case where the row is deleted.



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

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

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


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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r904422923


##########
core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java:
##########
@@ -47,4 +47,8 @@ public boolean isDeleted(long position) {
   public boolean isEmpty() {
     return roaring64Bitmap.isEmpty();
   }
+
+  public long numberOfPositionsDeleted() {

Review Comment:
   It makes sense to put it in the interface. The change of `CustomizedPositionDeleteIndex` is minor. 
   Minor suggestion: `numberOfPositionsDeleted` -> `numberOfPositionDeletes`



##########
core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java:
##########
@@ -47,4 +47,8 @@ public boolean isDeleted(long position) {
   public boolean isEmpty() {
     return roaring64Bitmap.isEmpty();
   }
+
+  public long numberOfPositionsDeleted() {

Review Comment:
   It makes sense to put it in the interface. The change of `CustomizedPositionDeleteIndex` will be minor. 
   Minor suggestion: `numberOfPositionsDeleted` -> `numberOfPositionDeletes`



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r879949910


##########
data/src/main/java/org/apache/iceberg/data/DeleteFilter.java:
##########
@@ -225,18 +257,19 @@ private CloseableIterable<T> applyPosDeletes(CloseableIterable<T> records) {
     List<CloseableIterable<Record>> deletes = Lists.transform(posDeletes, this::openPosDeletes);
 
     // if there are fewer deletes than a reasonable number to keep in memory, use a set
-    if (posDeletes.stream().mapToLong(DeleteFile::recordCount).sum() < setFilterThreshold) {
-      return Deletes.filter(records, this::pos, Deletes.toPositionIndex(filePath, deletes));
+    if (posDeletes.stream().mapToLong(DeleteFile::recordCount).sum() < streamFilterThreshold) {
+      return Deletes.filter(records, this::pos, Deletes.toPositionIndex(filePath, deletes, counter));
     }
 
-    return Deletes.streamingFilter(records, this::pos, Deletes.deletePositions(filePath, deletes));
+    return Deletes.streamingFilter(records, this::pos, Deletes.deletePositions(filePath, deletes), counter);

Review Comment:
   This is a code path that was not previously exercised by the unit tests.
   We add a configuration that allows us to control the `streamFilterThreshold` and cause this code path to be taken.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r889370152


##########
core/src/main/java/org/apache/iceberg/deletes/Deletes.java:
##########
@@ -84,27 +89,46 @@ public static StructLikeSet toEqualitySet(CloseableIterable<StructLike> eqDelete
   }
 
   public static <T extends StructLike> PositionDeleteIndex toPositionIndex(CharSequence dataLocation,
-                                                                           List<CloseableIterable<T>> deleteFiles) {
+                                                                           List<CloseableIterable<T>> deleteFiles,
+                                                                           DeleteCounter counter) {
     DataFileFilter<T> locationFilter = new DataFileFilter<>(dataLocation);
     List<CloseableIterable<Long>> positions = Lists.transform(deleteFiles, deletes ->
         CloseableIterable.transform(locationFilter.filter(deletes), row -> (Long) POSITION_ACCESSOR.get(row)));
-    return toPositionIndex(CloseableIterable.concat(positions));
+    return toPositionIndex(CloseableIterable.concat(positions), counter);
   }
 
-  public static PositionDeleteIndex toPositionIndex(CloseableIterable<Long> posDeletes) {
+  public static PositionDeleteIndex toPositionIndex(CloseableIterable<Long> posDeletes, DeleteCounter counter) {
     try (CloseableIterable<Long> deletes = posDeletes) {
       PositionDeleteIndex positionDeleteIndex = new BitmapPositionDeleteIndex();
       deletes.forEach(positionDeleteIndex::delete);
+      counter.increment(((BitmapPositionDeleteIndex) positionDeleteIndex).numberOfPositionsDeleted());

Review Comment:
   If we add `numberOfPositionsDeleted()` in the `PositionDeleteIndex` interface, then we don't need the cast either.



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r888030554


##########
data/src/main/java/org/apache/iceberg/data/DeleteFilter.java:
##########
@@ -212,7 +244,7 @@ public PositionDeleteIndex deletedRowPositions() {
 
     if (deleteRowPositions == null) {
       List<CloseableIterable<Record>> deletes = Lists.transform(posDeletes, this::openPosDeletes);
-      deleteRowPositions = Deletes.toPositionIndex(filePath, deletes);
+      deleteRowPositions = Deletes.toPositionIndex(filePath, deletes, counter);

Review Comment:
   Instead of passing through the counter here couldn't we just do
   counter.increment(deleteRowPositions.deleteCount)?



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r888062423


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java:
##########
@@ -139,7 +193,21 @@ protected void dropTable(String name) {
   }
 
   @Override
-  public StructLikeSet rowSet(String name, Table table, String... columns) {
+  protected boolean countDeletes() {

Review Comment:
   Instead of the apis here, could we just modify "extractDeleteCount()" 
   
   to instead be something like "
   ```java 
   long lastMetricCount(String metricName) {
   // Get executionID of last run thing
   // Find metricName
   // return value for MetricName
   }
   ```
   
   This way we can avoid some state in the job?



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r888122799


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java:
##########
@@ -152,9 +220,53 @@ public StructLikeSet rowSet(String name, Table table, String... columns) {
       set.add(rowWrapper.wrap(row));
     });
 
+    extractDeleteCount();
     return set;
   }
 
+  private void extractDeleteCount() {
+    // Get the executionId of the query we just executed
+    List<Long> executed = listener.executed();
+    long lastExecuted = executed.get(executed.size() - 1);
+    // Ensure that the execution end was registered

Review Comment:
   Oh this is great, Looking into the apis you dug up I think there may be an even easier way to do this. Sorry I didn't realize you could do this before
   
   Looking into sharedState you can do something like
   ```scala
   // Some operation
   scala> spark.range(1, 100).withColumn("x", col("id")).withColumnRenamed("id", "y").writeTo("local.default.test").append
   
   // UI Metrics
   scala> spark.sharedState.statusStore.executionsList.last.metrics
   res41: Seq[org.apache.spark.sql.execution.ui.SQLPlanMetric] = List(SQLPlanMetric(number of output rows,25,sum), SQLPlanMetric(duration,24,timing))
   
   // Description
   scala> spark.sharedState.statusStore.executionsList.last.description
   res42: String = append at <console>:23
   
   // Execution id
   scala> spark.sharedState.statusStore.executionsList.last.executionId
   res47: Long = 6
   ```
   
   So this block could be like
   ```java
     private long lastExecutedMetric(String metricName) {
       SQLExecutionUIData lastExecution = spark.sharedState().statusStore().executionsList().last();
       Option<SQLPlanMetric> sqlPlanMetric = lastExecution.metrics().find(metric -> metric.name().equals(metricName));
       Assert.assertTrue(String.format("Metric %s not present in last execution", metricName), sqlPlanMetric.isDefined());
       long metricId = sqlPlanMetric.get().accumulatorId();
   
       // Refresh metricValues, they will remain null until the execution is complete and metrics are aggregated
       int attempts = 3;
       while (lastExecution.metricValues() == null && attempts > 0) {
         try {
           Thread.sleep(100);
           attempts -= 1;
         } catch (InterruptedException e) {
           throw new RuntimeException(e);
         }
         lastExecution = spark.sharedState().statusStore().execution(lastExecution.executionId()).get();
       }
       Assert.assertNotNull(String.format("Metric %s was never finalized", metricName), lastExecution.metricValues());
       return Long.parseLong(lastExecution.metricValues().get(metricId).get());
     }
     ```
     
     Then we just have
     ```java
     @Override
     protected void checkDeleteCount(long expectedDeletes) {
       Assert.assertEquals("Unexpected number of deletes recorded", expectedDeletes,
           lastExecutedMetric(NumDeletes.DISPLAY_STRING));
     }
     ```
     
     And we can drop the other state in the test class.
   



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

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

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


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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r905319015


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumSplits.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.source.metrics;
+
+import java.text.NumberFormat;
+import org.apache.spark.sql.connector.metric.CustomMetric;
+
+public class NumSplits implements CustomMetric {
+
+  @Override
+  public String name() {
+    return "numSplits";
+  }
+
+  @Override
+  public String description() {
+    return "number of file splits read";
+  }
+
+  @Override
+  public String aggregateTaskMetrics(long[] taskMetrics) {
+    long sum = initialValue;
+    for (int i = 0; i < taskMetrics.length; i++) {
+      sum += taskMetrics[i];
+    }
+    return NumberFormat.getIntegerInstance().format(sum);
+  }

Review Comment:
   Same here, to extend from CustomSumMetric?



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

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

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


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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r905365603


##########
core/src/main/java/org/apache/iceberg/deletes/Deletes.java:
##########
@@ -90,27 +103,45 @@ public static StructLikeSet toEqualitySet(CloseableIterable<StructLike> eqDelete
   }
 
   public static <T extends StructLike> PositionDeleteIndex toPositionIndex(CharSequence dataLocation,
-                                                                           List<CloseableIterable<T>> deleteFiles) {
+                                                                           List<CloseableIterable<T>> deleteFiles,
+                                                                           DeleteCounter counter) {
     DataFileFilter<T> locationFilter = new DataFileFilter<>(dataLocation);
     List<CloseableIterable<Long>> positions = Lists.transform(deleteFiles, deletes ->
         CloseableIterable.transform(locationFilter.filter(deletes), row -> (Long) POSITION_ACCESSOR.get(row)));
-    return toPositionIndex(CloseableIterable.concat(positions));
+    return toPositionIndex(CloseableIterable.concat(positions), counter);
   }
 
-  public static PositionDeleteIndex toPositionIndex(CloseableIterable<Long> posDeletes) {
+  public static PositionDeleteIndex toPositionIndex(CloseableIterable<Long> posDeletes, DeleteCounter counter) {
     try (CloseableIterable<Long> deletes = posDeletes) {
-      PositionDeleteIndex positionDeleteIndex = new BitmapPositionDeleteIndex();
+      BitmapPositionDeleteIndex positionDeleteIndex = new BitmapPositionDeleteIndex();
       deletes.forEach(positionDeleteIndex::delete);
+      counter.increment(positionDeleteIndex.numberOfPositionsDeleted());

Review Comment:
   This is not the right way. positionDeleteIndex has all pos deletes for the data file, but not necessary all deletes are used in a read. We need to do the same thing as batch eq delete(line 223 in DeleteFilter.java)



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r855383946


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java:
##########
@@ -139,18 +151,55 @@ protected void dropTable(String name) {
   }
 
   @Override
-  public StructLikeSet rowSet(String name, Table table, String... columns) {
-    Dataset<Row> df = spark.read()
-        .format("iceberg")
-        .load(TableIdentifier.of("default", name).toString())
-        .selectExpr(columns);
+  protected boolean countDeletes() {
+    return true;
+  }
 
-    Types.StructType projection = table.schema().select(columns).asStruct();
-    StructLikeSet set = StructLikeSet.create(projection);
-    df.collectAsList().forEach(row -> {
-      SparkStructLike rowWrapper = new SparkStructLike(projection);
-      set.add(rowWrapper.wrap(row));
-    });
+  private void setDeleteCount(long count) {
+    deleteCount = count;
+  }
+
+  @Override
+  protected long deleteCount() {
+    return deleteCount;
+  }
+
+  @Override
+  public StructLikeSet rowSet(String name, Table table, String... columns) throws IOException {
+    Schema schema = table.schema().select(columns);
+    StructType sparkSchema = SparkSchemaUtil.convert(schema);
+    Types.StructType type = schema.asStruct();
+    StructLikeSet set = StructLikeSet.create(type);
+
+    CloseableIterable<CombinedScanTask> tasks = TableScanUtil.planTasks(
+        table.newScan().planFiles(),
+        TableProperties.METADATA_SPLIT_SIZE_DEFAULT,
+        TableProperties.SPLIT_LOOKBACK_DEFAULT,
+        TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT);
+
+    long delCount = 0L;

Review Comment:
   I could replace 
   ```
       CloseableIterable<CombinedScanTask> tasks = TableScanUtil.planTasks(
           table.newScan().planFiles(),
           TableProperties.METADATA_SPLIT_SIZE_DEFAULT,
           TableProperties.SPLIT_LOOKBACK_DEFAULT,
           TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT);
   ```
   with
   ```
       SparkScanBuilder scanBuilder = new SparkScanBuilder(spark, table, CaseInsensitiveStringMap.empty());
       scanBuilder.pruneColumns(sparkSchema);
       SparkScan scan = (SparkScan) scanBuilder.buildMergeOnReadScan();
       List<CombinedScanTask> tasks = scan.tasks();
   ```
   That is the path that a Spark read takes. I am also doing most of the rest of what Spark does once we have the tasks and hand them to Spark in the form of `InputPartition`s, which is in each `InputPartition`, get a `PartitionReader` (which is a subclass of either `BatchDataReader` or `RowDataReader`) and iterate through the reader. I just don't assemble the `InternalRow`s into a `DataFrame`.



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r855322628


##########
core/src/main/java/org/apache/iceberg/deletes/Deletes.java:
##########
@@ -133,11 +158,17 @@ protected EqualitySetDeleteFilter(Function<T, StructLike> extractEq,
 
     @Override
     protected boolean shouldKeep(T row) {
-      return !deletes.contains(extractEqStruct.apply(row));
+      boolean keep = !deletes.contains(extractEqStruct.apply(row));
+      if (!keep) {

Review Comment:
   I'm not sure these trace messages will be very valuable, do you think we need them for the metric code or where they just temporary for testing?



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r855351058


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumDeletes.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.source.metrics;
+
+import java.text.NumberFormat;
+import org.apache.spark.sql.connector.metric.CustomMetric;
+
+public class NumDeletes implements CustomMetric {

Review Comment:
   Should probably extend CustomSumMetric
   
   https://spark.apache.org/docs/latest/api/java/org/apache/spark/sql/connector/metric/CustomSumMetric.html



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r853347186


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnVectorWithFilter.java:
##########
@@ -97,9 +97,18 @@ public byte[] getBinary(int rowId) {
     return accessor().getBinary(rowIdMapping[rowId]);
   }
 
+  static ConstantColumnVector constantColumnVector(VectorHolder holder, int numRows) {
+    Object constant = ((ConstantVectorHolder) holder).getConstant();
+    if (constant instanceof Boolean) {
+      return new ConstantColumnVector(Types.BooleanType.get(), numRows, constant);
+    } else {
+      return new ConstantColumnVector(Types.IntegerType.get(), numRows, constant);
+    }
+  }
+
   public static ColumnVector forHolder(VectorHolder holder, int[] rowIdMapping, int numRows) {
     return holder.isDummy() ?
-        new ConstantColumnVector(Types.IntegerType.get(), numRows, ((ConstantVectorHolder) holder).getConstant()) :

Review Comment:
   I do not know why the `ConstantColumnVector` is constructed with a hardcoded integer type. I observe that this code here can get called with `ConstantVectorHolder`s containing a constant of other primitive types. If `copy()` is called on an `InternalRow` in the `ColumnarBatch` containing this `ColumnVector`, then we get a `ClassCastException`. In the situations where I call `copy()`, the `ConstantColumnVector` contains either a boolean or an integer, so I only fix it for those possibilities.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r856754359


##########
core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java:
##########
@@ -40,7 +46,11 @@ public void delete(long posStart, long posEnd) {
 
   @Override
   public boolean isDeleted(long position) {
-    return roaring64Bitmap.contains(position);
+    boolean posIsDeleted = roaring64Bitmap.contains(position);
+    if (counter != null && posIsDeleted) {
+      counter.increment();
+    }

Review Comment:
   Added a note to the javadoc for `PositionDeleteIndex#isDeleted`.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r855392863


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumDeletes.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.source.metrics;
+
+import java.text.NumberFormat;
+import org.apache.spark.sql.connector.metric.CustomMetric;
+
+public class NumDeletes implements CustomMetric {

Review Comment:
   I am aware of `CustomSumMetric` but there is no benefit in extending it instead of `CustomMetric`, since I would still override its `aggregateTaskMetrics` method as that returns the `String` representation of the long instead of a locale-formatted string as I do (and as other longs are shown in the Spark UI). So if there are 1,000 deletes, it should show as "1,000" and not "1000".



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r876406261


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeleteCount.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.source;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+import org.apache.spark.sql.vectorized.ColumnarBatch;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestSparkReaderDeleteCount extends SparkDeleteReadTestsBase {
+
+  private long deleteCount;
+
+  public TestSparkReaderDeleteCount(String format, boolean vectorized) {
+    super(format, vectorized);
+  }
+
+  @Parameterized.Parameters(name = "format = {0}, vectorized = {1}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] {"parquet", true},

Review Comment:
   Should there be a "parquet", false here as well?



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r877678381


##########
core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java:
##########
@@ -40,7 +46,11 @@ public void delete(long posStart, long posEnd) {
 
   @Override
   public boolean isDeleted(long position) {
-    return roaring64Bitmap.contains(position);
+    boolean posIsDeleted = roaring64Bitmap.contains(position);
+    if (counter != null && posIsDeleted) {
+      counter.increment();
+    }

Review Comment:
   Since it has been almost 4 weeks since I touched this (it seems that's how long the last update has gone unreviewed), I had forgotten the details of what I had investigated and had done. I went through the code paths again, and I see now that there is just one place where `PositionDeleteIndex#delete()` is called and where I need to track the delete count in the `PositionDeleteIndex`. So your suggestion is very feasible and not messy as I feared!
   Thank you for the suggestion. I'll update the PR tomorrow. 



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r937374663


##########
data/src/main/java/org/apache/iceberg/data/DeleteFilter.java:
##########
@@ -67,22 +72,26 @@
   private final List<DeleteFile> eqDeletes;
   private final Schema requiredSchema;
   private final Accessor<StructLike> posAccessor;
+  private final DeleteCounter counter;
 
   private PositionDeleteIndex deleteRowPositions = null;
-  private Predicate<T> eqDeleteRows = null;
 
-  protected DeleteFilter(String filePath, List<DeleteFile> deletes, Schema tableSchema, Schema requestedSchema) {
+  protected DeleteFilter(String filePath, List<DeleteFile> deletes, Schema tableSchema, Schema requestedSchema,
+      DeleteCounter counter) {

Review Comment:
   Update: The `DeleteCounter` is now created in `BaseReader` and passed to the instance of `BaseReader.SparkDeleteFilter` constructed in the `open(FileScanTask)` method `RowDataReader`/`BatchDataReader`. The argument still stands: we need a single counter in the reader which is used to aggregate the delete count over all the `FileScanTask`s.



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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r956525210


##########
core/src/main/java/org/apache/iceberg/deletes/DeleteCounter.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.deletes;
+
+/** A counter to be used to count deletes as they are applied. */
+public class DeleteCounter {
+
+  private long count = 0L;
+
+  /** Increment the counter by one. */
+  public void increment() {
+    count++;
+  }
+
+  /** Return the current value of the counter. */
+  public long get() {
+    return count;
+  }

Review Comment:
   Agreed. And I can see how the MetricsContext work took place alongside this work (or realistically after it).
   
   Unifying the two should be kept in mind, but the `DeleteCounter` is a well named, specific class that can’t really be mistaken for something else and can be updated later.
   
   Thanks for clarifying @wypoon 



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

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

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


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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r954355333


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java:
##########
@@ -362,6 +388,8 @@ public void testEqualityDeleteWithDeletedColumn() throws IOException {
         rowSet(tableName, PROJECTION_SCHEMA.asStruct(), "id", "data", "_deleted");
 
     Assert.assertEquals("Table should contain expected row", expected, actual);
+    long expectedDeletes = 3L;
+    checkDeleteCount(expectedDeletes);

Review Comment:
   Nit: seems not necessary to have them in two lines, but just my personal preference.



##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java:
##########
@@ -130,16 +140,19 @@ protected Table createTable(String name, Schema schema, PartitionSpec spec) {
     TableOperations ops = ((BaseTable) table).operations();
     TableMetadata meta = ops.current();
     ops.commit(meta, meta.upgradeToFormatVersion(2));
-    if (vectorized) {
+    table.updateProperties().set(TableProperties.DEFAULT_FILE_FORMAT, format).commit();
+    if (format.equals("parquet") && vectorized) {
       table
           .updateProperties()
           .set(TableProperties.PARQUET_VECTORIZATION_ENABLED, "true")
           .set(
               TableProperties.PARQUET_BATCH_SIZE,
               "4") // split 7 records to two batches to cover more code paths
           .commit();
-    } else {
+    } else if (format.equals("parquet")) { // in this case, non-vectorized
       table.updateProperties().set(TableProperties.PARQUET_VECTORIZATION_ENABLED, "false").commit();
+    } else if (format.equals("orc")) { // we only have non-vectorized for orc in our parameters
+      table.updateProperties().set(TableProperties.ORC_VECTORIZATION_ENABLED, "false").commit();

Review Comment:
   Minor suggestion: this will cover all cases, and no comment is needed.
   ```
       if (format.equals("parquet")) {
         table.updateProperties().set(TableProperties.PARQUET_VECTORIZATION_ENABLED, String.valueOf(vectorized)).commit();
         if (vectorized) {
           table.updateProperties().set(TableProperties.PARQUET_BATCH_SIZE, "4").commit();
         }
       } else if (format.equals("orc")) {
         table.updateProperties().set(TableProperties.ORC_VECTORIZATION_ENABLED, String.valueOf(vectorized)).commit();
         if (vectorized) {
           table.updateProperties().set(TableProperties.ORC_BATCH_SIZE, "4").commit();
         }
       }
   ```



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

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

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


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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r950762704


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java:
##########
@@ -183,6 +187,8 @@ Pair<int[], Integer> buildPosDelRowIdMapping(PositionDeleteIndex deletedRowPosit
           currentRowId++;
         } else if (hasIsDeletedColumn) {
           isDeleted[originalRowId] = true;
+        } else {
+          deletes.incrementDeleteCount();
         }

Review Comment:
   It is triggered by either projection or filtering. Here are two examples:
   ```
   select name, _deleted from student
   ```
   ```
   select name from student where _deleted=true
   ```
   It is true that user can get row-level deleted rows, users can also get both deleted and un-deleted rows with `_deleted` column, the first sql above returns that. The way I saw it is that the delete count metric is orthogonal to the metadata column `_deleted`. It reports how many row-level deletes applied as its label indicates, `public static final String DISPLAY_STRING = "number of row deletes applied"`. I don’t have strong option on it though. What do you think? BTW, the non-vectorized read also didn’t report the metric in case of `_deleted` column, see line 255 in class `DeleteFilter`.



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

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

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


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


[GitHub] [iceberg] wypoon commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1221193925

   @flyrain thanks for reviewing this again. I responded to your comments. Aside from the issue regarding counting the deletes for the case where there is a _is_deleted metadata column, I'll go ahead and make the changes.


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

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

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


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


[GitHub] [iceberg] pvary commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
pvary commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1236057132

   Thanks @wypoon for the PR and @flyrain, @kbendick and @RussellSpitzer for the reviews!


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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r961198753


##########
core/src/main/java/org/apache/iceberg/deletes/Deletes.java:
##########
@@ -74,9 +75,24 @@ public static <T> CloseableIterable<T> markDeleted(
         });
   }
 
+  /**
+   * Returns the remaining rows (the ones that are not deleted), while counting the deleted ones.

Review Comment:
   Fixed.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r956356267


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java:
##########
@@ -362,6 +388,8 @@ public void testEqualityDeleteWithDeletedColumn() throws IOException {
         rowSet(tableName, PROJECTION_SCHEMA.asStruct(), "id", "data", "_deleted");
 
     Assert.assertEquals("Table should contain expected row", expected, actual);
+    long expectedDeletes = 3L;
+    checkDeleteCount(expectedDeletes);

Review Comment:
   Fixed.



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

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

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


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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r956518890


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java:
##########
@@ -181,8 +181,11 @@ Pair<int[], Integer> buildPosDelRowIdMapping(PositionDeleteIndex deletedRowPosit
         if (!deletedRowPositions.isDeleted(originalRowId + rowStartPosInBatch)) {
           posDelRowIdMapping[currentRowId] = originalRowId;
           currentRowId++;
-        } else if (hasIsDeletedColumn) {
-          isDeleted[originalRowId] = true;
+        } else {
+          if (hasIsDeletedColumn) {
+            isDeleted[originalRowId] = true;
+          }
+          deletes.incrementDeleteCount();

Review Comment:
   Here is our discussion on this, https://github.com/apache/iceberg/pull/4588#discussion_r945959579. In general, I don't have strong option on either side, but prefer to consider this metric is orthogonal to metadata column `_deleted`.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r956524588


##########
core/src/main/java/org/apache/iceberg/deletes/DeleteCounter.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.deletes;
+
+/** A counter to be used to count deletes as they are applied. */
+public class DeleteCounter {
+
+  private long count = 0L;
+
+  /** Increment the counter by one. */
+  public void increment() {
+    count++;
+  }
+
+  /** Return the current value of the counter. */
+  public long get() {
+    return count;
+  }

Review Comment:
   This wasn't raised previously. I wasn't aware of `org.apache.iceberg.metrics.Counter` and in fact, it didn't exist when I started this work several months ago (although `org.apache.iceberg.metrics.MetricsContext.Counter` did exist).
   I think `org.apache.iceberg.metrics.Counter` is a more complex API than needed here (in particular, we do not need a name or a unit), although one could use it. I'd prefer not to refactor the code. A more generic counter could be used, but `DeleteCounter` is specific to this use case and is for internal bookkeeping only. 



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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r956526025


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java:
##########
@@ -181,8 +181,11 @@ Pair<int[], Integer> buildPosDelRowIdMapping(PositionDeleteIndex deletedRowPosit
         if (!deletedRowPositions.isDeleted(originalRowId + rowStartPosInBatch)) {
           posDelRowIdMapping[currentRowId] = originalRowId;
           currentRowId++;
-        } else if (hasIsDeletedColumn) {
-          isDeleted[originalRowId] = true;
+        } else {
+          if (hasIsDeletedColumn) {
+            isDeleted[originalRowId] = true;
+          }
+          deletes.incrementDeleteCount();

Review Comment:
   One small nit: Can we add a blank line between the if block and the metrics increment? It’s in-line with the style guide and I think would reduce the tendency to wonder if the final else is intentional.



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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r956525053


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java:
##########
@@ -181,8 +181,11 @@ Pair<int[], Integer> buildPosDelRowIdMapping(PositionDeleteIndex deletedRowPosit
         if (!deletedRowPositions.isDeleted(originalRowId + rowStartPosInBatch)) {
           posDelRowIdMapping[currentRowId] = originalRowId;
           currentRowId++;
-        } else if (hasIsDeletedColumn) {
-          isDeleted[originalRowId] = true;
+        } else {
+          if (hasIsDeletedColumn) {
+            isDeleted[originalRowId] = true;
+          }
+          deletes.incrementDeleteCount();

Review Comment:
   Oh I see. Yeah then we’d want to keep the metric whether we had the column or not.
   
   Thanks for pointing me in the right direction.



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

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

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


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


[GitHub] [iceberg] kbendick commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
kbendick commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1229045023

   > Great. Thanks @wypoon for working on this. Hi @RussellSpitzer and @kbendick, do you want to take another look and approve it? There is an authentication issue from my side that I still don't have the permission to merge even though I'm a committer now. Hopefully, my authentication issue could be resolved next week.
   
   Yeah sure let me do another pass. Congrats again on becoming a committer!


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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r952029325


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java:
##########
@@ -183,6 +187,8 @@ Pair<int[], Integer> buildPosDelRowIdMapping(PositionDeleteIndex deletedRowPosit
           currentRowId++;
         } else if (hasIsDeletedColumn) {
           isDeleted[originalRowId] = true;
+        } else {
+          deletes.incrementDeleteCount();
         }

Review Comment:
   I was mistaken about needing to modify `Deletes#markDeleted`, `Deletes#streamingMarker` and `Deletes.PositionStreamDeleteMarker`. I just needed to change `BaseReader.SparkDeleteFilter#markRowDeleted` to increment the counter there. `BaseReader.SparkDeleteFilter` keeps a `DeleteCounter` instance.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r856703592


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java:
##########
@@ -46,25 +47,34 @@
 import org.apache.iceberg.types.TypeUtil;
 import org.apache.spark.rdd.InputFileBlockHolder;
 import org.apache.spark.sql.catalyst.InternalRow;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 class RowDataReader extends BaseDataReader<InternalRow> {
-
+  private static final Logger LOG = LoggerFactory.getLogger(RowDataReader.class);
   private final Schema tableSchema;
   private final Schema expectedSchema;
   private final String nameMapping;
   private final boolean caseSensitive;
+  private final DeleteCounter counter;
 
   RowDataReader(CombinedScanTask task, Table table, Schema expectedSchema, boolean caseSensitive) {
     super(table, task);
     this.tableSchema = table.schema();
     this.expectedSchema = expectedSchema;
     this.nameMapping = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING);
     this.caseSensitive = caseSensitive;
+    this.counter = new DeleteCounter();
+  }
+
+  protected DeleteCounter counter() {
+    return counter;
   }
 
   @Override
   CloseableIterator<InternalRow> open(FileScanTask task) {
-    SparkDeleteFilter deletes = new SparkDeleteFilter(task, tableSchema, expectedSchema);
+    LOG.debug("calling open, constructing SparkDeleteFilter ...");

Review Comment:
   Removed.



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r855336574


##########
data/src/main/java/org/apache/iceberg/data/DeleteFilter.java:
##########
@@ -226,17 +261,18 @@ private CloseableIterable<T> applyPosDeletes(CloseableIterable<T> records) {
 
     // if there are fewer deletes than a reasonable number to keep in memory, use a set
     if (posDeletes.stream().mapToLong(DeleteFile::recordCount).sum() < setFilterThreshold) {
-      return Deletes.filter(records, this::pos, Deletes.toPositionIndex(filePath, deletes));
+      return Deletes.filter(records, this::pos, Deletes.toPositionIndex(filePath, deletes, counter));
     }
 
-    return Deletes.streamingFilter(records, this::pos, Deletes.deletePositions(filePath, deletes));
+    return Deletes.streamingFilter(records, this::pos, Deletes.deletePositions(filePath, deletes), counter);
   }
 
   private CloseableIterable<Record> openPosDeletes(DeleteFile file) {
     return openDeletes(file, POS_DELETE_SCHEMA);
   }
 
   private CloseableIterable<Record> openDeletes(DeleteFile deleteFile, Schema deleteSchema) {
+    LOG.trace("opening delete file {}", deleteFile.path());

Review Comment:
   Should start with a Capital



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r855486081


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java:
##########
@@ -139,18 +151,55 @@ protected void dropTable(String name) {
   }
 
   @Override
-  public StructLikeSet rowSet(String name, Table table, String... columns) {
-    Dataset<Row> df = spark.read()
-        .format("iceberg")
-        .load(TableIdentifier.of("default", name).toString())
-        .selectExpr(columns);
+  protected boolean countDeletes() {
+    return true;
+  }
 
-    Types.StructType projection = table.schema().select(columns).asStruct();
-    StructLikeSet set = StructLikeSet.create(projection);
-    df.collectAsList().forEach(row -> {
-      SparkStructLike rowWrapper = new SparkStructLike(projection);
-      set.add(rowWrapper.wrap(row));
-    });
+  private void setDeleteCount(long count) {
+    deleteCount = count;
+  }
+
+  @Override
+  protected long deleteCount() {
+    return deleteCount;
+  }
+
+  @Override
+  public StructLikeSet rowSet(String name, Table table, String... columns) throws IOException {
+    Schema schema = table.schema().select(columns);
+    StructType sparkSchema = SparkSchemaUtil.convert(schema);
+    Types.StructType type = schema.asStruct();
+    StructLikeSet set = StructLikeSet.create(type);
+
+    CloseableIterable<CombinedScanTask> tasks = TableScanUtil.planTasks(
+        table.newScan().planFiles(),
+        TableProperties.METADATA_SPLIT_SIZE_DEFAULT,
+        TableProperties.SPLIT_LOOKBACK_DEFAULT,
+        TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT);
+
+    long delCount = 0L;

Review Comment:
   Here is another option:
   I'll leave `TestSparkReaderDeletes` alone, and instead create a new class that also extends `DeleteReadTests` (or even `TestSparkReaderDeletes`) and override `rowSet` there. It is perhaps also not necessary to run all the tests in `DeleteReadTests` or `TestSparkReaderDeletes` in this new class. I had also increased the parameter space of `TestSparkReaderDeletes` in order to cover all 3 file formats, but I can dial it back and distribute the increased coverage to the new class. Basically, there will be no loss of test coverage.



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

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

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


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


[GitHub] [iceberg] pvary commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
pvary commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r960232909


##########
core/src/main/java/org/apache/iceberg/deletes/Deletes.java:
##########
@@ -215,19 +239,27 @@ boolean isDeleted(T row) {
   }
 
   private static class PositionStreamDeleteFilter<T> extends PositionStreamDeleteIterable<T> {
-    private PositionStreamDeleteFilter(
+    private final DeleteCounter counter;
+
+    PositionStreamDeleteFilter(
         CloseableIterable<T> rows,
         Function<T, Long> rowToPosition,
-        CloseableIterable<Long> deletePositions) {
+        CloseableIterable<Long> deletePositions,
+        DeleteCounter counter) {
       super(rows, rowToPosition, deletePositions);
+      this.counter = counter;
     }
 
     @Override
     protected CloseableIterator<T> applyDelete(CloseableIterator<T> items) {
       return new FilterIterator<T>(items) {
         @Override
         protected boolean shouldKeep(T item) {
-          return !isDeleted(item);
+          boolean deleted = isDeleted(item);
+          if (deleted) {
+            counter.increment();
+          }

Review Comment:
   Nit: new line 



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

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

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


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


[GitHub] [iceberg] pvary commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
pvary commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r960232615


##########
core/src/main/java/org/apache/iceberg/deletes/Deletes.java:
##########
@@ -74,9 +75,24 @@ public static <T> CloseableIterable<T> markDeleted(
         });
   }
 
+  /**
+   * Returns the remaining rows (the ones that are not deleted), while counting the deleted ones.
+   */
   public static <T> CloseableIterable<T> filterDeleted(
-      CloseableIterable<T> rows, Predicate<T> isDeleted) {
-    return CloseableIterable.filter(rows, isDeleted.negate());
+      CloseableIterable<T> rows, Predicate<T> isDeleted, DeleteCounter counter) {
+    Filter<T> remainingRowsFilter =
+        new Filter<T>() {
+          @Override
+          protected boolean shouldKeep(T item) {
+            boolean deleted = isDeleted.test(item);
+            if (deleted) {
+              counter.increment();
+            }

Review Comment:
   Nit: new line after the block



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

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

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


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


[GitHub] [iceberg] kbendick commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
kbendick commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1233292724

   > @flyrain @kbendick since you have approved this, can one of you please merge it? (The only update since is one adding some blank lines after if blocks and tweaking of comments in the test class.)
   
   Unfortunately I’m not able to merge yet as I’m not a committer. Hopefully either @flyrain’s credentials will get worked out or somebody else will handle it!
   
   And sorry for any delay in response, I’ve been out very sick this week but trying to stay on top of things when I have the energy. 😷 


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

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

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


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


[GitHub] [iceberg] pvary commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
pvary commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r960237699


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/SparkSQLExecutionHelper.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.source;
+
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.execution.ui.SQLAppStatusStore;
+import org.apache.spark.sql.execution.ui.SQLExecutionUIData;
+import org.apache.spark.sql.execution.ui.SQLPlanMetric;
+import org.junit.Assert;
+import scala.Option;
+
+public class SparkSQLExecutionHelper {
+
+  private SparkSQLExecutionHelper() {}
+
+  /**
+   * Finds the value of a specified metric for the last SQL query that was executed. Metric values
+   * are stored in the `SQLAppStatusStore` as strings.
+   *
+   * @param spark SparkSession used to run the SQL query
+   * @param metricName name of the metric
+   * @return value of the metric
+   */
+  public static String lastExecutedMetricValue(SparkSession spark, String metricName) {
+    SQLAppStatusStore statusStore = spark.sharedState().statusStore();
+    SQLExecutionUIData lastExecution = statusStore.executionsList().last();
+    Option<SQLPlanMetric> sqlPlanMetric =
+        lastExecution.metrics().find(metric -> metric.name().equals(metricName));
+    Assert.assertTrue(
+        String.format("Metric '%s' not found in last execution", metricName),
+        sqlPlanMetric.isDefined());
+    long metricId = sqlPlanMetric.get().accumulatorId();
+
+    // Refresh metricValues, they will remain null until the execution is complete and metrics are
+    // aggregated
+    int attempts = 3;
+    while (lastExecution.metricValues() == null && attempts > 0) {
+      try {
+        Thread.sleep(100);
+        attempts -= 1;
+      } catch (InterruptedException e) {
+        throw new RuntimeException(e);
+      }

Review Comment:
   Nit: newline 



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

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

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


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


[GitHub] [iceberg] pvary commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
pvary commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1233768759

   @wypoon: LGTM. Left a few formatting asks, nothing big. For the next time, AFAIK we do separate PRs for the different versions. We implement stuff first for the latest version and then backport to the older ones. This is generally easier for the reviewer.
   Thanks for the PR.
   Peter


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

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

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


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


[GitHub] [iceberg] flyrain commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
flyrain commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1235846065

   @wypoon I'd love to help on here. Unfortunately the issue is still there :-(. I wish secretary@apache.org could be a bit more responsive.


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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1130603098

   @flyrain could you take a look at this?


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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r878936151


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java:
##########
@@ -94,11 +141,15 @@ public static void startMetastoreAndSpark() {
 
     spark = SparkSession.builder()
         .master("local[2]")
+        .config("spark.appStateStore.asyncTracking.enable", false)

Review Comment:
   The default is true, which means that when the `SQLAppStatusListener` receives a `SparkListenerSQLExecutionEnd` event, it asks its `ElementTrackingStore` to asynchronously aggregate the metrics. This async call sometimes results in the metric we need not being available. To make the tests more predictable, ensure that a synchronous call to aggregate metrics is used.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r854398773


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnVectorWithFilter.java:
##########
@@ -97,9 +97,18 @@ public byte[] getBinary(int rowId) {
     return accessor().getBinary(rowIdMapping[rowId]);
   }
 
+  static ConstantColumnVector constantColumnVector(VectorHolder holder, int numRows) {
+    Object constant = ((ConstantVectorHolder) holder).getConstant();
+    if (constant instanceof Boolean) {
+      return new ConstantColumnVector(Types.BooleanType.get(), numRows, constant);
+    } else {
+      return new ConstantColumnVector(Types.IntegerType.get(), numRows, constant);
+    }
+  }
+
   public static ColumnVector forHolder(VectorHolder holder, int[] rowIdMapping, int numRows) {
     return holder.isDummy() ?
-        new ConstantColumnVector(Types.IntegerType.get(), numRows, ((ConstantVectorHolder) holder).getConstant()) :

Review Comment:
   @flyrain this code was added by you; can you please explain?



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r888122799


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java:
##########
@@ -152,9 +220,53 @@ public StructLikeSet rowSet(String name, Table table, String... columns) {
       set.add(rowWrapper.wrap(row));
     });
 
+    extractDeleteCount();
     return set;
   }
 
+  private void extractDeleteCount() {
+    // Get the executionId of the query we just executed
+    List<Long> executed = listener.executed();
+    long lastExecuted = executed.get(executed.size() - 1);
+    // Ensure that the execution end was registered

Review Comment:
   Oh this is great, Looking into the apis you dug up I think there may be an even easier way to do this. Sorry I didn't realize you could do this before
   
   Looking into sharedState you can do something like
   ```scala
   // Some operation
   scala> spark.range(1, 100).withColumn("x", col("id")).withColumnRenamed("id", "y").writeTo("local.default.test").append
   
   // UI Metrics
   scala> spark.sharedState.statusStore.executionsList.last.metrics
   res41: Seq[org.apache.spark.sql.execution.ui.SQLPlanMetric] = List(SQLPlanMetric(number of output rows,25,sum), SQLPlanMetric(duration,24,timing))
   
   // Description
   scala> spark.sharedState.statusStore.executionsList.last.description
   res42: String = append at <console>:23
   
   // Execution id
   scala> spark.sharedState.statusStore.executionsList.last.executionId
   res47: Long = 6
   ```
   
   So this block could be like
   ```java
     private long lastExecutedMetric(String metricName) {
       SQLExecutionUIData lastExecution = spark.sharedState().statusStore().executionsList().last();
       Option<SQLPlanMetric> sqlPlanMetric = lastExecution.metrics().find(metric -> metric.name().equals(metricName));
       Assert.assertTrue(String.format("Metric %s not present in last execution", metricName), sqlPlanMetric.isDefined());
       long metricId = sqlPlanMetric.get().accumulatorId();
   
       // Refresh metricValues, they will remain null until the execution is complete and metrics are aggregated
       int attempts = 3;
       while (lastExecution.metricValues() == null && attempts > 0) {
         try {
           Thread.sleep(100);
           attempts -= 1;
         } catch (InterruptedException e) {
           throw new RuntimeException(e);
         }
         lastExecution = spark.sharedState().statusStore().execution(lastExecution.executionId()).get();
       }
       Assert.assertNotNull(String.format("Metric %s was never finalized", metricName), lastExecution.metricValues());
       return Long.parseLong(lastExecution.metricValues().get(metricId).get());
     }
     ```
     
     Then we just have
     ```java
     @Override
     protected void checkDeleteCount(long expectedDeletes) {
       Assert.assertEquals("Unexpected number of deletes recorded", expectedDeletes,
           lastExecutedMetric(NumDeletes.DISPLAY_STRING));
     }
     ```
     
     And we can drop the other state in the test class, and we won't need the executionListener either I think?
   



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

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

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


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


[GitHub] [iceberg] wypoon commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1145105433

   > I think we would also really benefit from @flyrain doing a full review on this as well especially now that we have the "markDelete" pathway as well. I assume for that we probably will just skip counting deletes since we don't really care.
   
   @RussellSpitzer I saw that `Deletes` and `DeleteFilter` have changed in master since I updated this PR (so this PR no longer merges), but I haven't had the time to understand the changes and how to reconcile this PR with them.
   I don't understand your statement that "we probably will just skip counting deletes since we don't really care." Are you saying that this PR is no longer worthwhile??
   


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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1145200860

   Benchmarking could be good, although I'm gonna be real surprised if it's that heavy. I'll take bets on undetectable difference ... I'm assuming all the function calls will end up getting inlined during runtime optimizations


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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r888033125


##########
core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java:
##########
@@ -47,4 +47,8 @@ public boolean isDeleted(long position) {
   public boolean isEmpty() {
     return roaring64Bitmap.isEmpty();
   }
+
+  public long numberOfPositionsDeleted() {

Review Comment:
   I think this may be ok to include in the interface itself, I can't think of an implementation which couldn't track this?



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

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

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


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


[GitHub] [iceberg] wypoon commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1223406246

   @RussellSpitzer @flyrain I have updated the PR.
   I went ahead and implemented the necessary changes so that the number of deletes applied is counted whether or not there is a `_deleted` metadata column in the read.
   I have also addressed the other feedback.


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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r950768444


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java:
##########
@@ -183,6 +187,8 @@ Pair<int[], Integer> buildPosDelRowIdMapping(PositionDeleteIndex deletedRowPosit
           currentRowId++;
         } else if (hasIsDeletedColumn) {
           isDeleted[originalRowId] = true;
+        } else {
+          deletes.incrementDeleteCount();
         }

Review Comment:
   @flyrain thank you for explaining how reads with `hasIsDeletedColumn` occur. 
   I didn't really understand that before, but based on @RussellSpitzer's comment I cited, I figured it was fine not to count deletes for such reads. I have done this consistently for both vectorized and non-vectorized reads, as you realize. This is deliberate.
   
   Now that I understand the use case for `hasIsDeletedColumn`, I agree that there is merit to the argument for considering "number of row deletes applied" metric to apply regardless of `hasIsDeletedColumn`. On the other hand, for the queries where `hasIsDeletedColumn` is true, you are able to infer the number of row deletes applied from the result. I don't have a strong opinion on what to do in the `hasIsDeletedColumn` scenario. But the current implementation is consistent. @RussellSpitzer do you want to weigh in? 
   
   If we want to count the deletes always, we'll need to modify `Deletes#markDeleted`, `Deletes#streamingMarker`  and `Deletes.PositionStreamDeleteMarker` (we'll need to pass in the `DeleteCounter`) as well as the changes you mention for `ColumnarBatchReader`.
   



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

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

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


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


[GitHub] [iceberg] wypoon closed pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon closed pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan
URL: https://github.com/apache/iceberg/pull/4588


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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r956526049


##########
core/src/main/java/org/apache/iceberg/deletes/DeleteCounter.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.deletes;
+
+/** A counter to be used to count deletes as they are applied. */
+public class DeleteCounter {
+
+  private long count = 0L;
+
+  /** Increment the counter by one. */
+  public void increment() {
+    count++;
+  }
+
+  /** Return the current value of the counter. */
+  public long get() {
+    return count;
+  }

Review Comment:
   Yes, I think exposing this through `MetricsContext` could be valuable and such work would be a separate PR; at that time, the `DeleteCounter` could be replaced by the more general `Counter`.



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

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

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


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


[GitHub] [iceberg] flyrain commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
flyrain commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1229042584

   Great. Thanks @wypoon for working on this.  Hi @RussellSpitzer and @kbendick, do you want to take another look and approve it? There is an authentication issue from my side that I still don't have the permission to merge even though I'm a committer now. Hopefully, my authentication issue could be resolved next week.


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

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

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


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


[GitHub] [iceberg] wypoon commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1229252076

   @kbendick thanks for reviewing. I have addressed some minor nits.


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

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

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


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


[GitHub] [iceberg] wypoon commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1233132529

   @flyrain @kbendick since you have approved this, can one of you please merge it? (The only update since is one adding some blank lines after if blocks and tweaking of comments in the test class.)


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

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

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


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


[GitHub] [iceberg] wypoon commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1204339304

   I ran `IcebergSourceParquetPosDeleteBenchmark` and `IcebergSourceParquetEqDeleteBenchmark` on the state of master when I last rebased and on my last change. The results for `readIceberg` and `readIcebergVectorized` are as follows:
   <img width="1395" alt="Screen Shot 2022-08-03 at 11 13 05 AM" src="https://user-images.githubusercontent.com/3925490/182683434-1f1a6583-703f-4b96-a671-9049a1ea46ab.png">
   
   There is hardly any difference for `IcebergSourceParquetPosDeleteBenchmark`.
   For `IcebergSourceParquetEqDeleteBenchmark`, the numbers show greater difference (some worse and some better, comparing after to before), but due to the much greater variances, I think the differences are within the variances.


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

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

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


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


[GitHub] [iceberg] wypoon commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1167867166

   @flyrain thanks for reviewing. I'll look into your comments.


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

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

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


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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r904452766


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java:
##########
@@ -133,6 +136,7 @@ void initRowIdMapping(int numRowsToRead) {
 
     Pair<int[], Integer> posDelRowIdMapping(int numRowsToRead) {
       if (deletes != null && deletes.hasPosDeletes()) {
+        LOG.debug("Building row id mapping from positional deletes");

Review Comment:
   Why do we need a debug log here? 



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

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

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


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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r905348730


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java:
##########
@@ -195,20 +199,8 @@ int[] initEqDeleteRowIdMapping(int numRowsToRead) {
      */
     void applyEqDelete() {
       Iterator<InternalRow> it = columnarBatch.rowIterator();
-      int rowId = 0;
-      int currentRowId = 0;
-      while (it.hasNext()) {
-        InternalRow row = it.next();
-        if (deletes.eqDeletedRowFilter().test(row)) {
-          // the row is NOT deleted
-          // skip deleted rows by pointing to the next undeleted row Id
-          rowIdMapping[currentRowId] = rowIdMapping[rowId];
-          currentRowId++;
-        }
-
-        rowId++;
-      }
-
+      LOG.debug("Applying equality deletes to row id mapping");
+      int currentRowId = deletes.applyEqDeletes(it, rowIdMapping);

Review Comment:
   I'd prefer not to move this logic to `DeleteFilter` since it is more relevant here. We may expose DeleteFilter's DeleteCounter as you suggested.



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

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

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


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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r905359413


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

Review Comment:
   +1 for the new class.



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

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

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


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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r905358295


##########
core/src/main/java/org/apache/iceberg/deletes/Deletes.java:
##########
@@ -202,17 +233,24 @@ boolean isDeleted(T row) {
   }
 
   private static class PositionStreamDeleteFilter<T> extends PositionStreamDeleteIterable<T> {
-    private PositionStreamDeleteFilter(CloseableIterable<T> rows, Function<T, Long> rowToPosition,
-                                       CloseableIterable<Long> deletePositions) {
+    private final DeleteCounter counter;
+
+    PositionStreamDeleteFilter(CloseableIterable<T> rows, Function<T, Long> rowToPosition,
+                               CloseableIterable<Long> deletePositions, DeleteCounter counter) {
       super(rows, rowToPosition, deletePositions);
+      this.counter = counter;
     }
 
     @Override
     protected CloseableIterator<T> applyDelete(CloseableIterator<T> items) {
       return new FilterIterator<T>(items) {
         @Override
         protected boolean shouldKeep(T item) {
-          return !isDeleted(item);
+          boolean deleted = isDeleted(item);
+          if (deleted) {
+            counter.increment();
+          }
+          return !deleted;

Review Comment:
   This may have perf impact since it is per-row basis. I don't think it is big though, but would recommend to use jmh to micro benchmark it.



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

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

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


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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r905361152


##########
core/src/main/java/org/apache/iceberg/deletes/Deletes.java:
##########
@@ -75,8 +75,21 @@ public static <T> CloseableIterable<T> markDeleted(CloseableIterable<T> rows, Pr
     });
   }
 
-  public static <T> CloseableIterable<T> filterDeleted(CloseableIterable<T> rows, Predicate<T> isDeleted) {
-    return CloseableIterable.filter(rows, isDeleted.negate());
+  // This returns the remaining rows (the ones that are not deleted) while counting the deleted ones
+  public static <T> CloseableIterable<T> filterDeleted(CloseableIterable<T> rows, Predicate<T> isDeleted,
+                                                       DeleteCounter counter) {
+    Filter<T> remainingRowsFilter = new Filter<T>() {
+      @Override
+      protected boolean shouldKeep(T item) {
+        boolean deleted = isDeleted.test(item);
+        if (deleted) {
+          counter.increment();
+        }
+        return !deleted;

Review Comment:
   The same perf concern here.



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

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

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


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


[GitHub] [iceberg] wypoon commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1235020008

   Rebased on master, and fixed the newly added `ChangelogRowReader`'s use of `SparkDeleteFilter`. @flyrain can you please take a quick look at that small change? Otherwise, the only other changes are in formatting and comments.


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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r878936581


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java:
##########
@@ -152,9 +220,53 @@ public StructLikeSet rowSet(String name, Table table, String... columns) {
       set.add(rowWrapper.wrap(row));
     });
 
+    extractDeleteCount();
     return set;
   }
 
+  private void extractDeleteCount() {
+    // Get the executionId of the query we just executed
+    List<Long> executed = listener.executed();
+    long lastExecuted = executed.get(executed.size() - 1);
+    // Ensure that the execution end was registered

Review Comment:
   Wait for the `SparkListenerSQLExecutionEnd` to be received. If our `SparkListener` has received the event, the `SQLAppStatusListener` should have received it too and called its `ElementTrackingStore` to aggregate the metrics.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r878937413


##########
core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java:
##########
@@ -40,7 +46,11 @@ public void delete(long posStart, long posEnd) {
 
   @Override
   public boolean isDeleted(long position) {
-    return roaring64Bitmap.contains(position);
+    boolean posIsDeleted = roaring64Bitmap.contains(position);
+    if (counter != null && posIsDeleted) {
+      counter.increment();
+    }

Review Comment:
   @kbendick I considered using an `Optional<DeleteCounter>` instead of a `DeleteCounter` that could be null, but it's six of one and half a dozen of the other. In the end, I adopted your suggestion to have a no-op `DeleteCounter`.



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

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

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


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


[GitHub] [iceberg] flyrain commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
flyrain commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1145147058

   Sorry, I haven't got chance to review. The changes of class `Deletes` and `DeleteFilter` are introduced by my PR #4683, which can read deleted rows by using metadata column `_deleted`. It may affect how we collect the deletes metrics. Would recommend to take a look at #4683, @wypoon. I will also review this PR a bit later.


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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r888122799


##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java:
##########
@@ -152,9 +220,53 @@ public StructLikeSet rowSet(String name, Table table, String... columns) {
       set.add(rowWrapper.wrap(row));
     });
 
+    extractDeleteCount();
     return set;
   }
 
+  private void extractDeleteCount() {
+    // Get the executionId of the query we just executed
+    List<Long> executed = listener.executed();
+    long lastExecuted = executed.get(executed.size() - 1);
+    // Ensure that the execution end was registered

Review Comment:
   Oh this is great, Looking into the apis you dug up I think there may be an even easier way to do this. Sorry I didn't realize you could do this before
   
   Looking into sharedState you can do something like
   ```scala
   // Some operation
   scala> spark.range(1, 100).withColumn("x", col("id")).withColumnRenamed("id", "y").writeTo("local.default.test").append
   
   // UI Metrics
   scala> spark.sharedState.statusStore.executionsList.last.metrics
   res41: Seq[org.apache.spark.sql.execution.ui.SQLPlanMetric] = List(SQLPlanMetric(number of output rows,25,sum), SQLPlanMetric(duration,24,timing))
   
   // Description
   scala> spark.sharedState.statusStore.executionsList.last.description
   res42: String = append at <console>:23
   
   // Execution id
   scala> spark.sharedState.statusStore.executionsList.last.executionId
   res47: Long = 6
   ```
   
   So this block could be like
   ```java
     private long lastExecutedMetric(String metricName) {
       SQLExecutionUIData lastExecution = spark.sharedState().statusStore().executionsList().last();
       Option<SQLPlanMetric> sqlPlanMetric = lastExecution.metrics().find(metric -> metric.name().equals(metricName));
       Assert.assertTrue("Metric xxxx not present in last execution", sqlPlanMetric.isDefined());
       long metricId = sqlPlanMetric.get().accumulatorId();
   
       // Refresh metricValues, they will remain null until the execution is complete and metrics are aggregated
       int attempts = 3;
       while (lastExecution.metricValues() == null && attempts > 0) {
         try {
           Thread.sleep(100);
           attempts -= 1;
         } catch (InterruptedException e) {
           throw new RuntimeException(e);
         }
         lastExecution = spark.sharedState().statusStore().execution(lastExecution.executionId()).get();
       }
       Assert.assertNotNull(String.format("Metric %s was never finalized", metricName), lastExecution.metricValues());
       return Long.parseLong(lastExecution.metricValues().get(metricId).get());
     }
     ```
     
     Then we just have
     ```
       @Override
     protected void checkDeleteCount(long expectedDeletes) {
       Assert.assertEquals("Unexpected number of deletes recorded", expectedDeletes,
           lastExecutedMetric(NumDeletes.DISPLAY_STRING));
     }
     ```
     
     And we can drop the other state in the test class.
   



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r889385568


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java:
##########
@@ -195,20 +199,8 @@ int[] initEqDeleteRowIdMapping(int numRowsToRead) {
      */
     void applyEqDelete() {
       Iterator<InternalRow> it = columnarBatch.rowIterator();
-      int rowId = 0;
-      int currentRowId = 0;
-      while (it.hasNext()) {
-        InternalRow row = it.next();
-        if (deletes.eqDeletedRowFilter().test(row)) {
-          // the row is NOT deleted
-          // skip deleted rows by pointing to the next undeleted row Id
-          rowIdMapping[currentRowId] = rowIdMapping[rowId];
-          currentRowId++;
-        }
-
-        rowId++;
-      }
-
+      LOG.debug("Applying equality deletes to row id mapping");
+      int currentRowId = deletes.applyEqDeletes(it, rowIdMapping);

Review Comment:
   Yes and no. There doesn't seem to be a natural way to get the `DeleteCounter` to `ColumnarBatchReader`. In this method, the `rowIdMapping` int array is updated by applying the equality deletes in the `DeleteFilter` `deletes`. As each row is tested, if it is deleted, we need to increment the `DeleteCounter`. The `DeleteFilter` has the `DeleteCounter`, so I moved the logic completely over to `DeleteFilter`. The alternative is to expose `DeleteFilter`'s `DeleteCounter` so that it can be used here, but I think it is better to encapsulate that in `DeleteFilter`.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r889381963


##########
data/src/main/java/org/apache/iceberg/data/DeleteFilter.java:
##########
@@ -212,7 +244,7 @@ public PositionDeleteIndex deletedRowPositions() {
 
     if (deleteRowPositions == null) {
       List<CloseableIterable<Record>> deletes = Lists.transform(posDeletes, this::openPosDeletes);
-      deleteRowPositions = Deletes.toPositionIndex(filePath, deletes);
+      deleteRowPositions = Deletes.toPositionIndex(filePath, deletes, counter);

Review Comment:
   Actually, this eventually calls `Deletes.toPositionIndex(CloseableIterable<Long>, DeleteCounter)` where we do
   ```
         PositionDeleteIndex positionDeleteIndex = new BitmapPositionDeleteIndex();
         deletes.forEach(positionDeleteIndex::delete);
         counter.increment(((BitmapPositionDeleteIndex) positionDeleteIndex).numberOfPositionsDeleted());
         return positionDeleteIndex;
   ```
   That method is called by other callers besides this. I think it is best to do the `counter.increment(positionDeleteIndex.numberOfPositionsDeleted())` right where `PositionDeleteIndex#delete` is called. I think the bookkeeping is easier (to reason about) that way.



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

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

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


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


[GitHub] [iceberg] wypoon commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1148191069

   @RussellSpitzer @flyrain I have rebased on master and reconciled the conflicts with #4683. Counting deletes only happen for the normal read path, not the new path added by #4683.
   Russell, I have adopted your suggestions for `TestSparkReaderDeletes`. For the other questions/points you raised, I have answered them earlier.
   I don't think this work has any performance impact. The costs of incrementing an in-memory counter are dwarfed by the costs of opening and reading delete files.


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

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

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


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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r854465525


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnVectorWithFilter.java:
##########
@@ -97,9 +97,18 @@ public byte[] getBinary(int rowId) {
     return accessor().getBinary(rowIdMapping[rowId]);
   }
 
+  static ConstantColumnVector constantColumnVector(VectorHolder holder, int numRows) {
+    Object constant = ((ConstantVectorHolder) holder).getConstant();
+    if (constant instanceof Boolean) {
+      return new ConstantColumnVector(Types.BooleanType.get(), numRows, constant);
+    } else {
+      return new ConstantColumnVector(Types.IntegerType.get(), numRows, constant);
+    }
+  }
+
   public static ColumnVector forHolder(VectorHolder holder, int[] rowIdMapping, int numRows) {
     return holder.isDummy() ?
-        new ConstantColumnVector(Types.IntegerType.get(), numRows, ((ConstantVectorHolder) holder).getConstant()) :

Review Comment:
   I copied some code from class `IcebergArrowColumnVector`, and didn't change its construction logic. But I agreed with you that we shouldn't hard-code to int type.



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r855346357


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java:
##########
@@ -93,6 +93,11 @@ class SparkBatchQueryScan extends SparkScan implements SupportsRuntimeFiltering
     }
   }
 
+  @Override
+  public String description() {
+    return "IcebergBatchScan " + super.description();

Review Comment:
   Not sure how these changes are related to the PR



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

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

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


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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r855347832


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java:
##########
@@ -193,15 +204,41 @@ public boolean supportColumnarReads(InputPartition partition) {
     }
   }
 
+  static long numFilesToScan(CombinedScanTask scanTask) {

Review Comment:
   should be number of splits now?



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

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

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


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


[GitHub] [iceberg] kbendick commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r856665333


##########
core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java:
##########
@@ -40,7 +46,11 @@ public void delete(long posStart, long posEnd) {
 
   @Override
   public boolean isDeleted(long position) {
-    return roaring64Bitmap.contains(position);
+    boolean posIsDeleted = roaring64Bitmap.contains(position);
+    if (counter != null && posIsDeleted) {
+      counter.increment();
+    }

Review Comment:
   Given that we have defined the `DeleteCounter` interface ourselves, does it make sense to define a default `NullDeleteCounter` that is a no-op?
   
   There are a number of places where we check `if (counter != null)`.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r856699808


##########
data/src/main/java/org/apache/iceberg/data/DeleteFilter.java:
##########
@@ -188,31 +202,52 @@ private CloseableIterable<T> applyEqDeletes(CloseableIterable<T> records) {
     Filter<T> remainingRowsFilter = new Filter<T>() {
       @Override
       protected boolean shouldKeep(T item) {
-        return remainingRows.test(item);
+        boolean keep = remainingRows.test(item); // false means row is deleted
+        if (!keep && counter != null) {
+          counter.increment();
+        }
+        return keep;
       }
     };
 
     return remainingRowsFilter.filter(records);
   }
 
-  public Predicate<T> eqDeletedRowFilter() {
-    if (eqDeleteRows == null) {
-      eqDeleteRows = applyEqDeletes().stream()
-          .map(Predicate::negate)
-          .reduce(Predicate::and)
-          .orElse(t -> true);
+  public int applyEqDeletes(Iterator<T> records, int[] mapping) {
+    Predicate<T> remainingRows = applyEqDeletes().stream()
+        .map(Predicate::negate)
+        .reduce(Predicate::and)
+        .orElse(t -> true);
+
+    int rowId = 0;
+    int currentRowId = 0;
+    while (records.hasNext()) {
+      T row = records.next();
+      if (remainingRows.test(row)) {
+        // the row is NOT deleted
+        // skip deleted rows by pointing to the next undeleted row Id
+        mapping[currentRowId] = mapping[rowId];
+        currentRowId++;
+      } else if (counter != null) {
+        counter.increment();
+      }
+
+      rowId++;
     }
-    return eqDeleteRows;
+
+    return currentRowId;
   }
 
   public PositionDeleteIndex deletedRowPositions() {
+    LOG.debug("calling deletedRowPositions()");

Review Comment:
   Removed.



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

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

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


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


[GitHub] [iceberg] wypoon commented on a diff in pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on code in PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#discussion_r856702210


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java:
##########
@@ -75,6 +86,7 @@ CloseableIterator<ColumnarBatch> open(FileScanTask task) {
     InputFile location = getInputFile(task);
     Preconditions.checkNotNull(location, "Could not find InputFile associated with FileScanTask");
     if (task.file().format() == FileFormat.PARQUET) {
+      LOG.debug("data file is parquet; calling deleteFilter to get SparkDeleteFilter ...");

Review Comment:
   Removed.



##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java:
##########
@@ -126,7 +138,8 @@ CloseableIterator<ColumnarBatch> open(FileScanTask task) {
   }
 
   private SparkDeleteFilter deleteFilter(FileScanTask task) {
-    return task.deletes().isEmpty() ? null : new SparkDeleteFilter(task, table().schema(), expectedSchema);
+    LOG.debug("is task.deletes() empty? {}", task.deletes().isEmpty());

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

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

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


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


[GitHub] [iceberg] wypoon commented on pull request #4588: Spark: Add custom metric for number of deletes applied by a SparkScan

Posted by GitBox <gi...@apache.org>.
wypoon commented on PR #4588:
URL: https://github.com/apache/iceberg/pull/4588#issuecomment-1117597202

   @RussellSpitzer please review again when you can. Thanks.


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

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

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


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