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

SHOW CREATE SCHEMA on Glue fails when the database contains parameters #24744

Open
ebyhr opened this issue Jan 20, 2025 · 0 comments
Open

SHOW CREATE SCHEMA on Glue fails when the database contains parameters #24744

ebyhr opened this issue Jan 20, 2025 · 0 comments
Labels
iceberg Iceberg connector

Comments

@ebyhr
Copy link
Member

ebyhr commented Jan 20, 2025

aws glue create-database --database-input "{\"Name\":\"tempdb\", \"Parameters\":{\"string\": \"string\"}}"
trino> SHOW CREATE SCHEMA iceberg.tempdb;
Query 20250120_064618_00002_9svff failed: No PropertyMetadata for property: string
io.trino.spi.TrinoException: No PropertyMetadata for property: string
	at io.trino.metadata.PropertyUtil.lambda$toSqlProperties$0(PropertyUtil.java:197)
	at com.google.common.collect.SingletonImmutableBiMap.forEach(SingletonImmutableBiMap.java:71)
	at io.trino.metadata.PropertyUtil.toSqlProperties(PropertyUtil.java:190)
	at io.trino.sql.rewrite.ShowQueriesRewrite$Visitor.showCreateSchema(ShowQueriesRewrite.java:674)
	at io.trino.sql.rewrite.ShowQueriesRewrite$Visitor.visitShowCreate(ShowQueriesRewrite.java:520)
	at io.trino.sql.rewrite.ShowQueriesRewrite$Visitor.visitShowCreate(ShowQueriesRewrite.java:219)
	at io.trino.sql.tree.ShowCreate.accept(ShowCreate.java:60)
	at io.trino.sql.tree.AstVisitor.process(AstVisitor.java:27)
	at io.trino.sql.rewrite.ShowQueriesRewrite.rewrite(ShowQueriesRewrite.java:216)
	at io.trino.sql.rewrite.StatementRewrite.rewrite(StatementRewrite.java:54)
	at io.trino.sql.analyzer.Analyzer.analyze(Analyzer.java:93)
	at io.trino.sql.analyzer.Analyzer.analyze(Analyzer.java:87)
	at io.trino.execution.SqlQueryExecution.analyze(SqlQueryExecution.java:289)
	at io.trino.execution.SqlQueryExecution.<init>(SqlQueryExecution.java:222)
	at io.trino.execution.SqlQueryExecution$SqlQueryExecutionFactory.createQueryExecution(SqlQueryExecution.java:892)
	at io.trino.dispatcher.LocalDispatchQueryFactory.lambda$createDispatchQuery$0(LocalDispatchQueryFactory.java:153)
	at io.trino.$gen.Trino_testversion____20250120_064244_71.call(Unknown Source)
	at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:131)
	at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:75)
	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:1575)

The cause is the following. We should check all catalogs (HMS, REST, JDBC, Snowflake, Nessie)

if (database.getParameters() != null) {
metadata.putAll(database.getParameters());
}

@ebyhr ebyhr added the iceberg Iceberg connector label Jan 20, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
iceberg Iceberg connector
Development

No branches or pull requests

1 participant