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.15k stars 2.93k forks source link

[DELTA] Trino creates empty `_last_checkpoint`, which later prevents read on the table #22748

Open vinay-kl opened 1 month ago

vinay-kl commented 1 month ago

Trino is unable to query the delta tables for which the _last_checkpoint file is empty or missing and on which the older log entries have been cleaned up

We are using Trino (v448) itself for writing the data onto this delta table, it seems like the TRINO was able to write JSON and checkpoint file for 251 table version but the _last_checkpoint file wasn't updated.

At the time of write in JAN-2024 we were using v434 of Trino

Query and failure stack-trace

trino> show create table udp.CLOSED_FUNNEL;
Query 20240721_053025_00000_a6jti failed: Metadata not found in transaction log for udp.closed_funnel
io.trino.spi.TrinoException: Metadata not found in transaction log for udp.closed_funnel
    at io.trino.plugin.deltalake.CorruptedDeltaLakeTableHandle.createException(CorruptedDeltaLakeTableHandle.java:38)
    at io.trino.plugin.deltalake.DeltaLakeMetadata.checkValidTableHandle(DeltaLakeMetadata.java:3610)
    at io.trino.plugin.deltalake.DeltaLakeMetadata.getTableMetadata(DeltaLakeMetadata.java:608)
    at io.trino.plugin.base.classloader.ClassLoaderSafeConnectorMetadata.getTableMetadata(ClassLoaderSafeConnectorMetadata.java:278)
    at io.trino.tracing.TracingConnectorMetadata.getTableMetadata(TracingConnectorMetadata.java:248)
    at io.trino.metadata.MetadataManager.getTableMetadata(MetadataManager.java:465)
    at io.trino.tracing.TracingMetadata.getTableMetadata(TracingMetadata.java:298)
    at io.trino.sql.rewrite.ShowQueriesRewrite$Visitor.visitShowCreate(ShowQueriesRewrite.java:678)
    at io.trino.sql.rewrite.ShowQueriesRewrite$Visitor.visitShowCreate(ShowQueriesRewrite.java:229)
    at io.trino.sql.tree.ShowCreate.accept(ShowCreate.java:70)
    at io.trino.sql.tree.AstVisitor.process(AstVisitor.java:27)
    at io.trino.sql.rewrite.ShowQueriesRewrite.rewrite(ShowQueriesRewrite.java:226)
    at io.trino.sql.rewrite.StatementRewrite.rewrite(StatementRewrite.java:54)
    at io.trino.sql.analyzer.Analyzer.analyze(Analyzer.java:92)
    at io.trino.sql.analyzer.Analyzer.analyze(Analyzer.java:86)
    at io.trino.execution.SqlQueryExecution.analyze(SqlQueryExecution.java:274)
    at io.trino.execution.SqlQueryExecution.<init>(SqlQueryExecution.java:209)
    at io.trino.execution.SqlQueryExecution$SqlQueryExecutionFactory.createQueryExecution(SqlQueryExecution.java:850)
    at io.trino.dispatcher.LocalDispatchQueryFactory.lambda$createDispatchQuery$0(LocalDispatchQueryFactory.java:153)
    at io.trino.$gen.Trino_437____20240717_100640_2.call(Unknown Source)
    at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:131)
    at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:76)
    at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:82)
    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:1583)
Caused by: io.trino.spi.TrinoException: Metadata not found in transaction log for udp.closed_funnel
    at io.trino.plugin.deltalake.DeltaLakeMetadata.getTableHandle(DeltaLakeMetadata.java:552)
    at io.trino.plugin.deltalake.DeltaLakeMetadata.getTableHandle(DeltaLakeMetadata.java:310)
    at io.trino.spi.connector.ConnectorMetadata.getTableHandle(ConnectorMetadata.java:141)
    at io.trino.plugin.base.classloader.ClassLoaderSafeConnectorMetadata.getTableHandle(ClassLoaderSafeConnectorMetadata.java:1227)
    at io.trino.tracing.TracingConnectorMetadata.getTableHandle(TracingConnectorMetadata.java:149)
    at io.trino.metadata.MetadataManager.lambda$getTableHandle$5(MetadataManager.java:285)
    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:1858)
    at io.trino.metadata.MetadataManager.getRedirectionAwareTableHandle(MetadataManager.java:1847)
    at io.trino.tracing.TracingMetadata.getRedirectionAwareTableHandle(TracingMetadata.java:1437)
    at io.trino.sql.rewrite.ShowQueriesRewrite$Visitor.visitShowCreate(ShowQueriesRewrite.java:672)
    ... 18 more

