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
9.88k stars 2.86k forks source link

After a major compaction Updates and Deletes start to fail for Hive transactional tables #6279

Open abhishekkhanna1 opened 3 years ago

abhishekkhanna1 commented 3 years ago

Once a major compaction takes all the delta files and replaces them with a base file per bucket, updates and deletes begin to fail. Seems like the BUCKET_PATH_MATCHER doesn't account for the base directory which might be causing this.

java.lang.IllegalArgumentException: bucketPath doesn't have the required format: s3://test-bucket/Schema3/testTable/base_0000010/bucket_00000
    at com.google.common.base.Preconditions.checkArgument(Preconditions.java:217)
    at io.prestosql.plugin.hive.HiveUpdatablePageSource.<init>(HiveUpdatablePageSource.java:143)
    at io.prestosql.plugin.hive.HivePageSourceProvider.createPageSource(HivePageSourceProvider.java:207)
    at io.prestosql.plugin.base.classloader.ClassLoaderSafeConnectorPageSourceProvider.createPageSource(ClassLoaderSafeConnectorPageSourceProvider.java:66)
    at io.prestosql.split.PageSourceManager.createPageSource(PageSourceManager.java:64)
    at io.prestosql.operator.ScanFilterAndProjectOperator$SplitToPages.process(ScanFilterAndProjectOperator.java:254)
    at io.prestosql.operator.ScanFilterAndProjectOperator$SplitToPages.process(ScanFilterAndProjectOperator.java:182)
    at io.prestosql.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:319)
    at io.prestosql.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:372)
    at io.prestosql.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:306)
    at io.prestosql.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:372)
    at io.prestosql.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:306)
    at io.prestosql.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:372)
    at io.prestosql.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:221)
    at io.prestosql.operator.WorkProcessorUtils.lambda$processStateMonitor$2(WorkProcessorUtils.java:200)
    at io.prestosql.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:372)
    at io.prestosql.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:221)
    at io.prestosql.operator.WorkProcessorUtils.lambda$finishWhen$3(WorkProcessorUtils.java:215)
    at io.prestosql.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:372)
    at io.prestosql.operator.WorkProcessorSourceOperatorAdapter.getOutput(WorkProcessorSourceOperatorAdapter.java:149)
    at io.prestosql.operator.Driver.processInternal(Driver.java:387)
    at io.prestosql.operator.Driver.lambda$processFor$9(Driver.java:291)
    at io.prestosql.operator.Driver.tryWithLock(Driver.java:683)
    at io.prestosql.operator.Driver.processFor(Driver.java:284)
    at io.prestosql.execution.SqlTaskExecution$DriverSplitRunner.processFor(SqlTaskExecution.java:1076)
    at io.prestosql.execution.executor.PrioritizedSplitRunner.process(PrioritizedSplitRunner.java:163)
    at io.prestosql.execution.executor.TaskExecutor$TaskRunner.run(TaskExecutor.java:484)
    at io.prestosql.$gen.Presto_unknown____20201209_080621_2.run(Unknown Source)
    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:834)

