Skip to content

Another race condition in Alluxio cache #25899

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
jirislav opened this issue Jun 2, 2025 · 3 comments
Open

Another race condition in Alluxio cache #25899

jirislav opened this issue Jun 2, 2025 · 3 comments

Comments

@jirislav
Copy link

jirislav commented Jun 2, 2025

We're now also having a race condition in a local Alluxio cache in the latest Trino 475.

But it's different from #21342:

See the logs attached

This is the log of Trino coordinator, which also acts as a worker. It happens only under heavy load. The local cache is an EBS GP3 volume, formatted with ext4.

2025-05-31T07:37:17.524Z	ERROR	SplitRunner-20250531_073709_00085_tnnxk.0.5.0-8-7824	alluxio.client.file.cache.LocalCacheManager	Data corrupted page PageId{FileId=e4c16a019a7f00ecc3665b4a7d475f87, PageIndex=8} from pageStore
alluxio.exception.PageCorruptedException: The page PageId{FileId=e4c16a019a7f00ecc3665b4a7d475f87, PageIndex=8} (/tmp/trino-cache/LOCAL/4194304/64/e4c16a019a7f00ecc3665b4a7d475f87/8) probably has been corrupted, page-offset 0, bytes to read 4194304, page file length 249856
	at alluxio.client.file.cache.store.LocalPageStore.get(LocalPageStore.java:106)
	at alluxio.client.file.cache.LocalCacheManager.getPage(LocalCacheManager.java:976)
	at alluxio.client.file.cache.LocalCacheManager.get(LocalCacheManager.java:628)
	at alluxio.client.file.cache.CacheManager.get(CacheManager.java:280)
	at alluxio.client.file.cache.NoExceptionCacheManager.get(NoExceptionCacheManager.java:123)
	at io.trino.filesystem.alluxio.TracingCacheManager.lambda$get$15(TracingCacheManager.java:231)
	at io.trino.filesystem.tracing.Tracing.withTracing(Tracing.java:51)
	at io.trino.filesystem.alluxio.TracingCacheManager.get(TracingCacheManager.java:231)
	at io.trino.filesystem.alluxio.AlluxioInputHelper.readPageFromCache(AlluxioInputHelper.java:146)
	at io.trino.filesystem.alluxio.AlluxioInputHelper.doInternalCacheRead(AlluxioInputHelper.java:112)
	at io.trino.filesystem.alluxio.AlluxioInputHelper.lambda$doCacheRead$0(AlluxioInputHelper.java:87)
	at io.trino.filesystem.tracing.Tracing.withTracing(Tracing.java:51)
	at io.trino.filesystem.alluxio.AlluxioInputHelper.doCacheRead(AlluxioInputHelper.java:85)
	at io.trino.filesystem.alluxio.AlluxioInput.readFully(AlluxioInput.java:69)
	at io.trino.plugin.hive.parquet.TrinoParquetDataSource.readInternal(TrinoParquetDataSource.java:64)
	at io.trino.parquet.AbstractParquetDataSource.readFully(AbstractParquetDataSource.java:122)
	at io.trino.parquet.AbstractParquetDataSource$ReferenceCountedReader.read(AbstractParquetDataSource.java:332)
	at io.trino.parquet.ChunkReader.readUnchecked(ChunkReader.java:31)
	at io.trino.parquet.reader.ChunkedInputStream.readNextChunk(ChunkedInputStream.java:150)
	at io.trino.parquet.reader.ChunkedInputStream.read(ChunkedInputStream.java:94)
	at java.base/java.io.InputStream.readNBytes(InputStream.java:508)
	at io.trino.parquet.reader.ChunkedInputStream.getSlice(ChunkedInputStream.java:72)
	at io.trino.parquet.reader.ParquetColumnChunkIterator.readDataPageV1(ParquetColumnChunkIterator.java:148)
	at io.trino.parquet.reader.ParquetColumnChunkIterator.next(ParquetColumnChunkIterator.java:95)
	at io.trino.parquet.reader.ParquetColumnChunkIterator.next(ParquetColumnChunkIterator.java:42)
	at com.google.common.collect.Iterators$PeekingImpl.next(Iterators.java:1200)
	at io.trino.parquet.reader.PageReader.readPage(PageReader.java:113)
	at io.trino.parquet.reader.NestedColumnReader.readNextPage(NestedColumnReader.java:470)
	at io.trino.parquet.reader.NestedColumnReader.readNullable(NestedColumnReader.java:170)
	at io.trino.parquet.reader.NestedColumnReader.readPrimitive(NestedColumnReader.java:144)
	at io.trino.parquet.reader.ParquetReader.readPrimitive(ParquetReader.java:645)
	at io.trino.parquet.reader.ParquetReader.readColumnChunk(ParquetReader.java:731)
	at io.trino.parquet.reader.ParquetReader.readStruct(ParquetReader.java:555)
	at io.trino.parquet.reader.ParquetReader.readColumnChunk(ParquetReader.java:722)
	at io.trino.parquet.reader.ParquetReader.readMap(ParquetReader.java:540)
	at io.trino.parquet.reader.ParquetReader.readColumnChunk(ParquetReader.java:725)
	at io.trino.parquet.reader.ParquetReader.readStruct(ParquetReader.java:555)
	at io.trino.parquet.reader.ParquetReader.readColumnChunk(ParquetReader.java:722)
	at io.trino.parquet.reader.ParquetReader.readBlock(ParquetReader.java:711)
	at io.trino.parquet.reader.ParquetReader$ParquetSourcePage.getBlock(ParquetReader.java:338)
	at io.trino.plugin.hive.TransformConnectorPageSource$DereferenceFieldTransform.apply(TransformConnectorPageSource.java:246)
	at io.trino.plugin.hive.TransformConnectorPageSource$DereferenceFieldTransform.apply(TransformConnectorPageSource.java:232)
	at io.trino.plugin.hive.TransformConnectorPageSource$TransformSourcePage.getBlock(TransformConnectorPageSource.java:337)
	at io.trino.operator.project.InputChannels$InputChannelsSourcePage.getBlock(InputChannels.java:145)
	at io.trino.$gen.PageFilter_20250531_073710_556.filter(Unknown Source)
	at io.trino.$gen.PageFilter_20250531_073710_556.filter(Unknown Source)
	at io.trino.sql.gen.columnar.PageFilterEvaluator.evaluate(PageFilterEvaluator.java:42)
	at io.trino.operator.project.PageProcessor.createWorkProcessor(PageProcessor.java:126)
	at io.trino.operator.ScanFilterAndProjectOperator$SplitToPages.lambda$processPageSource$2(ScanFilterAndProjectOperator.java:289)
	at io.trino.operator.WorkProcessorUtils.lambda$flatMap$4(WorkProcessorUtils.java:285)
	at io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:359)
	at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:423)
	at io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:346)
	at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:423)
	at io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:346)
	at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:423)
	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:423)
	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:423)
	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:423)
	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:423)
	at io.trino.operator.WorkProcessorSourceOperatorAdapter.getOutput(WorkProcessorSourceOperatorAdapter.java:132)
	at io.trino.operator.Driver.processInternal(Driver.java:403)
	at io.trino.operator.Driver.lambda$process$8(Driver.java:306)
	at io.trino.operator.Driver.tryWithLock(Driver.java:709)
	at io.trino.operator.Driver.process(Driver.java:298)
	at io.trino.operator.Driver.processForDuration(Driver.java:269)
	at io.trino.execution.SqlTaskExecution$DriverSplitRunner.processFor(SqlTaskExecution.java:890)
	at io.trino.execution.executor.dedicated.SplitProcessor.run(SplitProcessor.java:77)
	at io.trino.execution.executor.dedicated.TaskEntry$VersionEmbedderBridge.lambda$run$0(TaskEntry.java:201)
	at io.trino.$gen.Trino_475____20250531_072204_2.run(Unknown Source)
	at io.trino.execution.executor.dedicated.TaskEntry$VersionEmbedderBridge.run(TaskEntry.java:202)
	at io.trino.execution.executor.scheduler.FairScheduler.runTask(FairScheduler.java:177)
	at io.trino.execution.executor.scheduler.FairScheduler.lambda$submit$0(FairScheduler.java:164)
	at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:545)
	at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:128)
	at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:74)
	at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:80)
	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1095)
	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:619)
	at java.base/java.lang.Thread.run(Thread.java:1447)

I think that the CacheHelper#put still doesn't guarantee that file is written by Alluxio and it is ready to be read, as pointed out in the related issue.

@raunaqmorarka
Copy link
Member

@apc999
Copy link
Contributor

apc999 commented Jun 2, 2025

Thanks for reporting this. @JiamingMai and I will take a look

@JiamingMai
Copy link
Contributor

It seems that it didn't finish writing the full page before reading. Can you share your Alluxio's configurations and see if there is any exception in Alluxio's logs?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Development

No branches or pull requests

4 participants