trinodb / trino

Official repository of Trino, the distributed SQL query engine for big data, formerly known as PrestoSQL (https://trino.io)
https://trino.io
Apache License 2.0
10.34k stars 2.97k forks source link

Retrying filesystem operations doesn't make sense for terminal exceptions (like bucket not exists) #22678

Closed oskar-szwajkowski closed 2 months ago

oskar-szwajkowski commented 3 months ago

The issue I get is specific to combination of:

but maybe fix for it could be extended to cover more table types / exception types.

What I can see is that query can take 15/20min+, without having a chance to complete successfully ever, due to underlying terminal exception.

Here is trace report from Datadog, where each red rectangle shows retry with some backoff: image

Root cause is:

software.amazon.awssdk.services.s3.model.NoSuchBucketException: The specified bucket does not exist

retry code in this case comes from AbstractIcebergTableOperations, and maybe even somewhere higher too, as it retries 20 times, but it does it multiple times as well (probably there is outer and inner retry loop, as you can see on errors count on screenshot, its way more than 20)

I think outer retry loop comes from here: https://github.com/trinodb/trino/blob/98cd6509d04ba0b427cb0c1aaa30d91ddc9a7bbd/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/AbstractTrinoCatalog.java#L198

And inner ones comes from here: https://github.com/trinodb/trino/blob/6b447560c77cd5d3ef3ea1381e30483093b66052/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/AbstractIcebergTableOperations.java#L261

Which leads to 200 retries in general (10x20), even tho bucket does not exist (and maybe its similar for other types of errors like forbidden etc?)

Reproducing should be possible with such steps:

On top of that I have PR open that addressed this specific issue, by calling isRetryable on AWS exception, and based on that throwing special kind of exception that interrupts retries. I didn't have a chance yet to merge this PR, but it might be helpful.

https://github.com/trinodb/trino/pull/19307

oskar-szwajkowski commented 3 months ago

Same issue is with this exception:

With this query is being retried for 30mins for no reason at all, it should've been terminated after first exception

image
software.amazon.awssdk.services.s3.model.S3Exception: Access Denied (Service: S3, Status Code: 403, Request ID:, Extended Request ID:)
    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.RetryableStage2.executeRequest(RetryableStage2.java:93)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage2.execute(RetryableStage2.java:56)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage2.execute(RetryableStage2.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:224)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.invoke(BaseSyncClientHandler.java:103)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.doExecute(BaseSyncClientHandler.java:173)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.lambda$execute$0(BaseSyncClientHandler.java:66)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.measureApiCallSuccess(BaseSyncClientHandler.java:182)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.execute(BaseSyncClientHandler.java:60)
    at software.amazon.awssdk.core.client.handler.SdkSyncClientHandler.execute(SdkSyncClientHandler.java:52)
    at software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler.execute(AwsSyncClientHandler.java:60)
    at software.amazon.awssdk.services.s3.DefaultS3Client.getObject(DefaultS3Client.java:5205)
    at software.amazon.awssdk.services.s3.S3Client.getObject(S3Client.java:9063)
    at io.trino.filesystem.s3.S3InputStream.readRange(S3InputStream.java:228)
    at io.trino.filesystem.s3.S3InputStream.seekStream(S3InputStream.java:197)
    at io.trino.filesystem.s3.S3InputStream.read(S3InputStream.java:113)
    at io.trino.plugin.iceberg.fileio.ForwardingSeekableInputStream.read(ForwardingSeekableInputStream.java:66)
    at com.fasterxml.jackson.core.json.ByteSourceJsonBootstrapper.ensureLoaded(ByteSourceJsonBootstrapper.java:547)
    at com.fasterxml.jackson.core.json.ByteSourceJsonBootstrapper.detectEncoding(ByteSourceJsonBootstrapper.java:137)
    at com.fasterxml.jackson.core.json.ByteSourceJsonBootstrapper.constructParser(ByteSourceJsonBootstrapper.java:266)
    at com.fasterxml.jackson.core.JsonFactory._createParser(JsonFactory.java:1874)
    at com.fasterxml.jackson.core.JsonFactory.createParser(JsonFactory.java:1273)
    at com.fasterxml.jackson.databind.ObjectMapper.readValue(ObjectMapper.java:3885)
    at org.apache.iceberg.TableMetadataParser.read(TableMetadataParser.java:280)
    at io.trino.plugin.iceberg.catalog.AbstractIcebergTableOperations.lambda$refreshFromMetadataLocation$1(AbstractIcebergTableOperations.java:240)
    at io.trino.plugin.iceberg.catalog.AbstractIcebergTableOperations.lambda$refreshFromMetadataLocation$3(AbstractIcebergTableOperations.java:265)
    at dev.failsafe.Functions.lambda$toCtxSupplier$11(Functions.java:243)
    at dev.failsafe.Functions.lambda$get$0(Functions.java:46)
    at dev.failsafe.internal.RetryPolicyExecutor.lambda$apply$0(RetryPolicyExecutor.java:74)
    at dev.failsafe.SyncExecutionImpl.executeSync(SyncExecutionImpl.java:187)
    at dev.failsafe.FailsafeExecutor.call(FailsafeExecutor.java:376)
    at dev.failsafe.FailsafeExecutor.get(FailsafeExecutor.java:112)
    at io.trino.plugin.iceberg.catalog.AbstractIcebergTableOperations.refreshFromMetadataLocation(AbstractIcebergTableOperations.java:265)
    at io.trino.plugin.iceberg.catalog.AbstractIcebergTableOperations.refreshFromMetadataLocation(AbstractIcebergTableOperations.java:238)
    at io.trino.plugin.iceberg.catalog.AbstractIcebergTableOperations.refresh(AbstractIcebergTableOperations.java:139)
    at io.trino.plugin.iceberg.catalog.AbstractIcebergTableOperations.current(AbstractIcebergTableOperations.java:122)
    at io.trino.plugin.iceberg.catalog.glue.TrinoGlueCatalog.lambda$loadTable$18(TrinoGlueCatalog.java:595)
    at com.google.common.cache.LocalCache$LocalManualCache$1.load(LocalCache.java:4938)
    at com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3576)
    at com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2318)
    at com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2191)
    at com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2081)
    at com.google.common.cache.LocalCache.get(LocalCache.java:4019)
    at com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4933)
    at io.trino.cache.EvictableCache.get(EvictableCache.java:118)
    at io.trino.cache.CacheUtils.uncheckedCacheGet(CacheUtils.java:39)
    at io.trino.plugin.iceberg.catalog.glue.TrinoGlueCatalog.loadTable(TrinoGlueCatalog.java:584)
    at io.trino.plugin.iceberg.IcebergMetadata.getTableHandle(IcebergMetadata.java:453)
    at io.trino.plugin.objectstore.ObjectStoreMetadata.getTableHandleInOrder(ObjectStoreMetadata.java:236)
    at io.trino.plugin.objectstore.ObjectStoreMetadata.getTableHandle(ObjectStoreMetadata.java:264)
    at io.trino.plugin.base.classloader.ClassLoaderSafeConnectorMetadata.getTableHandle(ClassLoaderSafeConnectorMetadata.java:1280)
    at io.trino.tracing.TracingConnectorMetadata.getTableHandle(TracingConnectorMetadata.java:140)
    at io.trino.metadata.MetadataManager.lambda$getTableHandle$5(MetadataManager.java:287)
    at java.base/java.util.Optional.flatMap(Optional.java:289)
    at io.trino.metadata.MetadataManager.getTableHandle(MetadataManager.java:278)
    at io.trino.metadata.MetadataManager.getRedirectionAwareTableHandle(MetadataManager.java:1948)
    at io.trino.metadata.MetadataManager.getRedirectionAwareTableHandle(MetadataManager.java:1940)
    at io.trino.metadata.MetadataManager.lambda$listTableColumns$16(MetadataManager.java:636)
    at java.base/java.util.Optional.or(Optional.java:313)
    at io.trino.metadata.MetadataManager.listTableColumns(MetadataManager.java:634)
    at io.trino.tracing.TracingMetadata.listTableColumns(TracingMetadata.java:355)
    at io.trino.metadata.MetadataListing.doListTableColumns(MetadataListing.java:261)
    at io.trino.metadata.MetadataListing.listTableColumns(MetadataListing.java:251)
    at io.trino.connector.informationschema.InformationSchemaPageSource.addColumnsRecords(InformationSchemaPageSource.java:250)
    at io.trino.connector.informationschema.InformationSchemaPageSource.buildPages(InformationSchemaPageSource.java:218)
    at io.trino.connector.informationschema.InformationSchemaPageSource.getNextPage(InformationSchemaPageSource.java:185)
    at io.trino.operator.ScanFilterAndProjectOperator$ConnectorPageSourceToPages.process(ScanFilterAndProjectOperator.java:385)
    at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:412)
    at io.trino.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:261)
    at io.trino.operator.WorkProcessorUtils$YieldingProcess.process(WorkProcessorUtils.java:181)
    at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:412)
    at io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:346)
    at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:412)
    at io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:346)
    at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:412)
    at io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:346)
    at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:412)
    at io.trino.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:261)
    at io.trino.operator.WorkProcessorUtils$BlockingProcess.process(WorkProcessorUtils.java:207)
    at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:412)
    at io.trino.operator.WorkProcessorUtils.lambda$flatten$6(WorkProcessorUtils.java:317)
    at io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:359)
    at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:412)
    at io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:346)
    at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:412)
    at io.trino.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:261)
    at io.trino.operator.WorkProcessorUtils.lambda$processStateMonitor$2(WorkProcessorUtils.java:240)
    at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:412)
    at io.trino.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:261)
    at io.trino.operator.WorkProcessorUtils.lambda$finishWhen$3(WorkProcessorUtils.java:255)
    at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:412)
    at io.trino.operator.WorkProcessorSourceOperatorAdapter.getOutput(WorkProcessorSourceOperatorAdapter.java:133)
    at io.trino.operator.Driver.processInternal(Driver.java:400)
    at io.trino.operator.Driver.lambda$process$8(Driver.java:303)
    at io.trino.operator.Driver.tryWithLock(Driver.java:706)
    at io.trino.operator.Driver.process(Driver.java:295)
    at io.trino.operator.Driver.processForDuration(Driver.java:266)
    at io.trino.execution.SqlTaskExecution$DriverSplitRunner.processFor(SqlTaskExecution.java:890)
    at io.trino.execution.executor.timesharing.PrioritizedSplitRunner.process(PrioritizedSplitRunner.java:187)
    at io.trino.execution.executor.timesharing.TimeSharingTaskExecutor$TaskRunner.run(TimeSharingTaskExecutor.java:565)
