Skip to content

Commit 5488360

Browse files
Add support for SHOW CREATE SCHEMA
1 parent 09fd893 commit 5488360

File tree

19 files changed

+212
-6
lines changed

19 files changed

+212
-6
lines changed

presto-docs/src/main/sphinx/sql.rst

+1
Original file line numberDiff line numberDiff line change
@@ -47,6 +47,7 @@ This chapter describes the SQL syntax used in Presto.
4747
sql/show-catalogs
4848
sql/show-columns
4949
sql/show-create-function
50+
sql/show-create-schema
5051
sql/show-create-table
5152
sql/show-create-view
5253
sql/show-functions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,37 @@
1+
=================
2+
SHOW CREATE SCHEMA
3+
=================
4+
5+
Synopsis
6+
--------
7+
8+
.. code-block:: none
9+
10+
SHOW CREATE SCHEMA schema_name
11+
12+
Description
13+
-----------
14+
15+
Show the SQL statement that creates the specified schema.
16+
17+
Examples
18+
--------
19+
20+
Show the SQL that can be run to create the ``sf1`` schema::
21+
22+
SHOW CREATE SCHEMA hive.sf1;
23+
24+
.. code-block:: none
25+
26+
Create Schema
27+
-----------------------------------------
28+
CREATE SCHEMA hive.sf1
29+
WITH (
30+
location = 'hdfs://localhost:9000/user/hive/warehouse/sf1.db'
31+
)
32+
(1 row)
33+
34+
See Also
35+
--------
36+
37+
:doc:`create-schema`

presto-hive-metastore/src/main/java/com/facebook/presto/hive/SchemaProperties.java

+9
Original file line numberDiff line numberDiff line change
@@ -13,8 +13,10 @@
1313
*/
1414
package com.facebook.presto.hive;
1515

16+
import com.facebook.presto.hive.metastore.Database;
1617
import com.facebook.presto.spi.session.PropertyMetadata;
1718
import com.google.common.collect.ImmutableList;
19+
import com.google.common.collect.ImmutableMap;
1820

1921
import java.util.List;
2022
import java.util.Map;
@@ -39,4 +41,11 @@ public static Optional<String> getLocation(Map<String, Object> schemaProperties)
3941
{
4042
return Optional.ofNullable((String) schemaProperties.get(LOCATION_PROPERTY));
4143
}
44+
45+
public static Map<String, Object> getDatabaseProperties(Database database)
46+
{
47+
ImmutableMap.Builder<String, Object> result = ImmutableMap.builder();
48+
database.getLocation().ifPresent(location -> result.put(LOCATION_PROPERTY, location));
49+
return result.build();
50+
}
4251
}

presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadata.java

+14
Original file line numberDiff line numberDiff line change
@@ -15,6 +15,7 @@
1515

1616
import com.facebook.airlift.json.JsonCodec;
1717
import com.facebook.airlift.json.smile.SmileCodec;
18+
import com.facebook.presto.common.CatalogSchemaName;
1819
import com.facebook.presto.common.Subfield;
1920
import com.facebook.presto.common.predicate.NullableValue;
2021
import com.facebook.presto.common.predicate.TupleDomain;
@@ -65,6 +66,7 @@
6566
import com.facebook.presto.spi.PrestoException;
6667
import com.facebook.presto.spi.QueryId;
6768
import com.facebook.presto.spi.RecordCursor;
69+
import com.facebook.presto.spi.SchemaNotFoundException;
6870
import com.facebook.presto.spi.SchemaTableName;
6971
import com.facebook.presto.spi.SchemaTablePrefix;
7072
import com.facebook.presto.spi.SortingProperty;
@@ -294,6 +296,7 @@
294296
import static com.facebook.presto.hive.PartitionUpdate.UpdateMode.APPEND;
295297
import static com.facebook.presto.hive.PartitionUpdate.UpdateMode.NEW;
296298
import static com.facebook.presto.hive.PartitionUpdate.UpdateMode.OVERWRITE;
299+
import static com.facebook.presto.hive.SchemaProperties.getDatabaseProperties;
297300
import static com.facebook.presto.hive.metastore.HivePrivilegeInfo.toHivePrivilege;
298301
import static com.facebook.presto.hive.metastore.MetastoreUtil.AVRO_SCHEMA_URL_KEY;
299302
import static com.facebook.presto.hive.metastore.MetastoreUtil.PRESTO_MATERIALIZED_VIEW_FLAG;
@@ -502,6 +505,17 @@ public List<String> listSchemaNames(ConnectorSession session)
502505
return metastore.getAllDatabases(getMetastoreContext(session));
503506
}
504507