2020-12-09T01:08:05.768-0800    ERROR   remote-task-callback-62 io.prestosql.execution.StageStateMachine    Stage 20201209_090801_00061_nk3pw.1 failed
java.lang.IllegalArgumentException: bucketPath doesn't have the required format: s3://test-bucket/Schema3/testTable/base_0000010/bucket_00000
    at com.google.common.base.Preconditions.checkArgument(Preconditions.java:217)
    at io.prestosql.plugin.hive.HiveUpdatablePageSource.<init>(HiveUpdatablePageSource.java:143)
    at io.prestosql.plugin.hive.HivePageSourceProvider.createPageSource(HivePageSourceProvider.java:207)
    at io.prestosql.plugin.base.classloader.ClassLoaderSafeConnectorPageSourceProvider.createPageSource(ClassLoaderSafeConnectorPageSourceProvider.java:66)
    at io.prestosql.split.PageSourceManager.createPageSource(PageSourceManager.java:64)
    at io.prestosql.operator.ScanFilterAndProjectOperator$SplitToPages.process(ScanFilterAndProjectOperator.java:254)
    at io.prestosql.operator.ScanFilterAndProjectOperator$SplitToPages.process(ScanFilterAndProjectOperator.java:182)
    at io.prestosql.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:319)
    at io.prestosql.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:372)
    at io.prestosql.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:306)
    at io.prestosql.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:372)
    at io.prestosql.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:306)
    at io.prestosql.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:372)
    at io.prestosql.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:221)
    at io.prestosql.operator.WorkProcessorUtils.lambda$processStateMonitor$2(WorkProcessorUtils.java:200)
    at io.prestosql.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:372)
    at io.prestosql.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:221)
    at io.prestosql.operator.WorkProcessorUtils.lambda$finishWhen$3(WorkProcessorUtils.java:215)
    at io.prestosql.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:372)
    at io.prestosql.operator.WorkProcessorSourceOperatorAdapter.getOutput(WorkProcessorSourceOperatorAdapter.java:149)
    at io.prestosql.operator.Driver.processInternal(Driver.java:387)
    at io.prestosql.operator.Driver.lambda$processFor$9(Driver.java:291)
    at io.prestosql.operator.Driver.tryWithLock(Driver.java:683)
    at io.prestosql.operator.Driver.processFor(Driver.java:284)
    at io.prestosql.execution.SqlTaskExecution$DriverSplitRunner.processFor(SqlTaskExecution.java:1076)
    at io.prestosql.execution.executor.PrioritizedSplitRunner.process(PrioritizedSplitRunner.java:163)
    at io.prestosql.execution.executor.TaskExecutor$TaskRunner.run(TaskExecutor.java:484)
    at io.prestosql.$gen.Presto_unknown____20201209_080621_2.run(Unknown Source)
    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:834)

2020-12-09T01:08:05.977-0800    ERROR   transaction-finishing-7 io.prestosql.transaction.InMemoryTransactionManager Connector threw exception on abort
java.lang.IllegalStateException: Tried to rollback buffered metastore operations after transaction has been committed/aborted
    at io.prestosql.plugin.hive.metastore.SemiTransactionalHiveMetastore.rollback(SemiTransactionalHiveMetastore.java:1159)
    at io.prestosql.plugin.hive.HiveMetadata.rollback(HiveMetadata.java:2891)
    at io.prestosql.plugin.hive.HiveConnector.rollback(HiveConnector.java:222)
    at io.prestosql.transaction.InMemoryTransactionManager$TransactionMetadata$ConnectorTransactionMetadata.abort(InMemoryTransactionManager.java:601)
    at io.prestosql.transaction.InMemoryTransactionManager$TransactionMetadata.safeAbort(InMemoryTransactionManager.java:531)
    at io.prestosql.transaction.InMemoryTransactionManager$TransactionMetadata.lambda$abortInternal$9(InMemoryTransactionManager.java:524)
    at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
    at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:125)
    at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:69)
    at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:78)
    at io.airlift.concurrent.BoundedExecutor.drainQueue(BoundedExecutor.java:80)
    at io.prestosql.$gen.Presto_unknown____20201209_080621_2.run(Unknown Source)
    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:834)
losipiuk commented 3 years ago

cc: @djsstarburst

djsstarburst commented 3 years ago

Oh my, I didn't know about this post-compaction format. I wonder why the existing tests for compaction didn't see this.

@abhishekkhanna1, could you supply instructions to reproduce this? Thanks!

djsstarburst commented 3 years ago

This is fixed by changes I just force-pushed to the #5861 UPDATE PR. Those changes included a unit test that demonstrate that major compaction files are handled correctly by UPDATE. Thanks @abhishekkhanna1!

findepi commented 2 years ago

This is fixed by changes I just force-pushed to the #5861 UPDATE PR.

@djsstarburst @abhishekkhanna1 can this issue be closed now?