Skip to content
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

[Iceberg] SELECT fails after nested field schema update with Parquet format #22922

Closed
sshkvar opened this issue Aug 2, 2024 · 0 comments · Fixed by #24243
Closed

[Iceberg] SELECT fails after nested field schema update with Parquet format #22922

sshkvar opened this issue Aug 2, 2024 · 0 comments · Fixed by #24243
Assignees
Labels
bug Something isn't working iceberg Iceberg connector

Comments

@sshkvar
Copy link
Contributor

sshkvar commented Aug 2, 2024

Hi,

We have a table with field top_level array(ROW(a varchar, b varchar))
When change type of nested ROW to top_level array(ROW(c varchar)) we can't query this field

Tested on Trino 453 and latest master

Query fails with:

Caused by: io.trino.spi.TrinoException: Failed to read Parquet file: local:///test_schema/test_table_schema-b9cc55871a7f425fb7388f5f59a38050/data/20240802_143046_00008_ayge4-3f8d3cd1-fb4a-4b17-a647-1f49a3b0ecc9.parquet
	at io.trino.plugin.iceberg.IcebergPageSourceProvider.handleException(IcebergPageSourceProvider.java:1459)
	at io.trino.plugin.iceberg.IcebergPageSourceProvider.lambda$createParquetPageSource$26(IcebergPageSourceProvider.java:1024)
	at io.trino.parquet.reader.ParquetBlockFactory$ParquetBlockLoader.load(ParquetBlockFactory.java:75)
	at io.trino.spi.block.LazyBlock$LazyData.load(LazyBlock.java:312)
	at io.trino.spi.block.LazyBlock$LazyData.getFullyLoadedBlock(LazyBlock.java:291)
	at io.trino.spi.block.LazyBlock.getLoadedBlock(LazyBlock.java:186)
	at io.trino.spi.Page.getLoadedPage(Page.java:238)
	at io.trino.operator.TableScanOperator.getOutput(TableScanOperator.java:271)
	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:191)
	at io.trino.$gen.Trino_testversion____20240802_143035_71.run(Unknown Source)
	at io.trino.execution.executor.dedicated.TaskEntry$VersionEmbedderBridge.run(TaskEntry.java:192)
	at io.trino.execution.executor.scheduler.FairScheduler.runTask(FairScheduler.java:168)
	at io.trino.execution.executor.scheduler.FairScheduler.lambda$submit$0(FairScheduler.java:155)
	at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:572)
	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.util.NoSuchElementException: No value present
	at java.base/java.util.Optional.get(Optional.java:143)
	at io.trino.parquet.reader.ParquetReader.readArray(ParquetReader.java:340)
	at io.trino.parquet.reader.ParquetReader.readColumnChunk(ParquetReader.java:540)
	at io.trino.parquet.reader.ParquetReader.readBlock(ParquetReader.java:526)
	at io.trino.parquet.reader.ParquetReader.lambda$nextPage$3(ParquetReader.java:252)
	at io.trino.parquet.reader.ParquetBlockFactory$ParquetBlockLoader.load(ParquetBlockFactory.java:72)
	... 24 more

Test to reproduce issue:

    @Test
    public void testReadTableAfterSchemaUpdate()
    {
        assertQuerySucceeds("CREATE TABLE iceberg.test_schema.test_table_schema (top_level array(ROW(a varchar, b varchar)))");

        assertQuerySucceeds("INSERT INTO iceberg.test_schema.test_table_schema select CAST(array[ROW('a', 'b')] AS array(ROW(a varchar, b varchar)))");

        assertQuerySucceeds("alter table iceberg.test_schema.test_table_schema drop column top_level.element.a");
        assertQuerySucceeds("alter table iceberg.test_schema.test_table_schema add column top_level.element.c varchar");
        assertQuerySucceeds("alter table iceberg.test_schema.test_table_schema drop column top_level.element.b");

        assertQuerySucceeds("INSERT INTO iceberg.test_schema.test_table_schema select CAST(array[ROW('c')] AS array(ROW(c varchar)))");

        assertQuerySucceeds("SELECT * from iceberg.test_schema.test_table_schema ");
    }
@findinpath findinpath added bug Something isn't working iceberg Iceberg connector labels Aug 5, 2024
@ebyhr ebyhr self-assigned this Aug 15, 2024
@ebyhr ebyhr changed the title [Iceberg] SELECT fails after nested field schema update [Iceberg] SELECT fails after nested field schema update with Parquet format Aug 15, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something isn't working iceberg Iceberg connector
Development

Successfully merging a pull request may close this issue.

3 participants