kbendick commented on a change in pull request #4162:
URL: https://github.com/apache/iceberg/pull/4162#discussion_r810217951



##########
File path: 
core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFilesAction.java
##########
@@ -300,6 +301,11 @@ void doReplace(Iterable<DataFile> deletedDataFiles, 
Iterable<DataFile> addedData
   private boolean isPartialFileScan(CombinedScanTask task) {
     if (task.files().size() == 1) {
       FileScanTask fileScanTask = task.files().iterator().next();
+      if (fileScanTask.file().format() == FileFormat.PARQUET) {
+        // in parquet format, there is an initial offset of 4 bytes
+        return fileScanTask.file().fileSizeInBytes() !=
+                (fileScanTask.length() + 
fileScanTask.file().splitOffsets().get(0));

Review comment:
       Nit: If we know that it's just 4 bytes, can we perhaps instead add a 
constant field for `PARQUET_MAGIC_BYTES` (I believe this is usually referred to 
as Parquet magic bytes?).
   
   Using `splitOffsets.get(0)` is a bit more confusing for me, particularly if 
we expect it to be an exact 4 bytes every time.

##########
File path: 
core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFilesAction.java
##########
@@ -300,6 +301,11 @@ void doReplace(Iterable<DataFile> deletedDataFiles, 
Iterable<DataFile> addedData
   private boolean isPartialFileScan(CombinedScanTask task) {
     if (task.files().size() == 1) {
       FileScanTask fileScanTask = task.files().iterator().next();
+      if (fileScanTask.file().format() == FileFormat.PARQUET) {
+        // in parquet format, there is an initial offset of 4 bytes
+        return fileScanTask.file().fileSizeInBytes() !=
+                (fileScanTask.length() + 
fileScanTask.file().splitOffsets().get(0));
+      }
       return fileScanTask.file().fileSizeInBytes() != fileScanTask.length();

Review comment:
       Does `fileScanTask.length()` not include the parquet initial offset of 4 
bytes if the task is a complete (not-partial) file scan?




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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to