On further inspection of telemetry data, we found out that Trino is trying to read 00000000000000000000.json which doesn't exist anymore and has been long deleted as part of log-entry cleanups

java.io.FileNotFoundException: abfs://container@account.dfs.core.windows.net/prod-data/udp.db/closed_funnel-c76a613ade8249469eb7b66d83a82e20/_delta_log/00000000000000000000.json
   at io.trino.filesystem.hdfs.HdfsInputFile.lambda$openFile$1(HdfsInputFile.java:124)
   at io.trino.hdfs.authentication.NoHdfsAuthentication.doAs(NoHdfsAuthentication.java:25)
   at io.trino.hdfs.HdfsEnvironment.doAs(HdfsEnvironment.java:134)
   at io.trino.filesystem.hdfs.HdfsInputFile.openFile(HdfsInputFile.java:117)
   at io.trino.filesystem.hdfs.HdfsInputFile.newStream(HdfsInputFile.java:72)
   at io.trino.filesystem.tracing.Tracing.withTracing(Tracing.java:47)
   at io.trino.filesystem.tracing.TracingInputFile.newStream(TracingInputFile.java:64)
   at io.trino.plugin.deltalake.transactionlog.checkpoint.TransactionLogTail.getEntriesFromJson(TransactionLogTail.java:124)
   at io.trino.plugin.deltalake.transactionlog.checkpoint.TransactionLogTail.loadNewTail(TransactionLogTail.java:81)
   at io.trino.plugin.deltalake.transactionlog.checkpoint.TransactionLogTail.loadNewTail(TransactionLogTail.java:59)
   at io.trino.plugin.deltalake.transactionlog.TableSnapshot.load(TableSnapshot.java:96)
   at io.trino.plugin.deltalake.transactionlog.TransactionLogAccess.lambda$loadSnapshot$1(TransactionLogAccess.java:167)
   at io.trino.cache.EmptyCache.get(EmptyCache.java:94)
   at io.trino.plugin.deltalake.transactionlog.TransactionLogAccess.loadSnapshot(TransactionLogAccess.java:166)
   at io.trino.plugin.deltalake.DeltaLakeMetadata.getSnapshot(DeltaLakeMetadata.java:464)
   at io.trino.plugin.deltalake.DeltaLakeMetadata.getTableHandle(DeltaLakeMetadata.java:533)
   at io.trino.plugin.deltalake.DeltaLakeMetadata.getTableHandle(DeltaLakeMetadata.java:310)
   at io.trino.spi.connector.ConnectorMetadata.getTableHandle(ConnectorMetadata.java:141)
   at io.trino.plugin.base.classloader.ClassLoaderSafeConnectorMetadata.getTableHandle(ClassLoaderSafeConnectorMetadata.java:1227)
   at io.trino.tracing.TracingConnectorMetadata.getTableHandle(TracingConnectorMetadata.java:149)
   at io.trino.metadata.MetadataManager.lambda$getTableHandle$5(MetadataManager.java:285)
   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:1855)
   at io.trino.metadata.MetadataManager.getRedirectionAwareTableHandle(MetadataManager.java:1847)
   at io.trino.tracing.TracingMetadata.getRedirectionAwareTableHandle(TracingMetadata.java:1437)
   at io.trino.sql.rewrite.ShowQueriesRewrite$Visitor.visitShowCreate(ShowQueriesRewrite.java:672)
   at io.trino.sql.rewrite.ShowQueriesRewrite$Visitor.visitShowCreate(ShowQueriesRewrite.java:229)
   at io.trino.sql.tree.ShowCreate.accept(ShowCreate.java:70)
   at io.trino.sql.tree.AstVisitor.process(AstVisitor.java:27)
   at io.trino.sql.rewrite.ShowQueriesRewrite.rewrite(ShowQueriesRewrite.java:226)
   at io.trino.sql.rewrite.StatementRewrite.rewrite(StatementRewrite.java:54)
   at io.trino.sql.analyzer.Analyzer.analyze(Analyzer.java:92)
   at io.trino.sql.analyzer.Analyzer.analyze(Analyzer.java:86)
   at io.trino.execution.SqlQueryExecution.analyze(SqlQueryExecution.java:274)
   at io.trino.execution.SqlQueryExecution.<init>(SqlQueryExecution.java:209)
   at io.trino.execution.SqlQueryExecution$SqlQueryExecutionFactory.createQueryExecution(SqlQueryExecution.java:850)
   at io.trino.dispatcher.LocalDispatchQueryFactory.lambda$createDispatchQuery$0(LocalDispatchQueryFactory.java:153)
   at io.trino.$gen.Trino_437____20240718_151634_2.call(Unknown Source)
   at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:131)
   at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:76)
   at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:82)
   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:1583)
Caused by: java.io.FileNotFoundException: Operation failed: "The specified path does not exist.", 404, HEAD, https://account.dfs.core.windows.net/container/prod-data/udp.db/closed_funnel-c76a613ade8249469eb7b66d83a82e20/_delta_log/00000000000000000000.json?upn=false&action=getStatus&timeout=90
   at org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem.checkException(AzureBlobFileSystem.java:1481)
   at org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem.open(AzureBlobFileSystem.java:276)
   at org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem.open(AzureBlobFileSystem.java:261)
   at io.trino.hdfs.TrinoFileSystemCache$FileSystemWrapper.open(TrinoFileSystemCache.java:346)
   at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:1004)
   at io.trino.filesystem.hdfs.HdfsInputFile.lambda$openFile$1(HdfsInputFile.java:119)
   ... 44 more
Caused by: Operation failed: "The specified path does not exist.", 404, HEAD, https://account.dfs.core.windows.net/container/prod-data/udp.db/closed_funnel-c76a613ade8249469eb7b66d83a82e20/_delta_log/00000000000000000000.json?upn=false&action=getStatus&timeout=90
   at org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.completeExecute(AbfsRestOperation.java:234)
   at org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.lambda$execute$0(AbfsRestOperation.java:194)
   at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.measureDurationOfInvocation(IOStatisticsBinding.java:494)
   at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation(IOStatisticsBinding.java:465)
   at org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.execute(AbfsRestOperation.java:192)
   at org.apache.hadoop.fs.azurebfs.services.AbfsClient.getPathStatus(AbfsClient.java:831)
   at org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.openFileForRead(AzureBlobFileSystemStore.java:775)
   at org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem.open(AzureBlobFileSystem.java:273)
   ... 48 more

File system listing of _delta_log folder

-rw-r--r--   1 sshuser sshuser     108179 2024-01-03 05:39 abfs://container@account.dfs.core.windows.net/prod-data/udp.db/closed_funnel-c76a613ade8249469eb7b66d83a82e20/_delta_log/00000000000000000251.checkpoint.parquet
-rw-r--r--   1 sshuser sshuser        343 2024-01-03 05:39 abfs://container@account.dfs.core.windows.net/prod-data/udp.db/closed_funnel-c76a613ade8249469eb7b66d83a82e20/_delta_log/00000000000000000251.json
drwxr-x---   - sshuser sshuser          0 2023-12-28 11:33 abfs://container@account.dfs.core.windows.net/prod-data/udp.db/closed_funnel-c76a613ade8249469eb7b66d83a82e20/_delta_log/__tmp_path_dir
-rw-r--r--   1 sshuser sshuser          0 2024-01-03 05:39 abfs://container@account.dfs.core.windows.net/prod-data/udp.db/closed_funnel-c76a613ade8249469eb7b66d83a82e20/_delta_log/_last_checkpoint
drwxr-x---   - sshuser sshuser          0 2023-08-30 18:32 abfs://container@account.dfs.core.windows.net/prod-data/udp.db/closed_funnel-c76a613ade8249469eb7b66d83a82e20/_delta_log/_trino_meta

Steps for re-creation purpose only, the actual issue could've happened due to other reasons

trino> create table dev_1.deletion_checkpoint_test WITH (checkpoint_interval = 1) as select 1 as a;
CREATE TABLE: 1 row

trino> select * from dev_1.deletion_checkpoint_test;
 a 
---
 1 
(1 row)

trino> insert into dev_1.deletion_checkpoint_test select 2 as a;
INSERT: 1 row

trino> select * from dev_1.deletion_checkpoint_test;
 a 
---
 2 
 1

Post the above, delete the 00000000000000000000.json and _last_checkpoint files

trino> select * from dev_1.deletion_checkpoint_test;
Query 20240721_070505_00006_a6jti failed: Metadata not found in transaction log for dev_1.deletion_checkpoint_test
io.trino.spi.TrinoException: Metadata not found in transaction log for dev_1.deletion_checkpoint_test
    at io.trino.plugin.deltalake.CorruptedDeltaLakeTableHandle.createException(CorruptedDeltaLakeTableHandle.java:38)
    at io.trino.plugin.deltalake.DeltaLakeMetadata.checkValidTableHandle(DeltaLakeMetadata.java:3610)
    at io.trino.plugin.deltalake.DeltaLakeMetadata.getTableMetadata(DeltaLakeMetadata.java:608)
    at io.trino.spi.connector.ConnectorMetadata.getTableSchema(ConnectorMetadata.java:256)

