You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "edgarRd (via GitHub)" <gi...@apache.org> on 2023/05/17 20:41:17 UTC

[GitHub] [iceberg] edgarRd opened a new issue, #7635: Failure to delete with merge-on-read in branch and main branch

edgarRd opened a new issue, #7635:
URL: https://github.com/apache/iceberg/issues/7635

   ### Apache Iceberg version
   
   1.2.1 (latest release)
   
   ### Query engine
   
   Spark
   
   ### Please describe the bug 🐞
   
   # Environment
   
   Spark 3.3.2
   Iceberg 1.2.1
   
   Tabular's docker environment: https://github.com/tabular-io/docker-spark-iceberg/blob/main/docker-compose.yml
   
   # Repro
   
   ## 1. Setup 
   ```sql
   CREATE TABLE demo.nyc.contacts2
   (
     id bigint NOT NULL COMMENT 'unique id',
     first_name string,
     last_name string,
     neighborhood string
   )
   TBLPROPERTIES(
     'format-version'='2',
     'write.delete.mode'='merge-on-read',
     'write.update.mode'='merge-on-read',
     'write.merge.mode'='merge-on-read'
   );
   
   ALTER TABLE demo.nyc.contacts2 SET IDENTIFIER FIELDS id;
   
   INSERT INTO demo.nyc.contacts2
   SELECT /*+ COALESCE(1) */ *
   FROM VALUES (1, 'Adam', 'Smith', 'SoHo'), 
   (2, 'Virginia', 'Smith', 'SoHo'), 
   (3, 'Thomas', 'Lao', 'Midtown'),
   (4, 'John', 'Books', 'Williamsburg'),
   (5, 'Anna', 'Frank', 'Midtown');
   ```
   
   Validate setup, expectations:
   * 1 data file
   * 5 rows
   ```
   SELECT * FROM demo.nyc.contacts2;
   SELECT * FROM demo.nyc.contacts2.files;
   ```
   
   ## 2. Setup branch
   
   ```sql
   ALTER TABLE demo.nyc.contacts2 CREATE BRANCH ds20230501 RETAIN 730 DAYS;
   
   INSERT INTO  demo.nyc.contacts2.branch_ds20230501
   SELECT /*+ COALESCE(1) */ *
   FROM VALUES (6, 'Peter', 'Smith', 'Chelsea'), (7, 'John', 'Connor', 'Greenwich Village');
   ```
   
   Validate branch setup, expectations:
   * 7 rows
   * 2 total data files (1 in main branch, 1 in branch `ds20230501`)
   
   ```sql
   SELECT * FROM demo.nyc.contacts2.branch_ds20230501;
   SELECT * FROM demo.nyc.contacts2.all_files;
   ```
   
   ## 3. Test merge-on-read delete on branch for 1 row within data file
   ```sql
   DELETE FROM demo.nyc.contacts2.branch_ds20230501 WHERE id=7;
   ```
   
   Previous command fails with:
   ```
   spark-sql> DELETE FROM demo.nyc.contacts2.branch_ds20230501 WHERE id=7;
   23/05/17 20:04:45 ERROR SparkSQLDriver: Failed in [DELETE FROM demo.nyc.contacts2.branch_ds20230501 WHERE id=7]
   org.apache.iceberg.exceptions.ValidationException: Cannot delete file where some, but not all, rows match filter ref(name="id") == 7: s3://warehouse/nyc/contacts2/data/00000-1051-74b41dde-49d4-4a85-844c-0ef79e1257f6-00001.parquet
   	at org.apache.iceberg.exceptions.ValidationException.check(ValidationException.java:49)
   	at org.apache.iceberg.ManifestFilterManager.manifestHasDeletedFiles(ManifestFilterManager.java:377)
   	at org.apache.iceberg.ManifestFilterManager.filterManifest(ManifestFilterManager.java:307)
   	at org.apache.iceberg.ManifestFilterManager.lambda$filterManifests$0(ManifestFilterManager.java:189)
   	at org.apache.iceberg.util.Tasks$Builder.runTaskWithRetry(Tasks.java:413)
   	at org.apache.iceberg.util.Tasks$Builder.access$300(Tasks.java:69)
   	at org.apache.iceberg.util.Tasks$Builder$1.run(Tasks.java:315)
   	at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
   	at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
   	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
   	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
   	at java.base/java.lang.Thread.run(Thread.java:829)
   org.apache.iceberg.exceptions.ValidationException: Cannot delete file where some, but not all, rows match filter ref(name="id") == 7: s3://warehouse/nyc/contacts2/data/00000-1051-74b41dde-49d4-4a85-844c-0ef79e1257f6-00001.parquet
   	at org.apache.iceberg.exceptions.ValidationException.check(ValidationException.java:49)
   	at org.apache.iceberg.ManifestFilterManager.manifestHasDeletedFiles(ManifestFilterManager.java:377)
   	at org.apache.iceberg.ManifestFilterManager.filterManifest(ManifestFilterManager.java:307)
   	at org.apache.iceberg.ManifestFilterManager.lambda$filterManifests$0(ManifestFilterManager.java:189)
   	at org.apache.iceberg.util.Tasks$Builder.runTaskWithRetry(Tasks.java:413)
   	at org.apache.iceberg.util.Tasks$Builder.access$300(Tasks.java:69)
   	at org.apache.iceberg.util.Tasks$Builder$1.run(Tasks.java:315)
   	at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
   	at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
   	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
   	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
   	at java.base/java.lang.Thread.run(Thread.java:829)
   ```
   
   ## 4. Delete from main branch also fails
   
   This error is different, but fails consistently with `400` (bad request) so I wonder if there's some incorrect handling here. Possibly related to the environment as well as using Tabular's docker setup.
   
   ```sql
   DELETE FROM demo.nyc.contacts2 WHERE id=3;
   ```
   
   ```
   Caused by: software.amazon.awssdk.services.s3.model.S3Exception: null (Service: S3, Status Code: 400, Request ID: 17600716A659BCC7, Extended Request ID: e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855)
   	at software.amazon.awssdk.protocols.xml.internal.unmarshall.AwsXmlPredicatedResponseHandler.handleErrorResponse(AwsXmlPredicatedResponseHandler.java:156)
   	at software.amazon.awssdk.protocols.xml.internal.unmarshall.AwsXmlPredicatedResponseHandler.handleResponse(AwsXmlPredicatedResponseHandler.java:108)
   	at software.amazon.awssdk.protocols.xml.internal.unmarshall.AwsXmlPredicatedResponseHandler.handle(AwsXmlPredicatedResponseHandler.java:85)
   	at software.amazon.awssdk.protocols.xml.internal.unmarshall.AwsXmlPredicatedResponseHandler.handle(AwsXmlPredicatedResponseHandler.java:43)
   	at software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler$Crc32ValidationResponseHandler.handle(AwsSyncClientHandler.java:95)
   	at software.amazon.awssdk.core.internal.handler.BaseClientHandler.lambda$successTransformationResponseHandler$7(BaseClientHandler.java:270)
   	at software.amazon.awssdk.core.internal.http.pipeline.stages.HandleResponseStage.execute(HandleResponseStage.java:40)
   	at software.amazon.awssdk.core.internal.http.pipeline.stages.HandleResponseStage.execute(HandleResponseStage.java:30)
   	at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
   	at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:73)
   	at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:42)
   	at software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:78)
   	at software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:40)
   	at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:50)
   	at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:36)
   	at software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:81)
   	at software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:36)
   	at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
   	at software.amazon.awssdk.core.internal.http.StreamManagingStage.execute(StreamManagingStage.java:56)
   	at software.amazon.awssdk.core.internal.http.StreamManagingStage.execute(StreamManagingStage.java:36)
   	at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.executeWithTimer(ApiCallTimeoutTrackingStage.java:80)
   	at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.execute(ApiCallTimeoutTrackingStage.java:60)
   	at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.execute(ApiCallTimeoutTrackingStage.java:42)
   	at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallMetricCollectionStage.execute(ApiCallMetricCollectionStage.java:48)
   	at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallMetricCollectionStage.execute(ApiCallMetricCollectionStage.java:31)
   	at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
   	at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206)
   	at software.amazon.awssdk.core.internal.http.pipeline.stages.ExecutionFailureExceptionReportingStage.execute(ExecutionFailureExceptionReportingStage.java:37)
   	at software.amazon.awssdk.core.internal.http.pipeline.stages.ExecutionFailureExceptionReportingStage.execute(ExecutionFailureExceptionReportingStage.java:26)
   	at software.amazon.awssdk.core.internal.http.AmazonSyncHttpClient$RequestExecutionBuilderImpl.execute(AmazonSyncHttpClient.java:193)
   	at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.invoke(BaseSyncClientHandler.java:103)
   	at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.doExecute(BaseSyncClientHandler.java:171)
   	at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.lambda$execute$1(BaseSyncClientHandler.java:82)
   	at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.measureApiCallSuccess(BaseSyncClientHandler.java:179)
   	at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.execute(BaseSyncClientHandler.java:76)
   	at software.amazon.awssdk.core.client.handler.SdkSyncClientHandler.execute(SdkSyncClientHandler.java:45)
   	at software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler.execute(AwsSyncClientHandler.java:56)
   	at software.amazon.awssdk.services.s3.DefaultS3Client.putObject(DefaultS3Client.java:9321)
   	at org.apache.iceberg.aws.s3.S3OutputStream.completeUploads(S3OutputStream.java:435)
   	at org.apache.iceberg.aws.s3.S3OutputStream.close(S3OutputStream.java:269)
   	at org.apache.iceberg.shaded.org.apache.parquet.io.DelegatingPositionOutputStream.close(DelegatingPositionOutputStream.java:38)
   	at org.apache.iceberg.shaded.org.apache.parquet.hadoop.ParquetFileWriter.end(ParquetFileWriter.java:1197)
   	at org.apache.iceberg.parquet.ParquetWriter.close(ParquetWriter.java:255)
   	at org.apache.iceberg.deletes.PositionDeleteWriter.close(PositionDeleteWriter.java:75)
   	at org.apache.iceberg.io.RollingFileWriter.closeCurrentWriter(RollingFileWriter.java:122)
   	at org.apache.iceberg.io.RollingFileWriter.close(RollingFileWriter.java:147)
   	at org.apache.iceberg.io.RollingPositionDeleteWriter.close(RollingPositionDeleteWriter.java:35)
   	at org.apache.iceberg.io.ClusteredWriter.closeCurrentWriter(ClusteredWriter.java:118)
   	at org.apache.iceberg.io.ClusteredWriter.close(ClusteredWriter.java:110)
   	at org.apache.iceberg.io.ClusteredPositionDeleteWriter.close(ClusteredPositionDeleteWriter.java:34)
   	at org.apache.iceberg.spark.source.SparkPositionDeltaWrite$DeleteOnlyDeltaWriter.close(SparkPositionDeltaWrite.java:477)
   	at org.apache.iceberg.spark.source.SparkPositionDeltaWrite$DeleteOnlyDeltaWriter.commit(SparkPositionDeltaWrite.java:460)
   	at org.apache.spark.sql.execution.datasources.v2.WritingSparkTask.$anonfun$run$1(WriteDeltaExec.scala:176)
   	at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1538)
   	at org.apache.spark.sql.execution.datasources.v2.WritingSparkTask.run(WriteDeltaExec.scala:203)
   	at org.apache.spark.sql.execution.datasources.v2.WritingSparkTask.run$(WriteDeltaExec.scala:142)
   	at org.apache.spark.sql.execution.datasources.v2.DeltaWithMetadataWritingSparkTask.run(WriteDeltaExec.scala:208)
   	at org.apache.spark.sql.execution.datasources.v2.ExtendedV2ExistingTableWriteExec.$anonfun$writeWithV2$2(WriteDeltaExec.scala:101)
   	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
   	at org.apache.spark.scheduler.Task.run(Task.scala:136)
   	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:548)
   	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1504)
   	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:551)
   	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
   	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
   	at java.base/java.lang.Thread.run(Thread.java:829)
   ```


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

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

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


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


