Open fritzb opened 1 year ago
I think this is the same issue as https://github.com/trinodb/trino/issues/14836
@findepi I applied the patch from #14836 and I still getting the exception. I think #14836 addressed issue with complex data structure (map, array) and I do not have any complex type columns in the schema.
here are the steps:
trino> update metrics.map_making_metrics set meta_row_state = 0;
Query 20230209_050316_00004_u7pum, FAILED, 3 nodes
http://trino.dev.deepmap.ai/ui/query.html?20230209_050316_00004_u7pum
Splits: 95 total, 0 done (0.00%)
CPU Time: 0.1s total, 5.31K rows/s, 211KB/s, 35% active
Per Node: 0.0 parallelism, 226 rows/s, 9.03KB/s
Parallelism: 0.1
Peak Memory: 1022KB
0.93 [632 rows, 25.2KB] [680 rows/s, 27.1KB/s]
Query 20230209_050316_00004_u7pum failed: Multiple entries with same key: 2147483646=$row_id._file and 2147483646=_file
io.trino.spi.TrinoException: Multiple entries with same key: 2147483646=$row_id._file and 2147483646=_file
at io.trino.plugin.iceberg.IcebergPageSource.getNextPage(IcebergPageSource.java:142)
at io.trino.operator.ScanFilterAndProjectOperator$ConnectorPageSourceToPages.process(ScanFilterAndProjectOperator.java:387)
at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:413)
at io.trino.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:262)
at io.trino.operator.WorkProcessorUtils$YieldingProcess.process(WorkProcessorUtils.java:182)
at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:413)
at io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:347)
at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:413)
at io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:347)
at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:413)
at io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:347)
at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:413)
at io.trino.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:262)
at io.trino.operator.WorkProcessorUtils$BlockingProcess.process(WorkProcessorUtils.java:208)
at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:413)
at io.trino.operator.WorkProcessorUtils.lambda$flatten$6(WorkProcessorUtils.java:318)
at io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:360)
at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:413)
at io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:347)
at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:413)
at io.trino.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:262)
at io.trino.operator.WorkProcessorUtils.lambda$processStateMonitor$2(WorkProcessorUtils.java:241)
at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:413)
at io.trino.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:262)
at io.trino.operator.WorkProcessorUtils.lambda$finishWhen$3(WorkProcessorUtils.java:256)
at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:413)
at io.trino.operator.WorkProcessorSourceOperatorAdapter.getOutput(WorkProcessorSourceOperatorAdapter.java:146)
at io.trino.operator.Driver.processInternal(Driver.java:394)
at io.trino.operator.Driver.lambda$process$8(Driver.java:297)
at io.trino.operator.Driver.tryWithLock(Driver.java:689)
at io.trino.operator.Driver.process(Driver.java:289)
at io.trino.operator.Driver.processForDuration(Driver.java:260)
at io.trino.execution.SqlTaskExecution$DriverSplitRunner.processFor(SqlTaskExecution.java:755)
at io.trino.execution.executor.PrioritizedSplitRunner.process(PrioritizedSplitRunner.java:165)
at io.trino.execution.executor.TaskExecutor$TaskRunner.run(TaskExecutor.java:523)
at io.trino.$gen.Trino_406_dirty____20230209_045148_2.run(Unknown Source)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
at java.base/java.lang.Thread.run(Thread.java:833)
Caused by: java.lang.IllegalArgumentException: Multiple entries with same key: 2147483646=$row_id._file and 2147483646=_file
at org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap.conflictException(ImmutableMap.java:377)
at org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap.checkNoConflict(ImmutableMap.java:371)
at org.apache.iceberg.relocated.com.google.common.collect.RegularImmutableMap.checkNoConflictInKeyBucket(RegularImmutableMap.java:241)
at org.apache.iceberg.relocated.com.google.common.collect.RegularImmutableMap.fromEntryArrayCheckingBucketOverflow(RegularImmutableMap.java:132)
at org.apache.iceberg.relocated.com.google.common.collect.RegularImmutableMap.fromEntryArray(RegularImmutableMap.java:94)
at org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap$Builder.build(ImmutableMap.java:573)
at org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap$Builder.buildOrThrow(ImmutableMap.java:601)
at org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap$Builder.build(ImmutableMap.java:588)
at org.apache.iceberg.types.IndexByName.byId(IndexByName.java:81)
at org.apache.iceberg.types.TypeUtil.indexNameById(TypeUtil.java:160)
at org.apache.iceberg.Schema.lazyIdToName(Schema.java:183)
at org.apache.iceberg.Schema.<init>(Schema.java:112)
at org.apache.iceberg.Schema.<init>(Schema.java:91)
at org.apache.iceberg.Schema.<init>(Schema.java:83)
at org.apache.iceberg.Schema.<init>(Schema.java:79)
at io.trino.plugin.iceberg.IcebergUtil.schemaFromHandles(IcebergUtil.java:250)
at io.trino.plugin.iceberg.delete.EqualityDeleteFilter.createPredicate(EqualityDeleteFilter.java:49)
at io.trino.plugin.iceberg.IcebergPageSourceProvider.lambda$createPageSource$7(IcebergPageSourceProvider.java:332)
at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
at java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1625)
at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509)
at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
at java.base/java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:921)
at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
at java.base/java.util.stream.ReferencePipeline.reduce(ReferencePipeline.java:662)
at io.trino.plugin.iceberg.IcebergPageSourceProvider.lambda$createPageSource$8(IcebergPageSourceProvider.java:333)
at com.google.common.base.Suppliers$NonSerializableMemoizingSupplier.get(Suppliers.java:183)
at io.trino.plugin.iceberg.IcebergPageSource.getNextPage(IcebergPageSource.java:125)
... 38 more
@findepi looks like the hash collide on metadata $row_id.file
and _file
2147483646=$row_id._file and 2147483646=_file
Here are more observations:
Multiple entries with same key: 2147483646=$row_id._file and 2147483646=_file
alter table metrics.map_making_metrics execute optimize;
After step 4, I can execute UPDATE.
So the workaround is alter table metrics.map_making_metrics execute optimize;
@fritzb Actually, this is a different problem with #14836 . I fix it in #16216 , please cc: @jackye1995
@Heltman @jackye1995
I tried the patch based on nested column fix, and it doesn't seem to work. Could you verify my patch ?
...
+ if (column.isUpdateRowIdColumn() || column.isMergeRowIdColumn()) {
+ field = NestedField.optional(column.getId(), column.getName(), toIcebergType(BOOLEAN, column.getColumnIdentity()));
+ }
+ else {
+ field = NestedField.optional(column.getId(), column.getName(), toIcebergType(column.getType(), column.getColumnIdentity()));
+ }
+ icebergColumns.add(field);
+ }
trino> update metrics.map_making_metrics set tags=null;
Query 20230223_042453_00013_t7pny, FAILED, 6 nodes
Splits: 1,577 total, 0 done (0.00%)
1.62 [1 rows, 13.8KB] [0 rows/s, 8.49KB/s]
Query 20230223_042453_00013_t7pny failed: Multiple entries with same key: 2147483646=$row_id._file and 2147483646=_file
@fritzb please give full stack log
@Heltman
trino> update metrics.map_making_metrics set tags=null;
Query 20230223_174330_02380_t7pny, FAILED, 6 nodes
http://trino.staging.deepmap.ai/ui/query.html?20230223_174330_02380_t7pny
Splits: 1,177 total, 0 done (0.00%)
CPU Time: 0.0s total, 0 rows/s, 0B/s, 80% active
Per Node: 0.0 parallelism, 0 rows/s, 0B/s
Parallelism: 0.0
Peak Memory: 7.26MB
1.21 [0 rows, 0B] [0 rows/s, 0B/s]
Query 20230223_174330_02380_t7pny failed: Multiple entries with same key: 2147483646=$row_id._file and 2147483646=_file
io.trino.spi.TrinoException: Multiple entries with same key: 2147483646=$row_id._file and 2147483646=_file
at io.trino.plugin.iceberg.IcebergPageSource.getNextPage(IcebergPageSource.java:142)
at io.trino.operator.ScanFilterAndProjectOperator$ConnectorPageSourceToPages.process(ScanFilterAndProjectOperator.java:387)
at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:413)
at io.trino.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:262)
at io.trino.operator.WorkProcessorUtils$YieldingProcess.process(WorkProcessorUtils.java:182)
at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:413)
at io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:347)
at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:413)
at io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:347)
at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:413)
at io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:347)
at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:413)
at io.trino.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:262)
at io.trino.operator.WorkProcessorUtils$BlockingProcess.process(WorkProcessorUtils.java:208)
at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:413)
at io.trino.operator.WorkProcessorUtils.lambda$flatten$6(WorkProcessorUtils.java:318)
at io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:360)
at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:413)
at io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:347)
at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:413)
at io.trino.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:262)
at io.trino.operator.WorkProcessorUtils.lambda$processStateMonitor$2(WorkProcessorUtils.java:241)
at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:413)
at io.trino.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:262)
at io.trino.operator.WorkProcessorUtils.lambda$finishWhen$3(WorkProcessorUtils.java:256)
at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:413)
at io.trino.operator.WorkProcessorSourceOperatorAdapter.getOutput(WorkProcessorSourceOperatorAdapter.java:146)
at io.trino.operator.Driver.processInternal(Driver.java:394)
at io.trino.operator.Driver.lambda$process$8(Driver.java:297)
at io.trino.operator.Driver.tryWithLock(Driver.java:689)
at io.trino.operator.Driver.process(Driver.java:289)
at io.trino.operator.Driver.processForDuration(Driver.java:260)
at io.trino.execution.SqlTaskExecution$DriverSplitRunner.processFor(SqlTaskExecution.java:755)
at io.trino.execution.executor.PrioritizedSplitRunner.process(PrioritizedSplitRunner.java:165)
at io.trino.execution.executor.TaskExecutor$TaskRunner.run(TaskExecutor.java:523)
at io.trino.$gen.Trino_406_dirty____20230223_042247_2.run(Unknown Source)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
at java.base/java.lang.Thread.run(Thread.java:833)
Caused by: java.lang.IllegalArgumentException: Multiple entries with same key: 2147483646=$row_id._file and 2147483646=_file
at org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap.conflictException(ImmutableMap.java:377)
at org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap.checkNoConflict(ImmutableMap.java:371)
at org.apache.iceberg.relocated.com.google.common.collect.RegularImmutableMap.checkNoConflictInKeyBucket(RegularImmutableMap.java:241)
at org.apache.iceberg.relocated.com.google.common.collect.RegularImmutableMap.fromEntryArrayCheckingBucketOverflow(RegularImmutableMap.java:132)
at org.apache.iceberg.relocated.com.google.common.collect.RegularImmutableMap.fromEntryArray(RegularImmutableMap.java:94)
at org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap$Builder.build(ImmutableMap.java:573)
at org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap$Builder.buildOrThrow(ImmutableMap.java:601)
at org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap$Builder.build(ImmutableMap.java:588)
at org.apache.iceberg.types.IndexByName.byId(IndexByName.java:81)
at org.apache.iceberg.types.TypeUtil.indexNameById(TypeUtil.java:160)
at org.apache.iceberg.Schema.lazyIdToName(Schema.java:183)
at org.apache.iceberg.Schema.<init>(Schema.java:112)
at org.apache.iceberg.Schema.<init>(Schema.java:91)
at org.apache.iceberg.Schema.<init>(Schema.java:83)
at org.apache.iceberg.Schema.<init>(Schema.java:79)
at io.trino.plugin.iceberg.IcebergUtil.schemaFromHandles(IcebergUtil.java:250)
at io.trino.plugin.iceberg.delete.EqualityDeleteFilter.createPredicate(EqualityDeleteFilter.java:49)
at io.trino.plugin.iceberg.IcebergPageSourceProvider.lambda$createPageSource$7(IcebergPageSourceProvider.java:332)
at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:197)
at java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1625)
at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509)
at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
at java.base/java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:921)
at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
at java.base/java.util.stream.ReferencePipeline.reduce(ReferencePipeline.java:662)
at io.trino.plugin.iceberg.IcebergPageSourceProvider.lambda$createPageSource$8(IcebergPageSourceProvider.java:333)
at com.google.common.base.Suppliers$NonSerializableMemoizingSupplier.get(Suppliers.java:183)
at io.trino.plugin.iceberg.IcebergPageSource.getNextPage(IcebergPageSource.java:125)
... 38 more
@fritzb Can you debug test TestIcebergV2.testUpdateV2tableWithEqualityDelete
? I add this for test this problem. When you breakpoint in field = NestedField.optional(column.getId(), column.getName(), toIcebergType(BOOLEAN, column.getColumnIdentity()));
, you will see what the problem is.
tips: you need start docker server before you run test.
Is there a wiki on how to run Trino in debugging mode (or through Docker)? Sorry, my experience with Trino was limited to patching and rebuilding the Docker image.
@fritzb
run install in idea or Simply run the following command from the project root directory
./mvnw clean install -DskipTests
make sure docker server is started
service docker status
idea code run test
Update from trino CLI failed with error
Multiple entries with same key: 3=$row_id.file_record_count and 3=metric_process_timestamp
. I have Flink writer withequalityFieldColumns
configured.metric_process_timestamp
is atimestamp(6)
type. The table do not have any complex type. It's either timestamp(6), double, int, or string. Trino is 406, Iceberg is 1.0.0, Flink 1.13Here is the Flink snippet code:
Here are the error message: