W-Ely opened a new issue, #9618:
URL: https://github.com/apache/iceberg/issues/9618

   ### Apache Iceberg version
   
   1.4.2
   
   ### Query engine
   
   Spark
   
   ### Please describe the bug 🐞
   
   We are running on AWS EMR so the version is technically 1.4.2-amaz-0.
   
   We found that approximately 10 seconds after the S3 file(s) were written and 
the Glue metadata entry updated, the S3 files were deleted but the metadata 
location in Glue was not reset so the current pointed at the S3 file that was 
removed.
   
   We had to manually update the Glue entry correcting the current to point to 
the previous and update the previous to the correct previous.  
   
   Due to the current still pointing to the metadata that we deleted from S3, 
this stack trace was raised:
   ```
   Exception in thread 'main'; org.apache.iceberg.exceptions.NotFoundException: 
Location does not exist: 
s3://lakehouse--managed-us-west-2--prod/main/uel_raw/push_notification/metadata/01788-f6d8e6b6-644e-4cf9-8129-65e9e03aa20e.metadata.json
      at 
org.apache.iceberg.aws.s3.S3InputStream.openStream(S3InputStream.java:251)
      at 
org.apache.iceberg.aws.s3.S3InputStream.positionStream(S3InputStream.java:234)
      at 
org.apache.iceberg.aws.s3.S3InputStream.lambda$read$1(S3InputStream.java:127)
      at org.apache.iceberg.util.Tasks$Builder.runTaskWithRetry(Tasks.java:413)
      at org.apache.iceberg.util.Tasks$Builder.runSingleThreaded(Tasks.java:219)
      at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:203)
      at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:196)
      at 
org.apache.iceberg.aws.s3.S3InputStream.retryAndThrow(S3InputStream.java:266)
      at org.apache.iceberg.aws.s3.S3InputStream.read(S3InputStream.java:123)
      at 
org.apache.iceberg.shaded.com.fasterxml.jackson.core.json.ByteSourceJsonBootstrapper.ensureLoaded(ByteSourceJsonBootstrapper.java:539)
      at 
org.apache.iceberg.shaded.com.fasterxml.jackson.core.json.ByteSourceJsonBootstrapper.detectEncoding(ByteSourceJsonBootstrapper.java:133)
      at 
org.apache.iceberg.shaded.com.fasterxml.jackson.core.json.ByteSourceJsonBootstrapper.constructParser(ByteSourceJsonBootstrapper.java:256)
      at 
org.apache.iceberg.shaded.com.fasterxml.jackson.core.JsonFactory._createParser(JsonFactory.java:1744)
      at 
org.apache.iceberg.shaded.com.fasterxml.jackson.core.JsonFactory.createParser(JsonFactory.java:1143)
      at 
org.apache.iceberg.shaded.com.fasterxml.jackson.databind.ObjectMapper.readValue(ObjectMapper.java:3809)
      at 
org.apache.iceberg.TableMetadataParser.read(TableMetadataParser.java:273)
      at 
org.apache.iceberg.TableMetadataParser.read(TableMetadataParser.java:266)
      at 
org.apache.iceberg.BaseMetastoreTableOperations.lambda$refreshFromMetadataLocation$0(BaseMetastoreTableOperations.java:189)
      at 
org.apache.iceberg.BaseMetastoreTableOperations.lambda$refreshFromMetadataLocation$1(BaseMetastoreTableOperations.java:208)
      at org.apache.iceberg.util.Tasks$Builder.runTaskWithRetry(Tasks.java:413)
      at org.apache.iceberg.util.Tasks$Builder.runSingleThreaded(Tasks.java:219)
      at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:203)
      at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:196)
      at 
org.apache.iceberg.BaseMetastoreTableOperations.refreshFromMetadataLocation(BaseMetastoreTableOperations.java:208)
      at 
org.apache.iceberg.BaseMetastoreTableOperations.refreshFromMetadataLocation(BaseMetastoreTableOperations.java:185)
      at 
org.apache.iceberg.BaseMetastoreTableOperations.refreshFromMetadataLocation(BaseMetastoreTableOperations.java:176)
      at 
org.apache.iceberg.aws.glue.GlueTableOperations.doRefresh(GlueTableOperations.java:141)
      at 
org.apache.iceberg.BaseMetastoreTableOperations.refresh(BaseMetastoreTableOperations.java:97)
      at org.apache.iceberg.SnapshotProducer.refresh(SnapshotProducer.java:354)
      at org.apache.iceberg.SnapshotProducer.apply(SnapshotProducer.java:219)
      at 
org.apache.iceberg.SnapshotProducer.lambda$commit$2(SnapshotProducer.java:376)
      at org.apache.iceberg.util.Tasks$Builder.runTaskWithRetry(Tasks.java:413)
      at org.apache.iceberg.util.Tasks$Builder.runSingleThreaded(Tasks.java:219)
      at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:203)
      at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:196)
      at org.apache.iceberg.SnapshotProducer.commit(SnapshotProducer.java:374)
      at 
org.apache.iceberg.spark.source.SparkWrite.commitOperation(SparkWrite.java:233)
      at 
org.apache.iceberg.spark.source.SparkWrite.access$1300(SparkWrite.java:84)
      at 
org.apache.iceberg.spark.source.SparkWrite$BatchAppend.commit(SparkWrite.java:296)
      at 
org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec.writeWithV2(WriteToDataSourceV2Exec.scala:399)
      at 
org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec.writeWithV2$(WriteToDataSourceV2Exec.scala:359)
      at 
org.apache.spark.sql.execution.datasources.v2.AppendDataExec.writeWithV2(WriteToDataSourceV2Exec.scala:225)
      at 
org.apache.spark.sql.execution.datasources.v2.V2ExistingTableWriteExec.run(WriteToDataSourceV2Exec.scala:337)
      at 
org.apache.spark.sql.execution.datasources.v2.V2ExistingTableWriteExec.run$(WriteToDataSourceV2Exec.scala:336)
      at 
org.apache.spark.sql.execution.datasources.v2.AppendDataExec.run(WriteToDataSourceV2Exec.scala:225)
      at 
org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result$lzycompute(V2CommandExec.scala:43)
      at 
org.apache.spark.sql.execution.datasources.v2.V2CommandExec.result(V2CommandExec.scala:43)
      at 
org.apache.spark.sql.execution.datasources.v2.V2CommandExec.executeCollect(V2CommandExec.scala:49)
      at 
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.$anonfun$applyOrElse$1(QueryExecution.scala:113)
      at 
org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:108)
      at 
org.apache.spark.sql.execution.SQLExecution$.withTracker(SQLExecution.scala:255)
      at 
org.apache.spark.sql.execution.SQLExecution$.executeQuery$1(SQLExecution.scala:129)
      at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$9(SQLExecution.scala:165)
      at 
org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:108)
      at 
org.apache.spark.sql.execution.SQLExecution$.withTracker(SQLExecution.scala:255)
      at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$8(SQLExecution.scala:165)
      at 
org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:276)
      at 
org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:164)
      at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900)
      at 
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:70)
      at 
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:110)
      at 
org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$1.applyOrElse(QueryExecution.scala:101)
      at 
org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:503)
      at 
org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:76)
      at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:503)
      at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:33)
      at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267)
      at 
org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263)
      at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:33)
      at 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:33)
      at 
org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:479)
      at 
org.apache.spark.sql.execution.QueryExecution.eagerlyExecuteCommands(QueryExecution.scala:101)
      at 
org.apache.spark.sql.execution.QueryExecution.commandExecuted$lzycompute(QueryExecution.scala:88)
      at 
org.apache.spark.sql.execution.QueryExecution.commandExecuted(QueryExecution.scala:86)
      at org.apache.spark.sql.Dataset.<init>(Dataset.scala:222)
      at org.apache.spark.sql.Dataset$.$anonfun$ofRows$2(Dataset.scala:102)
      at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900)
      at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:99)
      at 
org.apache.spark.sql.SparkSession.$anonfun$sql$4(SparkSession.scala:691)
      at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900)
      at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:682)
      at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:713)
      at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:744)
      at 
com.remitly.lakehouse.raw.UELS3ToLakehouseWorker.lakehouseTableUpdate(UELS3ToLakehouseWorker.scala:184)
      at 
com.remitly.lakehouse.raw.UELS3ToLakehouseWorker.lakehouseWrite(UELS3ToLakehouseWorker.scala:178)
      at 
com.remitly.lakehouse.raw.UELS3ToLakehouseWorker.writeData(UELS3ToLakehouseWorker.scala:137)
      at 
com.remitly.lakehouse.raw.UELS3ToLakehouseWorker.$anonfun$processTable$1(UELS3ToLakehouseWorker.scala:78)
      at 
com.remitly.lakehouse.raw.UELS3ToLakehouseWorker.$anonfun$processTable$1$adapted(UELS3ToLakehouseWorker.scala:73)
      at scala.collection.immutable.List.foreach(List.scala:431)
      at 
com.remitly.lakehouse.raw.UELS3ToLakehouseWorker.processTable(UELS3ToLakehouseWorker.scala:73)
      at 
com.remitly.lakehouse.raw.UELS3ToLakehouseWorker.$anonfun$run$1(UELS3ToLakehouseWorker.scala:48)
      at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
      at scala.util.Try$.apply(Try.scala:213)
      at 
com.remitly.lakehouse.raw.UELS3ToLakehouseWorker.retry(UELS3ToLakehouseWorker.scala:241)
      at 
com.remitly.lakehouse.raw.UELS3ToLakehouseWorker.run(UELS3ToLakehouseWorker.scala:48)
      at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
      at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      at java.lang.Thread.run(Thread.java:750)
   Caused by: software.amazon.awssdk.services.s3.model.NoSuchKeyException: The 
specified key does not exist. (Service: S3, Status Code: 404, Request ID: 
XSE4WS8T9Z5B2T07, Extended Request ID: 
uEngnDPadZItGQIUd5jNzee1cAgESyUVIwmRfvFt2YgUHnE7UeGscO5fDjPrRm2m2aAu8u/vIbw=)
      at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handleErrorResponse(CombinedResponseHandler.java:125)
      at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handleResponse(CombinedResponseHandler.java:82)
      at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handle(CombinedResponseHandler.java:60)
      at 
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handle(CombinedResponseHandler.java:41)
      at 
software.amazon.awssdk.core.internal.http.pipeline.stages.HandleResponseStage.execute(HandleResponseStage.java:50)
      at 
software.amazon.awssdk.core.internal.http.pipeline.stages.HandleResponseStage.execute(HandleResponseStage.java:38)
      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:72)
      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:55)
      at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:39)
      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:50)
      at 
software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallMetricCollectionStage.execute(ApiCallMetricCollectionStage.java:32)
      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:198)
      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$0(BaseSyncClientHandler.java:68)
      at 
software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.measureApiCallSuccess(BaseSyncClientHandler.java:179)
      at 
software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.execute(BaseSyncClientHandler.java:62)
      at 
software.amazon.awssdk.core.client.handler.SdkSyncClientHandler.execute(SdkSyncClientHandler.java:52)
      at 
software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler.execute(AwsSyncClientHandler.java:63)
      at 
software.amazon.awssdk.services.s3.DefaultS3Client.getObject(DefaultS3Client.java:4533)
      at 
org.apache.iceberg.aws.s3.S3InputStream.openStream(S3InputStream.java:249)
      ... 97 more
      ```
   We suspect that when a Glue operation succeeds but the success isn't 
communicated something in this bit:
   
https://github.com/apache/iceberg/blob/9de693f1e7f46024f47cdc971d8603fd76d87705/core/src/main/java/org/apache/iceberg/SnapshotProducer.java#L383
   doesn't work quite right.  
   
   
   Similar but not the same issues 
https://github.com/apache/iceberg/issues/9411l , 
https://github.com/apache/iceberg/issues/8927
   


-- 
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-unsubscr...@iceberg.apache.org.apache.org

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


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

Reply via email to