[GitHub] [iceberg] edgarRd commented on issue #7635: Failure to delete with merge-on-read in branch and main branch

Posted by "edgarRd (via GitHub)" <gi...@apache.org>.
edgarRd commented on issue #7635:
URL: https://github.com/apache/iceberg/issues/7635#issuecomment-1629678809

   Hi @amogh-jahagirdar - any updates on this? let me know if there's anything I can do to 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


Re: [I] Failure to delete with merge-on-read in branch and main branch [iceberg]

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on issue #7635:
URL: https://github.com/apache/iceberg/issues/7635#issuecomment-1815363187

   OK after some debugging and writing some local tests, what I'm seeing is the following:
   
   1.) In certain cases, when there's branches the DELETE in Spark gets executed via the `DeleteFromTableExec` execution path. This goes through Iceberg's `DeleteFiles` API and expectedly fails the validation (since we cannot delete a file where some records may not match the condition).
   
   2.) In other cases (deleting from main table state or even some cases in branches)  goes through `ReplaceDataExec` which will go through Iceberg's `Overwrite` API which is writing out entirely new files and thus succeeds.
   
   I'm working on determining on codifying what exactly the difference is that leads to this different physical execution. 
   
   It seems like a possible workaround in the interim to unblock deletions on branches could be to go through `MERGE INTO` path and self join on the deletion criteria because that seems to always go through the `ReplaceDataExec` path, but still need to validate 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