508+
@Override
509+
public Map<String, Object> getSchemaProperties(ConnectorSession session, CatalogSchemaName schemaName)
510+
{
511+
MetastoreContext metastoreContext = getMetastoreContext(session);
512+
Optional<Database> database = metastore.getDatabase(metastoreContext, schemaName.getSchemaName());
513+
if (database.isPresent()) {
514+
return getDatabaseProperties(database.get());
515+
}
516+
throw new SchemaNotFoundException(schemaName.getSchemaName());
517+
}
518+
505519
@Override
506520
public HiveTableHandle getTableHandle(ConnectorSession session, SchemaTableName tableName)
507521
{

presto-hive/src/test/java/com/facebook/presto/hive/TestHiveIntegrationSmokeTest.java

+18
Original file line numberDiff line numberDiff line change
@@ -2781,6 +2781,24 @@ public void testShowCreateTable()
27812781
actualResult = computeActual("SHOW CREATE TABLE \"test_show_create_table'2\"");
27822782
assertEquals(getOnlyElement(actualResult.getOnlyColumnAsSet()), createTableSql);
27832783
}
2784+
@Test
2785+
public void testShowCreateSchema()
2786+
{
2787+
String createSchemaSql = "CREATE SCHEMA show_create_hive_schema";
2788+
assertUpdate(createSchemaSql);
2789+
String expectedShowCreateSchema = "CREATE SCHEMA show_create_hive_schema\n" +
2790+
"WITH (\n" +
2791+
" location = '.*show_create_hive_schema'\n" +
2792+
")";
2793+
2794+
MaterializedResult actualResult = computeActual("SHOW CREATE SCHEMA show_create_hive_schema");
2795+
assertThat(getOnlyElement(actualResult.getOnlyColumnAsSet()).toString().matches(expectedShowCreateSchema));
2796+
2797+
assertQueryFails(format("SHOW CREATE SCHEMA %s.%s", getSession().getCatalog().get(), ""), ".*mismatched input '.'. Expecting: <EOF>");
2798+
assertQueryFails(format("SHOW CREATE SCHEMA %s.%s.%s", getSession().getCatalog().get(), "show_create_hive_schema", "tabletest"), ".*Too many parts in schema name: hive.show_create_hive_schema.tabletest");
2799+
assertQueryFails(format("SHOW CREATE SCHEMA %s", "schema_not_exist"), ".*Schema 'hive.schema_not_exist' does not exist");
2800+
assertUpdate("DROP SCHEMA show_create_hive_schema");
2801+
}
27842802

27852803
@Test
27862804
public void testTextfileAmbiguousTimestamp()

presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergHiveMetadata.java

+13
Original file line numberDiff line numberDiff line change
@@ -14,6 +14,7 @@
1414
package com.facebook.presto.iceberg;
1515

1616
import com.facebook.airlift.json.JsonCodec;
17+
import com.facebook.presto.common.CatalogSchemaName;
1718
import com.facebook.presto.common.type.Type;
1819
import com.facebook.presto.common.type.TypeManager;
1920
import com.facebook.presto.hive.HdfsContext;
@@ -96,6 +97,7 @@
9697
import static com.facebook.presto.hive.HiveUtil.decodeViewData;
9798
import static com.facebook.presto.hive.HiveUtil.encodeViewData;
9899
import static com.facebook.presto.hive.HiveUtil.hiveColumnHandles;
100+
import static com.facebook.presto.hive.SchemaProperties.getDatabaseProperties;
99101
import static com.facebook.presto.hive.SchemaProperties.getLocation;
100102
import static com.facebook.presto.hive.metastore.HivePrivilegeInfo.HivePrivilege.DELETE;
101103
import static com.facebook.presto.hive.metastore.HivePrivilegeInfo.HivePrivilege.INSERT;
@@ -232,6 +234,17 @@ public List<String> listSchemaNames(ConnectorSession session)
232234
return metastore.getAllDatabases(getMetastoreContext(session));
233235
}
234236