Telemetry stack-trace

java.io.FileNotFoundException: abfs://container@storage.dfs.core.windows.net/prod-data/dev_1/deletion_checkpoint_test/_delta_log/00000000000000000000.json
   at io.trino.filesystem.hdfs.HdfsInputFile.lambda$openFile$1(HdfsInputFile.java:124)
   at io.trino.hdfs.authentication.NoHdfsAuthentication.doAs(NoHdfsAuthentication.java:27)
   at io.trino.hdfs.HdfsEnvironment.doAs(HdfsEnvironment.java:134)
   at io.trino.filesystem.hdfs.HdfsInputFile.openFile(HdfsInputFile.java:117)
   at io.trino.filesystem.hdfs.HdfsInputFile.newStream(HdfsInputFile.java:72)
   at io.trino.filesystem.tracing.Tracing.withTracing(Tracing.java:47)
   at io.trino.filesystem.tracing.TracingInputFile.newStream(TracingInputFile.java:64)
   at io.trino.plugin.deltalake.transactionlog.checkpoint.TransactionLogTail.getEntriesFromJson(TransactionLogTail.java:126)
   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 io.trino.cache.EmptyCache.get(EmptyCache.java:94)
   at io.trino.plugin.deltalake.transactionlog.TransactionLogAccess.loadSnapshot(TransactionLogAccess.java:178)
   at io.trino.plugin.deltalake.DeltaLakeMetadata.getSnapshot(DeltaLakeMetadata.java:477)
   at io.trino.plugin.deltalake.DeltaLakeMetadata.getTableHandle(DeltaLakeMetadata.java:592)
   at io.trino.plugin.deltalake.DeltaLakeMetadata.getTableHandle(DeltaLakeMetadata.java:323)
   at io.trino.plugin.base.classloader.ClassLoaderSafeConnectorMetadata.getTableHandle(ClassLoaderSafeConnectorMetadata.java:1259)
   at io.trino.tracing.TracingConnectorMetadata.getTableHandle(TracingConnectorMetadata.java:149)
   at io.trino.metadata.MetadataManager.lambda$getTableHandle$5(MetadataManager.java:283)
   at java.base/java.util.Optional.flatMap(Optional.java:289)
   at io.trino.metadata.MetadataManager.getTableHandle(MetadataManager.java:277)
   at io.trino.metadata.MetadataManager.getRedirectionAwareTableHandle(MetadataManager.java:1915)
   at io.trino.metadata.MetadataManager.getRedirectionAwareTableHandle(MetadataManager.java:1907)
   at io.trino.tracing.TracingMetadata.getRedirectionAwareTableHandle(TracingMetadata.java:1493)
   at io.trino.sql.analyzer.StatementAnalyzer$Visitor.getTableHandle(StatementAnalyzer.java:5843)
   at io.trino.sql.analyzer.StatementAnalyzer$Visitor.visitTable(StatementAnalyzer.java:2292)
   at io.trino.sql.analyzer.StatementAnalyzer$Visitor.visitTable(StatementAnalyzer.java:520)
   at io.trino.sql.tree.Table.accept(Table.java:60)
   at io.trino.sql.tree.AstVisitor.process(AstVisitor.java:27)
   at io.trino.sql.analyzer.StatementAnalyzer$Visitor.process(StatementAnalyzer.java:539)
   at io.trino.sql.analyzer.StatementAnalyzer$Visitor.analyzeFrom(StatementAnalyzer.java:4892)
   at io.trino.sql.analyzer.StatementAnalyzer$Visitor.visitQuerySpecification(StatementAnalyzer.java:3082)
   at io.trino.sql.analyzer.StatementAnalyzer$Visitor.visitQuerySpecification(StatementAnalyzer.java:520)
   at io.trino.sql.tree.QuerySpecification.accept(QuerySpecification.java:155)
   at io.trino.sql.tree.AstVisitor.process(AstVisitor.java:27)
   at io.trino.sql.analyzer.StatementAnalyzer$Visitor.process(StatementAnalyzer.java:539)
   at io.trino.sql.analyzer.StatementAnalyzer$Visitor.process(StatementAnalyzer.java:547)
   at io.trino.sql.analyzer.StatementAnalyzer$Visitor.visitQuery(StatementAnalyzer.java:1570)
   at io.trino.sql.analyzer.StatementAnalyzer$Visitor.visitQuery(StatementAnalyzer.java:520)
   at io.trino.sql.tree.Query.accept(Query.java:118)
   at io.trino.sql.tree.AstVisitor.process(AstVisitor.java:27)
   at io.trino.sql.analyzer.StatementAnalyzer$Visitor.process(StatementAnalyzer.java:539)
   at io.trino.sql.analyzer.StatementAnalyzer.analyze(StatementAnalyzer.java:499)
   at io.trino.sql.analyzer.StatementAnalyzer.analyze(StatementAnalyzer.java:488)
   at io.trino.sql.analyzer.Analyzer.analyze(Analyzer.java:97)
   at io.trino.sql.analyzer.Analyzer.analyze(Analyzer.java:86)
   at io.trino.execution.SqlQueryExecution.analyze(SqlQueryExecution.java:285)
   at io.trino.execution.SqlQueryExecution.<init>(SqlQueryExecution.java:218)
   at io.trino.execution.SqlQueryExecution$SqlQueryExecutionFactory.createQueryExecution(SqlQueryExecution.java:884)
   at io.trino.dispatcher.LocalDispatchQueryFactory.lambda$createDispatchQuery$0(LocalDispatchQueryFactory.java:153)
   at io.trino.$gen.Trino_448____20240721_073438_2.call(Unknown Source)
   at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:131)
   at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:76)
   at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:82)
   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)