Re: [I] Failure to delete with merge-on-read in branch and main branch [iceberg]

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on issue #7635:
URL: https://github.com/apache/iceberg/issues/7635#issuecomment-1813498341

   Also, I should note that I can repro this only when there's the `COALESCE` query hint and it gets applied. When there's no `COALESCE` and there's separate files the delete works as expected.


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

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

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


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


Re: [I] Failure to delete with merge-on-read in branch and main branch [iceberg]

Posted by "edgarRd (via GitHub)" <gi...@apache.org>.
edgarRd commented on issue #7635:
URL: https://github.com/apache/iceberg/issues/7635#issuecomment-1769711711

   Ping @amogh-jahagirdar - Is there any progress on this? 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


Re: [I] Failure to delete with merge-on-read in branch and main branch [iceberg]

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue closed issue #7635: Failure to delete with merge-on-read in branch and main branch
URL: https://github.com/apache/iceberg/issues/7635


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

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

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


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


Re: [I] Failure to delete with merge-on-read in branch and main branch [iceberg]

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on issue #7635:
URL: https://github.com/apache/iceberg/issues/7635#issuecomment-1813387970

   Sorry for the delay on this issue. I'm looking at it right now. Thanks for the clear reproduction steps, I can repro this and I also see this same failure happens for copy on write. I'm going to step through the debugger and see what's going on.


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

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

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


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