oskar-szwajkowski commented 2 months ago

Here is other example, this time from delta lake connector, it also gets retried several time, but it cannot complete due to 404 being non retryable exception

software.amazon.awssdk.services.s3.model.NoSuchKeyException: The specified key does not exist. (Service: S3, Status Code: 404, Request ID: xx, Extended Request ID: xx)
    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.RetryableStage2.executeRequest(RetryableStage2.java:93)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage2.execute(RetryableStage2.java:56)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage2.execute(RetryableStage2.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:224)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.invoke(BaseSyncClientHandler.java:103)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.doExecute(BaseSyncClientHandler.java:173)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.lambda$execute$0(BaseSyncClientHandler.java:66)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.measureApiCallSuccess(BaseSyncClientHandler.java:182)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.execute(BaseSyncClientHandler.java:60)
    at software.amazon.awssdk.core.client.handler.SdkSyncClientHandler.execute(SdkSyncClientHandler.java:52)
    at software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler.execute(AwsSyncClientHandler.java:60)
    at software.amazon.awssdk.services.s3.DefaultS3Client.getObject(DefaultS3Client.java:5210)
    at software.amazon.awssdk.services.s3.S3Client.getObject(S3Client.java:9063)
    at io.trino.filesystem.s3.S3InputStream.readRange(S3InputStream.java:228)
    at io.trino.filesystem.s3.S3InputStream.seekStream(S3InputStream.java:197)
    at io.trino.filesystem.s3.S3InputStream.read(S3InputStream.java:113)
    at java.base/sun.nio.cs.StreamDecoder.readBytes(StreamDecoder.java:350)
    at java.base/sun.nio.cs.StreamDecoder.implRead(StreamDecoder.java:393)
    at java.base/sun.nio.cs.StreamDecoder.lockedRead(StreamDecoder.java:217)
    at java.base/sun.nio.cs.StreamDecoder.read(StreamDecoder.java:171)
    at java.base/java.io.InputStreamReader.read(InputStreamReader.java:186)
    at java.base/java.io.BufferedReader.fill(BufferedReader.java:160)
    at java.base/java.io.BufferedReader.implReadLine(BufferedReader.java:370)
    at java.base/java.io.BufferedReader.readLine(BufferedReader.java:347)
    at java.base/java.io.BufferedReader.readLine(BufferedReader.java:436)
    at io.trino.plugin.deltalake.transactionlog.checkpoint.TransactionLogTail.getEntriesFromJson(TransactionLogTail.java:129)
    at io.trino.plugin.deltalake.transactionlog.checkpoint.TransactionLogTail.getEntriesFromJson(TransactionLogTail.java:119)
    at io.trino.plugin.deltalake.transactionlog.checkpoint.TransactionLogTail.loadNewTail(TransactionLogTail.java:77)
    at io.trino.plugin.deltalake.transactionlog.TableSnapshot.load(TableSnapshot.java:101)
    at io.trino.plugin.deltalake.transactionlog.TransactionLogAccess.lambda$loadSnapshot$1(TransactionLogAccess.java:179)
    at com.google.common.cache.LocalCache$LocalManualCache$1.load(LocalCache.java:4938)
    at com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3576)
    at com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2318)
    at com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2191)
    at com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2081)
    at com.google.common.cache.LocalCache.get(LocalCache.java:4019)
    at com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4933)
    at io.trino.cache.EvictableCache.get(EvictableCache.java:118)
    at io.trino.plugin.deltalake.transactionlog.TransactionLogAccess.loadSnapshot(TransactionLogAccess.java:178)
    at io.trino.plugin.deltalake.DeltaLakeMetadata.getSnapshot(DeltaLakeMetadata.java:494)
    at io.trino.plugin.deltalake.DeltaLakeMetadata.getTableHandle(DeltaLakeMetadata.java:603)
    at io.trino.plugin.deltalake.DeltaLakeMetadata.getTableHandle(DeltaLakeMetadata.java:331)
    at io.trino.plugin.objectstore.ObjectStoreMetadata.getTableHandleInOrder(ObjectStoreMetadata.java:237)
    at io.trino.plugin.objectstore.ObjectStoreMetadata.getTableHandle(ObjectStoreMetadata.java:265)
    at io.trino.plugin.base.classloader.ClassLoaderSafeConnectorMetadata.getTableHandle(ClassLoaderSafeConnectorMetadata.java:1289)
    at io.trino.tracing.TracingConnectorMetadata.getTableHandle(TracingConnectorMetadata.java:141)
    at io.trino.metadata.MetadataManager.lambda$getTableHandle$5(MetadataManager.java:288)
    at java.base/java.util.Optional.flatMap(Optional.java:289)
    at io.trino.metadata.MetadataManager.getTableHandle(MetadataManager.java:279)
    at io.trino.metadata.MetadataManager.getRedirectionAwareTableHandle(MetadataManager.java:1948)
    at io.trino.metadata.MetadataManager.getRedirectionAwareTableHandle(MetadataManager.java:1940)
    at io.trino.metadata.MetadataManager.lambda$listTableColumns$16(MetadataManager.java:637)
    at java.base/java.util.Optional.or(Optional.java:313)
    at io.trino.metadata.MetadataManager.listTableColumns(MetadataManager.java:635)
    at io.trino.tracing.TracingMetadata.listTableColumns(TracingMetadata.java:356)
    at io.trino.metadata.MetadataListing.doListTableColumns(MetadataListing.java:261)
    at io.trino.metadata.MetadataListing.listTableColumns(MetadataListing.java:251)
    at io.trino.connector.informationschema.InformationSchemaPageSource.addColumnsRecords(InformationSchemaPageSource.java:250)
    at io.trino.connector.informationschema.InformationSchemaPageSource.buildPages(InformationSchemaPageSource.java:218)
    at io.trino.connector.informationschema.InformationSchemaPageSource.getNextPage(InformationSchemaPageSource.java:185)
    at io.trino.operator.ScanFilterAndProjectOperator$ConnectorPageSourceToPages.process(ScanFilterAndProjectOperator.java:385)
    at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:412)
    at io.trino.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:261)
    at io.trino.operator.WorkProcessorUtils$YieldingProcess.process(WorkProcessorUtils.java:181)
    at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:412)
    at io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:346)
    at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:412)
    at io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:346)
    at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:412)
    at io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:346)
    at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:412)
    at io.trino.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:261)
    at io.trino.operator.WorkProcessorUtils$BlockingProcess.process(WorkProcessorUtils.java:207)
    at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:412)
    at io.trino.operator.WorkProcessorUtils.lambda$flatten$6(WorkProcessorUtils.java:317)
    at io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:359)
    at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:412)
    at io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:346)
    at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:412)
    at io.trino.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:261)
    at io.trino.operator.WorkProcessorUtils.lambda$processStateMonitor$2(WorkProcessorUtils.java:240)
    at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:412)
    at io.trino.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:261)
    at io.trino.operator.WorkProcessorUtils.lambda$finishWhen$3(WorkProcessorUtils.java:255)
    at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:412)
    at io.trino.operator.WorkProcessorSourceOperatorAdapter.getOutput(WorkProcessorSourceOperatorAdapter.java:133)
    at io.trino.operator.Driver.processInternal(Driver.java:400)
    at io.trino.operator.Driver.lambda$process$8(Driver.java:303)
    at io.trino.operator.Driver.tryWithLock(Driver.java:706)
    at io.trino.operator.Driver.process(Driver.java:295)
    at io.trino.operator.Driver.processForDuration(Driver.java:266)
    at io.trino.execution.SqlTaskExecution$DriverSplitRunner.processFor(SqlTaskExecution.java:890)
    at io.trino.execution.executor.timesharing.PrioritizedSplitRunner.process(PrioritizedSplitRunner.java:187)
    at io.trino.execution.executor.timesharing.TimeSharingTaskExecutor$TaskRunner.run(TimeSharingTaskExecutor.java:565)
    at io.trino.$gen.Trino.run(Unknown Source)
    at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1144)
    at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:642)
    at java.base/java.lang.Thread.run(Thread.java:1570)