Caused by: java.io.FileNotFoundException: Operation failed: "The specified path does not exist.", 404, HEAD, https://storage.dfs.core.windows.net/container/prod-data/dev_1/deletion_checkpoint_test/_delta_log/00000000000000000000.json?upn=false&action=getStatus&timeout=90
   at org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem.checkException(AzureBlobFileSystem.java:1481)
   at org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem.open(AzureBlobFileSystem.java:276)
   at org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem.open(AzureBlobFileSystem.java:261)
   at io.trino.hdfs.TrinoFileSystemCache$FileSystemWrapper.open(TrinoFileSystemCache.java:345)
   at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:993)
   at io.trino.filesystem.hdfs.HdfsInputFile.lambda$openFile$1(HdfsInputFile.java:119)
   ... 57 more
Caused by: Operation failed: "The specified path does not exist.", 404, HEAD, https://storage.dfs.core.windows.net/container/prod-data/dev_1/deletion_checkpoint_test/_delta_log/00000000000000000000.json?upn=false&action=getStatus&timeout=90
   at org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.completeExecute(AbfsRestOperation.java:234)
   at org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.lambda$execute$0(AbfsRestOperation.java:194)
   at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.measureDurationOfInvocation(IOStatisticsBinding.java:494)
   at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation(IOStatisticsBinding.java:465)
   at org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation.execute(AbfsRestOperation.java:192)
   at org.apache.hadoop.fs.azurebfs.services.AbfsClient.getPathStatus(AbfsClient.java:831)
   at org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.openFileForRead(AzureBlobFileSystemStore.java:775)
   at org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem.open(AzureBlobFileSystem.java:273)
   ... 61 more

FYI, DBR and OSS is able to read & write to the same table without any issues

findinpath commented 1 month ago

This issues seems to be related to previous work done on:

We've identified before a situation where the file was opened for being rewritten and in case of a failure while writing the new content, the file was left empty.

We may be dealing with the same thing on Azure. I see though that you're not using the native file systems, but rather hdfs. I highly recommend switching to Trino native azure client instead of HDFS.

https://github.com/trinodb/trino/blob/0002afaf400fe2e7ce69416f633ec562966cb933/lib/trino-filesystem-azure/src/main/java/io/trino/filesystem/azure/AzureOutputFile.java#L69-L75

https://github.com/trinodb/trino/blob/0002afaf400fe2e7ce69416f633ec562966cb933/lib/trino-hdfs/src/main/java/io/trino/filesystem/hdfs/HdfsOutputFile.java#L66-L72

vinay-kl commented 1 month ago

I highly recommend switching to Trino native azure client instead of HDFS.

@findinpath we make use of both was & abfs protocols which basically deals with gen2 and gen2 with HNS enabled. we are on the verge of making it abfs/abfss streamlined so we can move to Oauth which is only supported with abfs[s] protocols.