[GitHub] [iceberg] amogh-jahagirdar commented on issue #7635: Failure to delete with merge-on-read in branch and main branch

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on issue #7635:
URL: https://github.com/apache/iceberg/issues/7635#issuecomment-1555121127

   Will take a look when I get some time, thanks for reporting this issue @edgarRd !


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

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

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


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


Re: [I] Failure to delete with merge-on-read in branch and main branch [iceberg]

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on issue #7635:
URL: https://github.com/apache/iceberg/issues/7635#issuecomment-1817116426

   @amogh-jahagirdar, that looks right to me.


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

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

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


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


[GitHub] [iceberg] Fokko commented on issue #7635: Failure to delete with merge-on-read in branch and main branch

Posted by "Fokko (via GitHub)" <gi...@apache.org>.
Fokko commented on issue #7635:
URL: https://github.com/apache/iceberg/issues/7635#issuecomment-1561750791

   Step 4 will be fixed by https://github.com/apache/iceberg/issues/7635


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

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

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


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


Re: [I] Failure to delete with merge-on-read in branch and main branch [iceberg]

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on issue #7635:
URL: https://github.com/apache/iceberg/issues/7635#issuecomment-1816926262

   Ok I think got the root cause, the problem is in `canDeleteUsingMetadata` https://github.com/apache/iceberg/blob/main/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java#L333 is that this resets everything on the scan context (the filter condition, snapshot, etc). So this ends up with the actual scan tasks being empty and Iterables.all surfacing true to Spark. 
   
   This has to be fixed to `scan = scan.useRef` to actually use the tag or just include the ref inline at the time of building. I'll put up a 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


Re: [I] Failure to delete with merge-on-read in branch and main branch [iceberg]

Posted by "edgarRd (via GitHub)" <gi...@apache.org>.
edgarRd commented on issue #7635:
URL: https://github.com/apache/iceberg/issues/7635#issuecomment-1930771143

   Thanks for fixing this issue, @amogh-jahagirdar!


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

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

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


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