237+
@Override
238+
public Map<String, Object> getSchemaProperties(ConnectorSession session, CatalogSchemaName schemaName)
239+
{
240+
MetastoreContext metastoreContext = getMetastoreContext(session);
241+
Optional<Database> database = metastore.getDatabase(metastoreContext, schemaName.getSchemaName());
242+
if (database.isPresent()) {
243+
return getDatabaseProperties(database.get());
244+
}
245+
throw new SchemaNotFoundException(schemaName.getSchemaName());
246+
}
247+
235248
@Override
236249
public List<SchemaTableName> listTables(ConnectorSession session, Optional<String> schemaName)
237250
{

presto-iceberg/src/test/java/com/facebook/presto/iceberg/hive/TestIcebergSmokeHive.java

+23
Original file line numberDiff line numberDiff line change
@@ -28,17 +28,21 @@
2828
import com.facebook.presto.iceberg.IcebergUtil;
2929
import com.facebook.presto.spi.ConnectorSession;
3030
import com.facebook.presto.spi.SchemaTableName;
31+
import com.facebook.presto.testing.MaterializedResult;
3132
import com.facebook.presto.tests.DistributedQueryRunner;
3233
import com.google.common.collect.ImmutableSet;
3334
import org.apache.iceberg.Table;
35+
import org.testng.annotations.Test;
3436

3537
import java.io.File;
3638
import java.nio.file.Path;
3739

3840
import static com.facebook.presto.hive.metastore.InMemoryCachingHiveMetastore.memoizeMetastore;
3941
import static com.facebook.presto.iceberg.CatalogType.HIVE;
4042
import static com.facebook.presto.iceberg.IcebergQueryRunner.getIcebergDataDirectoryPath;
43+
import static com.google.common.collect.Iterables.getOnlyElement;
4144
import static java.lang.String.format;
45+
import static org.assertj.core.api.Assertions.assertThat;
4246

4347
public class TestIcebergSmokeHive
4448
extends IcebergDistributedSmokeTestBase
@@ -83,4 +87,23 @@ protected Table getIcebergTable(ConnectorSession session, String schema, String
8387
session,
8488
SchemaTableName.valueOf(schema + "." + tableName));
8589
}
90+
91+
@Test
92+
public void testShowCreateSchema()
93+
{
94+
String createSchemaSql = "CREATE SCHEMA show_create_iceberg_schema";
95+
assertUpdate(createSchemaSql);
96+
String expectedShowCreateSchema = "CREATE SCHEMA show_create_iceberg_schema\n" +
97+
"WITH (\n" +
98+
" location = '.*show_create_iceberg_schema'\n" +
99+
")";
100+
101+
MaterializedResult actualResult = computeActual("SHOW CREATE SCHEMA show_create_iceberg_schema");
102+
assertThat(getOnlyElement(actualResult.getOnlyColumnAsSet()).toString().matches(expectedShowCreateSchema));
103+
104+
assertQueryFails(format("SHOW CREATE SCHEMA %s.%s", getSession().getCatalog().get(), ""), ".*mismatched input '.'. Expecting: <EOF>");
105+
assertQueryFails(format("SHOW CREATE SCHEMA %s.%s.%s", getSession().getCatalog().get(), "show_create_iceberg_schema", "tabletest"), ".*Too many parts in schema name: iceberg.show_create_iceberg_schema.tabletest");
106+
assertQueryFails(format("SHOW CREATE SCHEMA %s", "schema_not_exist"), ".*Schema 'iceberg.schema_not_exist' does not exist");
107+
assertUpdate("DROP SCHEMA show_create_iceberg_schema");
108+
}
86109
}

