Skip to content

Conversation

@electrum
Copy link
Member

@electrum electrum commented Mar 14, 2023

Fixes #15372

Release notes

(x) This is not user-visible or docs only and no release notes are required.

@cla-bot cla-bot bot added the cla-signed label Mar 14, 2023
@electrum electrum force-pushed the iceberg-hadoop-v120 branch 2 times, most recently from 2ac9d23 to b4c6785 Compare March 15, 2023 00:49
@github-actions github-actions bot added hive Hive connector hudi Hudi connector iceberg Iceberg connector tests:hive labels Mar 15, 2023
@electrum electrum force-pushed the iceberg-hadoop-v120 branch 3 times, most recently from 94b45ef to 9056908 Compare March 15, 2023 05:43
Copy link
Member

@findepi findepi left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM
when it comes to rest integration I'd defer to @alexjo2144

@electrum electrum force-pushed the iceberg-hadoop-v120 branch from 1f7ee78 to 43d905f Compare March 29, 2023 02:57
@electrum
Copy link
Member Author

The TestIcebergSparkCompatibility test failed with

Cannot determine whether the commit was successful or not, the underlying data files may or may not be needed. Manual intervention via the Remove Orphan Files Action can remove these files when a connection to the Catalog can be re-established if the commit was actually unsuccessful.
Please check to see whether or not your commit was successful before retrying this commit. Retrying an already successful operation will result in duplicate records or unintentional modifications.
At this time no files will be deleted including possibly unused manifest lists.
	at org.apache.iceberg.rest.ErrorHandlers$CommitErrorHandler.accept(ErrorHandlers.java:85)
	at org.apache.iceberg.rest.ErrorHandlers$CommitErrorHandler.accept(ErrorHandlers.java:71)
	at org.apache.iceberg.rest.HTTPClient.throwFailure(HTTPClient.java:177)
	at org.apache.iceberg.rest.HTTPClient.execute(HTTPClient.java:286)
	at org.apache.iceberg.rest.HTTPClient.execute(HTTPClient.java:220)
	at org.apache.iceberg.rest.HTTPClient.post(HTTPClient.java:331)
	at org.apache.iceberg.rest.RESTClient.post(RESTClient.java:112)
	at org.apache.iceberg.rest.RESTTableOperations.commit(RESTTableOperations.java:144)
	at org.apache.iceberg.RemoveSnapshots.lambda$commit$2(RemoveSnapshots.java:307)
	at org.apache.iceberg.util.Tasks$Builder.runTaskWithRetry(Tasks.java:413)
	at org.apache.iceberg.util.Tasks$Builder.runSingleThreaded(Tasks.java:219)
	at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:203)
	at org.apache.iceberg.util.Tasks$Builder.run(Tasks.java:196)
	at org.apache.iceberg.RemoveSnapshots.commit(RemoveSnapshots.java:304)
	at io.trino.plugin.iceberg.IcebergMetadata.executeExpireSnapshots(IcebergMetadata.java:1326)
	at io.trino.plugin.iceberg.IcebergMetadata.executeTableExecute(IcebergMetadata.java:1259)
	at io.trino.plugin.base.classloader.ClassLoaderSafeConnectorMetadata.executeTableExecute(ClassLoaderSafeConnectorMetadata.java:208)
	at io.trino.metadata.MetadataManager.executeTableExecute(MetadataManager.java:358)
	at io.trino.operator.SimpleTableExecuteOperator.getOutput(SimpleTableExecuteOperator.java:128)
	at io.trino.operator.Driver.processInternal(Driver.java:402)
	at io.trino.operator.Driver.lambda$process$8(Driver.java:305)
	at io.trino.operator.Driver.tryWithLock(Driver.java:701)
	at io.trino.operator.Driver.process(Driver.java:297)
	at io.trino.operator.Driver.processForDuration(Driver.java:268)
	at io.trino.execution.SqlTaskExecution$DriverSplitRunner.processFor(SqlTaskExecution.java:845)
	at io.trino.execution.executor.PrioritizedSplitRunner.process(PrioritizedSplitRunner.java:165)
	at io.trino.execution.executor.TaskExecutor$TaskRunner.run(TaskExecutor.java:537)
	at io.trino.$gen.Trino_410_265_g372c594____20230328_053742_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: org.apache.iceberg.exceptions.ServiceFailureException: Service failed: 500: {
"cause1":"java.lang.IllegalArgumentException: Cannot parse to an integer value: snapshot-id: 2168225879919926620",
"servlet":"org.apache.iceberg.rest.RESTCatalogServlet-3bbc39f8",
"cause0":"com.fasterxml.jackson.databind.JsonMappingException: Cannot parse to an integer value: snapshot-id: 2168225879919926620 (through reference chain: org.apache.iceberg.rest.requests.UpdateTableRequest["updates"]->java.util.ArrayList[2])",
"message":"com.fasterxml.jackson.databind.JsonMappingException: Cannot parse to an integer value: snapshot-id: 2168225879919926620 (through reference chain: org.apache.iceberg.rest.requests.UpdateTableRequest["updates"]->java.util.ArrayList[2])",
"url":"/v1/namespaces/default/tables/test_spark_reads_trino_partitioned_table_after_expiring_snapshotsparquet",
"status":"500"
}
	... 31 more

