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/01/29 07:46:22 UTC

[GitHub] [iceberg] coolderli opened a new pull request #4005: Core: Add delete_file_count for PartitionsTable

coolderli opened a new pull request #4005:
URL: https://github.com/apache/iceberg/pull/4005


   At present, the `PartitionsTable` only contains `file_count`, when using the v2 table, we can't know the count of delete files on each partition. In this pr, I add `delete_frile_count` to `PartitionsTable`, so we can know the count of data files and delete files when we query `select * from prod.db.table.partitions`. 


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

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

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



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


[GitHub] [iceberg] coolderli commented on a change in pull request #4005: Core: Add delete file details for PartitionsTable

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



##########
File path: core/src/main/java/org/apache/iceberg/PartitionsTable.java
##########
@@ -48,7 +48,10 @@
     this.schema = new Schema(
         Types.NestedField.required(1, "partition", table.spec().partitionType()),
         Types.NestedField.required(2, "record_count", Types.LongType.get()),
-        Types.NestedField.required(3, "file_count", Types.IntegerType.get())
+        Types.NestedField.required(3, "file_count", Types.IntegerType.get()),
+        Types.NestedField.required(4, "delete_file_count", Types.IntegerType.get()),
+        Types.NestedField.required(5, "equality_delete_count", Types.LongType.get()),

Review comment:
       file_size, equality_delete_file_size and position_delete_file_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] coolderli commented on a change in pull request #4005: Core: Add delete file details for PartitionsTable

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



##########
File path: core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java
##########
@@ -516,6 +522,101 @@ public void testPartitionColumnNamedPartition() throws Exception {
     validateIncludesPartitionScan(tasksAndEq, 0);
   }
 
+  @Test
+  public void testPartitionsTableScanWithDeleteFilesInFilter() throws IOException {
+    Assume.assumeTrue(formatVersion == 2);
+    Configuration conf = new Configuration();
+    HadoopTables tables = new HadoopTables(conf);

Review comment:
       I have tried to use `new PartitionTable(table.ops(), table)`, but it return a NPE on `ops.current().metadataFileLocation()`. https://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/PartitionsTable.java#L84




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

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

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



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


[GitHub] [iceberg] szehon-ho commented on a change in pull request #4005: Core: Add delete file details for PartitionsTable

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on a change in pull request #4005:
URL: https://github.com/apache/iceberg/pull/4005#discussion_r832520019



##########
File path: core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java
##########
@@ -516,6 +522,101 @@ public void testPartitionColumnNamedPartition() throws Exception {
     validateIncludesPartitionScan(tasksAndEq, 0);
   }
 
+  @Test
+  public void testPartitionsTableScanWithDeleteFilesInFilter() throws IOException {
+    Assume.assumeTrue(formatVersion == 2);
+    Configuration conf = new Configuration();
+    HadoopTables tables = new HadoopTables(conf);

Review comment:
       That's strange, actually it works in most of the tests.  Do you do it after populating the table, like the other 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] coolderli commented on a change in pull request #4005: Core: Add delete_file_count for PartitionsTable

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



##########
File path: core/src/main/java/org/apache/iceberg/PartitionsTable.java
##########
@@ -167,16 +172,25 @@ Partition get(StructLike key) {
     private final StructLike key;
     private long recordCount;
     private int fileCount;
+    private int deleteFileCount;
 
     Partition(StructLike key) {
       this.key = key;
       this.recordCount = 0;
       this.fileCount = 0;
+      this.deleteFileCount = 0;
     }
 
     void update(DataFile file) {
       this.recordCount += file.recordCount();
       this.fileCount += 1;
     }
+
+    void updateDeleteFile(List<DeleteFile> deleteFiles) {

Review comment:
       Yes, if there is a metadata table that can load delete files, I think this is reasonable.




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

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

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



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


[GitHub] [iceberg] coolderli commented on a change in pull request #4005: Core: Add delete file details for PartitionsTable

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



##########
File path: core/src/main/java/org/apache/iceberg/PartitionsTable.java
##########
@@ -48,7 +48,10 @@
     this.schema = new Schema(
         Types.NestedField.required(1, "partition", table.spec().partitionType()),
         Types.NestedField.required(2, "record_count", Types.LongType.get()),
-        Types.NestedField.required(3, "file_count", Types.IntegerType.get())
+        Types.NestedField.required(3, "file_count", Types.IntegerType.get()),
+        Types.NestedField.required(4, "delete_file_count", Types.IntegerType.get()),
+        Types.NestedField.required(5, "equality_delete_count", Types.LongType.get()),

Review comment:
       I agree the name should be clear. But I use the old name is to keep consistent with [SnapshotSummary.](https://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/SnapshotSummary.java#L30)




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #4005: Core: Add delete_file_count for PartitionsTable

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



##########
File path: core/src/main/java/org/apache/iceberg/PartitionsTable.java
##########
@@ -167,16 +172,25 @@ Partition get(StructLike key) {
     private final StructLike key;
     private long recordCount;
     private int fileCount;
+    private int deleteFileCount;
 
     Partition(StructLike key) {
       this.key = key;
       this.recordCount = 0;
       this.fileCount = 0;
+      this.deleteFileCount = 0;
     }
 
     void update(DataFile file) {
       this.recordCount += file.recordCount();
       this.fileCount += 1;
     }
+
+    void updateDeleteFile(List<DeleteFile> deleteFiles) {

Review comment:
       I think I would prefer to do this a little differently. Instead of updating the caller and passing both `task.file()` and `task.deletes()` separately, why not combine these into `update(FileScanTask)` and update both the file and deletes at once?




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

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

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



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


[GitHub] [iceberg] coolderli commented on a change in pull request #4005: Core: Add delete_file_count for PartitionsTable

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



##########
File path: core/src/main/java/org/apache/iceberg/PartitionsTable.java
##########
@@ -167,16 +172,25 @@ Partition get(StructLike key) {
     private final StructLike key;
     private long recordCount;
     private int fileCount;
+    private int deleteFileCount;
 
     Partition(StructLike key) {
       this.key = key;
       this.recordCount = 0;
       this.fileCount = 0;
+      this.deleteFileCount = 0;
     }
 
     void update(DataFile file) {
       this.recordCount += file.recordCount();
       this.fileCount += 1;
     }
+
+    void updateDeleteFile(List<DeleteFile> deleteFiles) {
+      for (DeleteFile deleteFile : deleteFiles) {
+        this.recordCount += deleteFile.recordCount();

Review comment:
       Thanks, I'll fix 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] coolderli commented on a change in pull request #4005: Core: Add delete_file_count for PartitionsTable

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



##########
File path: core/src/main/java/org/apache/iceberg/PartitionsTable.java
##########
@@ -167,16 +172,25 @@ Partition get(StructLike key) {
     private final StructLike key;
     private long recordCount;
     private int fileCount;
+    private int deleteFileCount;
 
     Partition(StructLike key) {
       this.key = key;
       this.recordCount = 0;
       this.fileCount = 0;
+      this.deleteFileCount = 0;
     }
 
     void update(DataFile file) {
       this.recordCount += file.recordCount();
       this.fileCount += 1;
     }
+
+    void updateDeleteFile(List<DeleteFile> deleteFiles) {

Review comment:
       Yes, I think it is reasonable.




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

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

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



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


[GitHub] [iceberg] coolderli commented on a change in pull request #4005: Core: Add delete_file_count for PartitionsTable

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



##########
File path: core/src/main/java/org/apache/iceberg/PartitionsTable.java
##########
@@ -167,16 +172,25 @@ Partition get(StructLike key) {
     private final StructLike key;
     private long recordCount;
     private int fileCount;
+    private int deleteFileCount;
 
     Partition(StructLike key) {
       this.key = key;
       this.recordCount = 0;
       this.fileCount = 0;
+      this.deleteFileCount = 0;
     }
 
     void update(DataFile file) {
       this.recordCount += file.recordCount();
       this.fileCount += 1;
     }
+
+    void updateDeleteFile(List<DeleteFile> deleteFiles) {
+      for (DeleteFile deleteFile : deleteFiles) {
+        this.recordCount += deleteFile.recordCount();

Review comment:
       > Thanks, @coolderli! This is going to be really helpful. Another thing that we should do is add a new metadata table that loads just delete files. Are you interested in working on that as well?
   
   Yes, I'll do that. I will fix this and push another pr for that 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] coolderli commented on pull request #4005: Core: Add delete file details for PartitionsTable

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


   @rdblue I have fixed it, could you review it again, 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] coolderli commented on a change in pull request #4005: Core: Add delete file details for PartitionsTable

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



##########
File path: core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java
##########
@@ -516,6 +522,101 @@ public void testPartitionColumnNamedPartition() throws Exception {
     validateIncludesPartitionScan(tasksAndEq, 0);
   }
 
+  @Test
+  public void testPartitionsTableScanWithDeleteFilesInFilter() throws IOException {
+    Assume.assumeTrue(formatVersion == 2);
+    Configuration conf = new Configuration();
+    HadoopTables tables = new HadoopTables(conf);
+    File tableLocation = new File(temp.newFolder(), "table");
+    Table table = tables.create(

Review comment:
       I use HadoopTable to load the PartitionTable, so I can read the records. 




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #4005: Core: Add delete file details for PartitionsTable

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



##########
File path: core/src/main/java/org/apache/iceberg/PartitionsTable.java
##########
@@ -167,16 +175,30 @@ Partition get(StructLike key) {
     private final StructLike key;
     private long recordCount;
     private int fileCount;
+    private int deleteFileCount;
+    private long equalityDeleteCount;
+    private long positionDeleteCount;
 
     Partition(StructLike key) {
       this.key = key;
       this.recordCount = 0;
       this.fileCount = 0;
+      this.deleteFileCount = 0;
+      this.equalityDeleteCount = 0L;
+      this.positionDeleteCount = 0L;
     }
 
-    void update(DataFile file) {
-      this.recordCount += file.recordCount();
+    void update(FileScanTask task) {
+      this.recordCount += task.file().recordCount();
       this.fileCount += 1;
+      this.deleteFileCount += task.deletes().size();

Review comment:
       +1 to this. This could definitely have duplicates!




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

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

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



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


[GitHub] [iceberg] coolderli commented on a change in pull request #4005: Core: Add delete file details for PartitionsTable

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



##########
File path: core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java
##########
@@ -516,6 +522,101 @@ public void testPartitionColumnNamedPartition() throws Exception {
     validateIncludesPartitionScan(tasksAndEq, 0);
   }
 
+  @Test
+  public void testPartitionsTableScanWithDeleteFilesInFilter() throws IOException {
+    Assume.assumeTrue(formatVersion == 2);
+    Configuration conf = new Configuration();
+    HadoopTables tables = new HadoopTables(conf);

Review comment:
       Yes, I checked again. I think the problem is that TestTableOperations does not write metadata.json file:https://github.com/apache/iceberg/blob/master/core/src/test/java/org/apache/iceberg/TestTables.java#L216.
   So the metadata location is always null. When reading the records. it should plan tasks,so the metadata.json should be used [here](https://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/PartitionsTable.java#L78).
   So I create a HadoopTable because HadoopTableOperation will create the metadata.json [here](https://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java#L151)




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

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

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



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


[GitHub] [iceberg] szehon-ho commented on a change in pull request #4005: Core: Add delete file details for PartitionsTable

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on a change in pull request #4005:
URL: https://github.com/apache/iceberg/pull/4005#discussion_r808661327



##########
File path: core/src/main/java/org/apache/iceberg/PartitionsTable.java
##########
@@ -167,16 +175,30 @@ Partition get(StructLike key) {
     private final StructLike key;
     private long recordCount;
     private int fileCount;
+    private int deleteFileCount;
+    private long equalityDeleteCount;
+    private long positionDeleteCount;
 
     Partition(StructLike key) {
       this.key = key;
       this.recordCount = 0;
       this.fileCount = 0;
+      this.deleteFileCount = 0;
+      this.equalityDeleteCount = 0L;
+      this.positionDeleteCount = 0L;
     }
 
-    void update(DataFile file) {
-      this.recordCount += file.recordCount();
+    void update(FileScanTask task) {
+      this.recordCount += task.file().recordCount();
       this.fileCount += 1;
+      this.deleteFileCount += task.deletes().size();

Review comment:
       Would this ever have duplicates?  Example multiple fileScanTask within the same partition referring to the same delete file, can it happen?

##########
File path: core/src/main/java/org/apache/iceberg/PartitionsTable.java
##########
@@ -48,7 +48,10 @@
     this.schema = new Schema(
         Types.NestedField.required(1, "partition", table.spec().partitionType()),
         Types.NestedField.required(2, "record_count", Types.LongType.get()),
-        Types.NestedField.required(3, "file_count", Types.IntegerType.get())
+        Types.NestedField.required(3, "file_count", Types.IntegerType.get()),
+        Types.NestedField.required(4, "delete_file_count", Types.IntegerType.get()),
+        Types.NestedField.required(5, "equality_delete_count", Types.LongType.get()),

Review comment:
       There might be a little confusion in the name (I thought it means the equality_delete_files and position_delete_files count at first).  Maybe we can change to "equality_delete_record_count" and "position_delete_record_count".
   
   Not sure what others think 

##########
File path: core/src/main/java/org/apache/iceberg/PartitionsTable.java
##########
@@ -167,16 +172,25 @@ Partition get(StructLike key) {
     private final StructLike key;
     private long recordCount;
     private int fileCount;
+    private int deleteFileCount;
 
     Partition(StructLike key) {
       this.key = key;
       this.recordCount = 0;
       this.fileCount = 0;

Review comment:
       Yea, this does seem like it should be renamed "file_count" to "data_file_count" if we can...(not sure the policy here), agree the original name is unfortunate.

##########
File path: core/src/test/java/org/apache/iceberg/TestPartitionTableScans.java
##########
@@ -0,0 +1,145 @@
+/*
+ * 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;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.TableTestBase.FILE_A;
+import static org.apache.iceberg.TableTestBase.FILE_A2_DELETES;
+import static org.apache.iceberg.TableTestBase.FILE_A_DELETES;
+import static org.apache.iceberg.TableTestBase.FILE_B;
+import static org.apache.iceberg.TableTestBase.FILE_B_DELETES;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+public class TestPartitionTableScans {

Review comment:
       Any reason we can't extend TableTestBase, or even put this in "TestMetadataTableScans"?




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

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

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



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


[GitHub] [iceberg] szehon-ho commented on a change in pull request #4005: Core: Add delete file details for PartitionsTable

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on a change in pull request #4005:
URL: https://github.com/apache/iceberg/pull/4005#discussion_r808661327



##########
File path: core/src/main/java/org/apache/iceberg/PartitionsTable.java
##########
@@ -167,16 +175,30 @@ Partition get(StructLike key) {
     private final StructLike key;
     private long recordCount;
     private int fileCount;
+    private int deleteFileCount;
+    private long equalityDeleteCount;
+    private long positionDeleteCount;
 
     Partition(StructLike key) {
       this.key = key;
       this.recordCount = 0;
       this.fileCount = 0;
+      this.deleteFileCount = 0;
+      this.equalityDeleteCount = 0L;
+      this.positionDeleteCount = 0L;
     }
 
-    void update(DataFile file) {
-      this.recordCount += file.recordCount();
+    void update(FileScanTask task) {
+      this.recordCount += task.file().recordCount();
       this.fileCount += 1;
+      this.deleteFileCount += task.deletes().size();

Review comment:
       Would this ever have duplicates?  Example multiple fileScanTask within the same partition referring to the same delete file, would this value be too high?




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

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

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



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


[GitHub] [iceberg] rdblue commented on pull request #4005: Core: Add delete_file_count for PartitionsTable

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


   Thanks, @coolderli! This is going to be really helpful. Another thing that we should do is add a new metadata table that loads just delete files. Are you interested in working on that 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] coolderli commented on a change in pull request #4005: Core: Add delete_file_count for PartitionsTable

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



##########
File path: core/src/main/java/org/apache/iceberg/PartitionsTable.java
##########
@@ -167,16 +172,25 @@ Partition get(StructLike key) {
     private final StructLike key;
     private long recordCount;
     private int fileCount;
+    private int deleteFileCount;
 
     Partition(StructLike key) {
       this.key = key;
       this.recordCount = 0;
       this.fileCount = 0;
+      this.deleteFileCount = 0;
     }
 
     void update(DataFile file) {
       this.recordCount += file.recordCount();
       this.fileCount += 1;
     }
+
+    void updateDeleteFile(List<DeleteFile> deleteFiles) {
+      for (DeleteFile deleteFile : deleteFiles) {
+        this.recordCount += deleteFile.recordCount();

Review comment:
       > Thanks, @coolderli! This is going to be really helpful. Another thing that we should do is add a new metadata table that loads just delete files. Are you interested in working on that as well?
   
   Yes, I'll do that. I will fix this and push another pr for that 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] coolderli commented on pull request #4005: Core: Add delete_file_count for PartitionsTable

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


   > Thanks, @coolderli! This is going to be really helpful. Another thing that we should do is add a new metadata table that loads just delete files. Are you interested in working on that as well?
   
   Yes, I'll do that. I will fix this and push another pr for that 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] coolderli commented on a change in pull request #4005: Core: Add delete file details for PartitionsTable

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



##########
File path: core/src/main/java/org/apache/iceberg/PartitionsTable.java
##########
@@ -48,7 +48,10 @@
     this.schema = new Schema(
         Types.NestedField.required(1, "partition", table.spec().partitionType()),
         Types.NestedField.required(2, "record_count", Types.LongType.get()),
-        Types.NestedField.required(3, "file_count", Types.IntegerType.get())
+        Types.NestedField.required(3, "file_count", Types.IntegerType.get()),
+        Types.NestedField.required(4, "delete_file_count", Types.IntegerType.get()),
+        Types.NestedField.required(5, "equality_delete_count", Types.LongType.get()),

Review comment:
       I'd like to add file_size to the schema, do you think it is necessary? @rdblue 




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

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

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



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


[GitHub] [iceberg] coolderli commented on a change in pull request #4005: Core: Add delete_file_count for PartitionsTable

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



##########
File path: core/src/main/java/org/apache/iceberg/PartitionsTable.java
##########
@@ -167,16 +172,25 @@ Partition get(StructLike key) {
     private final StructLike key;
     private long recordCount;
     private int fileCount;
+    private int deleteFileCount;
 
     Partition(StructLike key) {
       this.key = key;
       this.recordCount = 0;
       this.fileCount = 0;
+      this.deleteFileCount = 0;
     }
 
     void update(DataFile file) {
       this.recordCount += file.recordCount();
       this.fileCount += 1;
     }
+
+    void updateDeleteFile(List<DeleteFile> deleteFiles) {

Review comment:
       Yes, if there is a metadata table that can load delete files, I think this is reasonable.




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

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

For queries about this service, please contact Infrastructure at:
users@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 change in pull request #4005: Core: Add delete_file_count for PartitionsTable

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



##########
File path: core/src/main/java/org/apache/iceberg/PartitionsTable.java
##########
@@ -167,16 +172,25 @@ Partition get(StructLike key) {
     private final StructLike key;
     private long recordCount;
     private int fileCount;
+    private int deleteFileCount;
 
     Partition(StructLike key) {
       this.key = key;
       this.recordCount = 0;
       this.fileCount = 0;

Review comment:
       If we're going to add this, likely we should also make this `dataFileCount`.
   
   But I would hold off on updating that part, as this is a somewhat larger change to the spec and we should ensure that it's going to be considered in its current form before making smaller updates like that (so you don't spend time unnecessarily).




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

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

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



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


[GitHub] [iceberg] coolderli commented on a change in pull request #4005: Core: Add delete file details for PartitionsTable

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



##########
File path: core/src/main/java/org/apache/iceberg/PartitionsTable.java
##########
@@ -167,16 +175,30 @@ Partition get(StructLike key) {
     private final StructLike key;
     private long recordCount;
     private int fileCount;
+    private int deleteFileCount;
+    private long equalityDeleteCount;
+    private long positionDeleteCount;
 
     Partition(StructLike key) {
       this.key = key;
       this.recordCount = 0;
       this.fileCount = 0;
+      this.deleteFileCount = 0;
+      this.equalityDeleteCount = 0L;
+      this.positionDeleteCount = 0L;
     }
 
-    void update(DataFile file) {
-      this.recordCount += file.recordCount();
+    void update(FileScanTask task) {
+      this.recordCount += task.file().recordCount();
       this.fileCount += 1;
+      this.deleteFileCount += task.deletes().size();

Review comment:
       Yes, I'll fix it. 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] rdblue commented on pull request #4005: Core: Add delete file details for PartitionsTable

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


   @coolderli did you want to update this after @szehon-ho's 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] rdblue commented on a change in pull request #4005: Core: Add delete_file_count for PartitionsTable

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



##########
File path: core/src/main/java/org/apache/iceberg/PartitionsTable.java
##########
@@ -167,16 +172,25 @@ Partition get(StructLike key) {
     private final StructLike key;
     private long recordCount;
     private int fileCount;
+    private int deleteFileCount;
 
     Partition(StructLike key) {
       this.key = key;
       this.recordCount = 0;
       this.fileCount = 0;
+      this.deleteFileCount = 0;
     }
 
     void update(DataFile file) {
       this.recordCount += file.recordCount();
       this.fileCount += 1;
     }
+
+    void updateDeleteFile(List<DeleteFile> deleteFiles) {
+      for (DeleteFile deleteFile : deleteFiles) {
+        this.recordCount += deleteFile.recordCount();

Review comment:
       This should not be the same record count. Data record counts are tracked separately from both equality delete count and position delete count. This should actually add 3 columns: delete_file_count, equality_delete_count, and position_delete_count.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #4005: Core: Add delete file details for PartitionsTable

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



##########
File path: core/src/main/java/org/apache/iceberg/PartitionsTable.java
##########
@@ -48,7 +48,10 @@
     this.schema = new Schema(
         Types.NestedField.required(1, "partition", table.spec().partitionType()),
         Types.NestedField.required(2, "record_count", Types.LongType.get()),
-        Types.NestedField.required(3, "file_count", Types.IntegerType.get())
+        Types.NestedField.required(3, "file_count", Types.IntegerType.get()),
+        Types.NestedField.required(4, "delete_file_count", Types.IntegerType.get()),
+        Types.NestedField.required(5, "equality_delete_count", Types.LongType.get()),

Review comment:
       I agree. It would be better to use `file` and `record` to clarify what each count means.
   
   It would also be nice to rename `file_count`, but that probably isn't a good idea because it would break existing queries. Instead, we should add field docs to 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] coolderli commented on a change in pull request #4005: Core: Add delete_file_count for PartitionsTable

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



##########
File path: core/src/main/java/org/apache/iceberg/PartitionsTable.java
##########
@@ -167,16 +172,25 @@ Partition get(StructLike key) {
     private final StructLike key;
     private long recordCount;
     private int fileCount;
+    private int deleteFileCount;
 
     Partition(StructLike key) {
       this.key = key;
       this.recordCount = 0;
       this.fileCount = 0;

Review comment:
       Thanks for your attention, but I think this change is necessary. In our company, users always query the partitions to check if the table is well compaction. When partial compaction is enabled, there may be some partition fails to compaction, but without this query, we can not know which partition is bad.




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

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

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



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


[GitHub] [iceberg] szehon-ho commented on a change in pull request #4005: Core: Add delete file details for PartitionsTable

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on a change in pull request #4005:
URL: https://github.com/apache/iceberg/pull/4005#discussion_r826462493



##########
File path: core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java
##########
@@ -516,6 +522,101 @@ public void testPartitionColumnNamedPartition() throws Exception {
     validateIncludesPartitionScan(tasksAndEq, 0);
   }
 
+  @Test
+  public void testPartitionsTableScanWithDeleteFilesInFilter() throws IOException {
+    Assume.assumeTrue(formatVersion == 2);
+    Configuration conf = new Configuration();
+    HadoopTables tables = new HadoopTables(conf);
+    File tableLocation = new File(temp.newFolder(), "table");
+    Table table = tables.create(

Review comment:
       On the same topic, I think we do not need to create a table again, as 'table' is already setup in this test.




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

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

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



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


[GitHub] [iceberg] szehon-ho commented on a change in pull request #4005: Core: Add delete file details for PartitionsTable

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on a change in pull request #4005:
URL: https://github.com/apache/iceberg/pull/4005#discussion_r826462290



##########
File path: core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java
##########
@@ -516,6 +522,101 @@ public void testPartitionColumnNamedPartition() throws Exception {
     validateIncludesPartitionScan(tasksAndEq, 0);
   }
 
+  @Test
+  public void testPartitionsTableScanWithDeleteFilesInFilter() throws IOException {
+    Assume.assumeTrue(formatVersion == 2);
+    Configuration conf = new Configuration();
+    HadoopTables tables = new HadoopTables(conf);

Review comment:
       Seems all the rest of the test in this class just use the constructor of  PartitionsTable (or whatever metadata table), is there any reason not to do it here?

##########
File path: core/src/main/java/org/apache/iceberg/PartitionsTable.java
##########
@@ -48,7 +48,10 @@
     this.schema = new Schema(
         Types.NestedField.required(1, "partition", table.spec().partitionType()),
         Types.NestedField.required(2, "record_count", Types.LongType.get()),
-        Types.NestedField.required(3, "file_count", Types.IntegerType.get())
+        Types.NestedField.required(3, "file_count", Types.IntegerType.get()),
+        Types.NestedField.required(4, "delete_file_count", Types.IntegerType.get()),
+        Types.NestedField.required(5, "equality_delete_count", Types.LongType.get()),

Review comment:
       The names still do not seem consistent to me ("delete_file" and "file" but "equality_delete" and "position_delete").  But yes good point this is following  SnapshotSummary, though I think this schema is already quite different than that.  Will leave to @rdblue or others to see what they think.

##########
File path: core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java
##########
@@ -516,6 +522,101 @@ public void testPartitionColumnNamedPartition() throws Exception {
     validateIncludesPartitionScan(tasksAndEq, 0);
   }
 
+  @Test
+  public void testPartitionsTableScanWithDeleteFilesInFilter() throws IOException {
+    Assume.assumeTrue(formatVersion == 2);
+    Configuration conf = new Configuration();
+    HadoopTables tables = new HadoopTables(conf);
+    File tableLocation = new File(temp.newFolder(), "table");
+    Table table = tables.create(

Review comment:
       On same line, I think we do not need to create a table again, as 'table' is already setup in this test.




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

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

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



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


[GitHub] [iceberg] coolderli commented on pull request #4005: Core: Add delete file details for PartitionsTable

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


   @rdblue I have updated, can you review it again? 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] coolderli commented on a change in pull request #4005: Core: Add delete file details for PartitionsTable

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



##########
File path: core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java
##########
@@ -516,6 +522,101 @@ public void testPartitionColumnNamedPartition() throws Exception {
     validateIncludesPartitionScan(tasksAndEq, 0);
   }
 
+  @Test
+  public void testPartitionsTableScanWithDeleteFilesInFilter() throws IOException {
+    Assume.assumeTrue(formatVersion == 2);
+    Configuration conf = new Configuration();
+    HadoopTables tables = new HadoopTables(conf);

Review comment:
       I have tried to use `new PartitionTable(table.ops(), table)`, but it return a NPE on `ops.current().metadataFileLocation()`. https://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/PartitionsTable.java#L84

##########
File path: core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java
##########
@@ -516,6 +522,101 @@ public void testPartitionColumnNamedPartition() throws Exception {
     validateIncludesPartitionScan(tasksAndEq, 0);
   }
 
+  @Test
+  public void testPartitionsTableScanWithDeleteFilesInFilter() throws IOException {
+    Assume.assumeTrue(formatVersion == 2);
+    Configuration conf = new Configuration();
+    HadoopTables tables = new HadoopTables(conf);
+    File tableLocation = new File(temp.newFolder(), "table");
+    Table table = tables.create(

Review comment:
       I use HadoopTable to load the PartitionTable, so I can read the records. 




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

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

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



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