presto-main/src/main/java/com/facebook/presto/metadata/DelegatingMetadataManager.java

+6
Original file line numberDiff line numberDiff line change
@@ -95,6 +95,12 @@ public List<String> listSchemaNames(Session session, String catalogName)
9595
return delegate.listSchemaNames(session, catalogName);
9696
}
9797

98+
@Override
99+
public Map<String, Object> getSchemaProperties(Session session, CatalogSchemaName schemaName)
100+
{
101+
return delegate.getSchemaProperties(session, schemaName);
102+
}
103+
98104
@Override
99105
public Optional<SystemTable> getSystemTable(Session session, QualifiedObjectName tableName)
100106
{

presto-main/src/main/java/com/facebook/presto/metadata/Metadata.java

+5
Original file line numberDiff line numberDiff line change
@@ -74,6 +74,11 @@ public interface Metadata
7474

7575
List<String> listSchemaNames(Session session, String catalogName);
7676

77+
/**
78+
* Gets the schema properties for the specified schema.
79+
*/
80+
Map<String, Object> getSchemaProperties(Session session, CatalogSchemaName schemaName);
81+
7782
Optional<SystemTable> getSystemTable(Session session, QualifiedObjectName tableName);
7883

7984
/**

presto-main/src/main/java/com/facebook/presto/metadata/MetadataManager.java

+17
Original file line numberDiff line numberDiff line change
@@ -73,6 +73,7 @@
7373
import com.facebook.presto.spi.statistics.TableStatisticsMetadata;
7474
import com.facebook.presto.sql.analyzer.FeaturesConfig;
7575
import com.facebook.presto.sql.analyzer.FunctionsConfig;
76+
import com.facebook.presto.sql.analyzer.SemanticException;
7677
import com.facebook.presto.sql.analyzer.TypeSignatureProvider;
7778
import com.facebook.presto.transaction.TransactionManager;
7879
import com.facebook.presto.type.TypeDeserializer;
@@ -129,6 +130,7 @@
129130
import static com.facebook.presto.spi.StandardErrorCode.SYNTAX_ERROR;
130131
import static com.facebook.presto.spi.TableLayoutFilterCoverage.NOT_APPLICABLE;
131132
import static com.facebook.presto.spi.analyzer.ViewDefinition.ViewColumn;
133+
import static com.facebook.presto.sql.analyzer.SemanticErrorCode.MISSING_SCHEMA;
132134
import static com.facebook.presto.sql.analyzer.TypeSignatureProvider.fromTypes;
133135
import static com.facebook.presto.transaction.InMemoryTransactionManager.createTestTransactionManager;
134136
import static com.google.common.base.Preconditions.checkArgument;
@@ -318,6 +320,21 @@ public List<String> listSchemaNames(Session session, String catalogName)
318320
return ImmutableList.copyOf(schemaNames.build());
319321
}
320322

323+
@Override
324+
public Map<String, Object> getSchemaProperties(Session session, CatalogSchemaName schemaName)
325+
{
326+
if (!getMetadataResolver(session).schemaExists(schemaName)) {
327+
throw new SemanticException(MISSING_SCHEMA, format("Schema '%s' does not exist", schemaName));
328+
}
329+
330+
Optional<CatalogMetadata> catalog = getOptionalCatalogMetadata(session, transactionManager, schemaName.getCatalogName());
331+
CatalogMetadata catalogMetadata = catalog.get();
332+
ConnectorSession connectorSession = session.toConnectorSession(catalogMetadata.getConnectorId());
333+
ConnectorMetadata metadata = catalogMetadata.getMetadataFor(catalogMetadata.getConnectorId());
334+
335+
return metadata.getSchemaProperties(connectorSession, schemaName);
336+
}
337+
321338
@Override
322339
public Optional<TableHandle> getTableHandleForStatisticsCollection(Session session, QualifiedObjectName table, Map<String, Object> analyzeProperties)
323340
{

presto-main/src/main/java/com/facebook/presto/sql/rewrite/ShowQueriesRewrite.java

+25-6
Original file line numberDiff line numberDiff line change
@@ -53,6 +53,7 @@
5353
import com.facebook.presto.sql.tree.ConstraintSpecification;
5454
import com.facebook.presto.sql.tree.CreateFunction;
5555
import com.facebook.presto.sql.tree.CreateMaterializedView;
56+
import com.facebook.presto.sql.tree.CreateSchema;
5657
import com.facebook.presto.sql.tree.CreateTable;
5758
import com.facebook.presto.sql.tree.CreateView;
5859
import com.facebook.presto.sql.tree.DoubleLiteral;
@@ -113,10 +114,12 @@
113114
import static com.facebook.presto.metadata.MetadataUtil.createCatalogSchemaName;
114115
import static com.facebook.presto.metadata.MetadataUtil.createQualifiedName;
115116
import static com.facebook.presto.metadata.MetadataUtil.createQualifiedObjectName;
117+
import static com.facebook.presto.metadata.MetadataUtil.getConnectorIdOrThrow;
116118
import static com.facebook.presto.metadata.SessionFunctionHandle.SESSION_NAMESPACE;
117119
import static com.facebook.presto.spi.StandardErrorCode.FUNCTION_NOT_FOUND;
118120
import static com.facebook.presto.spi.StandardErrorCode.GENERIC_USER_ERROR;
119121
import static com.facebook.presto.spi.StandardErrorCode.INVALID_COLUMN_PROPERTY;
122+
import static com.facebook.presto.spi.StandardErrorCode.INVALID_SCHEMA_PROPERTY;
120123
import static com.facebook.presto.spi.StandardErrorCode.INVALID_TABLE_PROPERTY;
121124
import static com.facebook.presto.sql.ExpressionUtils.combineConjuncts;
122125
import static com.facebook.presto.sql.QueryUtil.aliased;
@@ -151,6 +154,7 @@
151154
import static com.facebook.presto.sql.tree.RoutineCharacteristics.Language;
152155
import static com.facebook.presto.sql.tree.RoutineCharacteristics.NullCallClause;
153156
import static com.facebook.presto.sql.tree.ShowCreate.Type.MATERIALIZED_VIEW;
157+
import static com.facebook.presto.sql.tree.ShowCreate.Type.SCHEMA;
154158
import static com.facebook.presto.sql.tree.ShowCreate.Type.TABLE;
155159
import static com.facebook.presto.sql.tree.ShowCreate.Type.VIEW;
156160
import static com.facebook.presto.util.AnalyzerUtil.createParsingOptions;
@@ -456,6 +460,22 @@ private static Expression toExpression(Object value)
456460
@Override
457461
protected Node visitShowCreate(ShowCreate node, Void context)
458462
{
463+
if (node.getType() == SCHEMA) {
464+
CatalogSchemaName catalogSchemaName = createCatalogSchemaName(session, node, Optional.of(node.getName()));
465+
if (!metadataResolver.schemaExists(catalogSchemaName)) {
466+
throw new SemanticException(MISSING_SCHEMA, node, "Schema '%s' does not exist", catalogSchemaName);
467+
}
468+
469+
Map<String, Object> properties = metadata.getSchemaProperties(session, catalogSchemaName);
470+
Map<String, PropertyMetadata<?>> allSchemaProperties = metadata.getSchemaPropertyManager().getAllProperties().get(getConnectorIdOrThrow(session, metadata, catalogSchemaName.getCatalogName()));
471+
List<Property> propertyNodes = buildProperties("schema " + catalogSchemaName, INVALID_SCHEMA_PROPERTY, properties, allSchemaProperties);
472+
CreateSchema createSchema = new CreateSchema(
473+
node.getName(),
474+
false,
475+
propertyNodes);
476+
return singleValueQuery("Create Schema", formatSql(createSchema, Optional.of(parameters)).trim());
477+
}
478+
459479
QualifiedObjectName objectName = createQualifiedObjectName(session, node, node.getName());
460480
Optional<ViewDefinition> viewDefinition = metadataResolver.getView(objectName);
461481
Optional<MaterializedViewDefinition> materializedViewDefinition = metadataResolver.getMaterializedView(objectName);
@@ -495,7 +515,7 @@ protected Node visitShowCreate(ShowCreate node, Void context)
495515
ConnectorTableMetadata connectorTableMetadata = metadata.getTableMetadata(session, tableHandle.get()).getMetadata();
496516
Map<String, Object> properties = connectorTableMetadata.getProperties();
497517
Map<String, PropertyMetadata<?>> allTableProperties = metadata.getTablePropertyManager().getAllProperties().get(tableHandle.get().getConnectorId());
498-
List<Property> propertyNodes = buildProperties(objectName, Optional.empty(), INVALID_TABLE_PROPERTY, properties, allTableProperties);
518+
List<Property> propertyNodes = buildProperties("materialized view " + objectName, INVALID_TABLE_PROPERTY, properties, allTableProperties);
499519

500520
CreateMaterializedView createMaterializedView = new CreateMaterializedView(
501521
Optional.empty(),
@@ -532,7 +552,7 @@ protected Node visitShowCreate(ShowCreate node, Void context)
532552
List<TableElement> columns = connectorTableMetadata.getColumns().stream()
533553
.filter(column -> !column.isHidden())
534554
.map(column -> {
535-
List<Property> propertyNodes = buildProperties(objectName, Optional.of(column.getName()), INVALID_COLUMN_PROPERTY, column.getProperties(), allColumnProperties);
555+
List<Property> propertyNodes = buildProperties(toQualifiedName(objectName, Optional.of(column.getName())), INVALID_COLUMN_PROPERTY, column.getProperties(), allColumnProperties);
536556
return new ColumnDefinition(
537557
QueryUtil.quotedIdentifier(column.getName()),
538558
column.getType().getDisplayName(),
@@ -544,7 +564,7 @@ protected Node visitShowCreate(ShowCreate node, Void context)
544564

545565
Map<String, Object> properties = connectorTableMetadata.getProperties();
546566
Map<String, PropertyMetadata<?>> allTableProperties = metadata.getTablePropertyManager().getAllProperties().get(tableHandle.get().getConnectorId());
547-
List<Property> propertyNodes = buildProperties(objectName, Optional.empty(), INVALID_TABLE_PROPERTY, properties, allTableProperties);
567+
List<Property> propertyNodes = buildProperties("table " + objectName, INVALID_TABLE_PROPERTY, properties, allTableProperties);
548568

549569
columns.addAll(connectorTableMetadata.getTableConstraintsHolder().getTableConstraints()
550570
.stream()
@@ -646,7 +666,6 @@ protected Node visitShowCreateFunction(ShowCreateFunction node, Void context)
646666

647667
private List<Property> buildProperties(
648668
Object objectName,
649-
Optional<String> columnName,
650669
StandardErrorCode errorCode,
651670
Map<String, Object> properties,
652671
Map<String, PropertyMetadata<?>> allProperties)
@@ -661,15 +680,15 @@ private List<Property> buildProperties(
661680
String propertyName = propertyEntry.getKey();
662681
Object value = propertyEntry.getValue();
663682
if (value == null) {
664-
throw new PrestoException(errorCode, format("Property %s for %s cannot have a null value", propertyName, toQualifiedName(objectName, columnName)));
683+
throw new PrestoException(errorCode, format("Property %s for %s cannot have a null value", propertyName, objectName));
665684
}
666685

667686
PropertyMetadata<?> property = allProperties.get(propertyName);
668687
if (!Primitives.wrap(property.getJavaType()).isInstance(value)) {
669688
throw new PrestoException(errorCode, format(
670689
"Property %s for %s should have value of type %s, not %s",
671690
propertyName,
672-
toQualifiedName(objectName, columnName),
691+
objectName,
673692
property.getJavaType().getName(),
674693
value.getClass().getName()));
675694
}

0 commit comments

Comments
 (0)