I'm going to try reverting the REST server container update.

@electrum
Copy link
Member Author

I'm guessing we need to update the Iceberg version for Spark as well: trinodb/docker-images#164

This seems to indicate that data written with Iceberg 1.2 is not readable by Iceberg 1.1, which is concerning.

@electrum electrum force-pushed the iceberg-hadoop-v120 branch from 43d905f to 15a2742 Compare March 29, 2023 06:37
@nastra
Copy link
Contributor

nastra commented Mar 29, 2023

I looked at the failures in TestIcebergSparkCompatibility. It seems that all of the cleanup tests are affected that call RemoveSnapshots. The underlying issue is that the snapshot-id is being parsed as an integer rather than a long here.
I've opened apache/iceberg#7235 to address that.

/cc @findepi

@electrum
Copy link
Member Author

Nice find. Does this mean that Iceberg 1.2.0 is unusable until this is fixed? Do we need to wait for 1.2.1 to upgrade?

@electrum
Copy link
Member Author

Hmm, it appears that this commit went into Iceberg 1.1.0. Any idea why it didn't break before? Or thoughts on a workaround?

@nastra
Copy link
Contributor

nastra commented Mar 29, 2023

Nice find. Does this mean that Iceberg 1.2.0 is unusable until this is fixed? Do we need to wait for 1.2.1 to upgrade?

I think this is only an issue with the removal of statistics files, but I don't know why it wasn't an issue with Iceberg 1.1.0. @findepi any ideas?

@nastra
Copy link
Contributor

nastra commented Mar 29, 2023

We might get a fix for this into 1.2.1

@electrum
Copy link
Member Author

@nastra @rdblue Any idea on when 1.2.1 could be released?

I'm wondering if snapshot ID generation was changed, such that IDs were previously valid 32-bit integers.

@danielcweeks
Copy link
Contributor

@electrum We're working on pulling together the 1.2.1 release (there's a small set of issues, so hopefully we'll have a quick turnaround for you). I believe the fix for this is already in mainline, so it'll be included in 1.2.1.

@nastra nastra mentioned this pull request Apr 6, 2023
@nastra
Copy link
Contributor

nastra commented Apr 12, 2023

@electrum Iceberg 1.2.1 has been officially released now. I've also created a new version of the REST server image: tabulario/iceberg-rest:0.4.0

@electrum electrum force-pushed the iceberg-hadoop-v120 branch from 15a2742 to d7376fd Compare April 18, 2023 21:20
@electrum electrum changed the title Update to Iceberg 1.2.0 Update to Iceberg 1.2.1 Apr 18, 2023
@electrum electrum force-pushed the iceberg-hadoop-v120 branch from d7376fd to 8626ad8 Compare April 18, 2023 21:21
@electrum
Copy link
Member Author

@nastra Thanks, updated. Hopefully all the tests pass this time, then I'll finally merge this

@electrum electrum merged commit e94db4f into trinodb:master Apr 19, 2023
@electrum electrum deleted the iceberg-hadoop-v120 branch April 19, 2023 01:43
@github-actions github-actions bot added this to the 414 milestone Apr 19, 2023
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

cla-signed hive Hive connector hudi Hudi connector iceberg Iceberg connector

Development

Successfully merging this pull request may close these issues.

Remove redundant Iceberg NPE workaround

6 participants