nineinchnick commented 2 months ago

Here's how the AWS SDK checks if an exception should be retried: https://github.com/aws/aws-sdk-java-v2/blob/master/core/aws-core/src/main/java/software/amazon/awssdk/awscore/retry/AwsRetryStrategy.java#L149

nineinchnick commented 2 months ago

Actually, since the AWS SDK does retries, we should not retry on any AWS SDK exceptions, depending on how the AWS SDK client is configured in the metadata service.

oskar-szwajkowski commented 2 months ago

Actually, since the AWS SDK does retries, we should not retry on any AWS SDK exceptions, depending on how the AWS SDK client is configured in the metadata service.

There is also for AWS exceptions: https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/core/exception/SdkException.html#retryable() and for google's: https://cloud.google.com/java/docs/reference/google-cloud-core/latest/com.google.cloud.BaseServiceException#com_google_cloud_BaseServiceException_isRetryable__

for azure afaict there is no easy way checking this, but maybe we could have our set of retryable exceptions embedded in file system

nineinchnick commented 2 months ago

Even if we catch a retriable SDK exception, the SDK might have already exhausted the number of retries, if retries are not disabled. The AWS SDK adds another suppressed exception to the last exception, with some retry attempt details. It wraps non-SDK exceptions in a SdkClientException.

oskar-szwajkowski commented 2 months ago

Here is what I found on retries in client libraries: Google: https://cloud.google.com/storage/docs/retry-strategy#client-libraries AWS: https://docs.aws.amazon.com/sdkref/latest/guide/feature-retry-behavior.html Azure: No particular docs, but class com.azure.storage.common.policy.RequestRetryOptions that we seem to use defaults

so each of those clients libraries do handle retries, which makes me think that we shouldn't retry on any of SDK exceptions, or we should either disable SDK retries and do our own, or configure it on client's level

doing external retry loop is making retries multiply and its unpredictable especially with exponential backoffs