Skip to content

Commit

Permalink
Add support for SHOW CREATE SCHEMA
Browse files Browse the repository at this point in the history
  • Loading branch information
agrawalreetika committed Jan 31, 2025
1 parent 09fd893 commit 5488360
Show file tree
Hide file tree
Showing 19 changed files with 212 additions and 6 deletions.
1 change: 1 addition & 0 deletions presto-docs/src/main/sphinx/sql.rst
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@ This chapter describes the SQL syntax used in Presto.
sql/show-catalogs
sql/show-columns
sql/show-create-function
sql/show-create-schema
sql/show-create-table
sql/show-create-view
sql/show-functions
Expand Down
37 changes: 37 additions & 0 deletions presto-docs/src/main/sphinx/sql/show-create-schema.rst
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
=================
SHOW CREATE SCHEMA
=================

Synopsis
--------

.. code-block:: none
SHOW CREATE SCHEMA schema_name
Description
-----------

Show the SQL statement that creates the specified schema.

Examples
--------

Show the SQL that can be run to create the ``sf1`` schema::

SHOW CREATE SCHEMA hive.sf1;

.. code-block:: none
Create Schema
-----------------------------------------
CREATE SCHEMA hive.sf1
WITH (
location = 'hdfs://localhost:9000/user/hive/warehouse/sf1.db'
)
(1 row)
See Also
--------

:doc:`create-schema`
Original file line number Diff line number Diff line change
Expand Up @@ -13,8 +13,10 @@
*/
package com.facebook.presto.hive;

import com.facebook.presto.hive.metastore.Database;
import com.facebook.presto.spi.session.PropertyMetadata;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;

import java.util.List;
import java.util.Map;
Expand All @@ -39,4 +41,11 @@ public static Optional<String> getLocation(Map<String, Object> schemaProperties)
{
return Optional.ofNullable((String) schemaProperties.get(LOCATION_PROPERTY));
}

public static Map<String, Object> getDatabaseProperties(Database database)
{
ImmutableMap.Builder<String, Object> result = ImmutableMap.builder();
database.getLocation().ifPresent(location -> result.put(LOCATION_PROPERTY, location));
return result.build();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@

import com.facebook.airlift.json.JsonCodec;
import com.facebook.airlift.json.smile.SmileCodec;
import com.facebook.presto.common.CatalogSchemaName;
import com.facebook.presto.common.Subfield;
import com.facebook.presto.common.predicate.NullableValue;
import com.facebook.presto.common.predicate.TupleDomain;
Expand Down Expand Up @@ -65,6 +66,7 @@
import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.QueryId;
import com.facebook.presto.spi.RecordCursor;
import com.facebook.presto.spi.SchemaNotFoundException;
import com.facebook.presto.spi.SchemaTableName;
import com.facebook.presto.spi.SchemaTablePrefix;
import com.facebook.presto.spi.SortingProperty;
Expand Down Expand Up @@ -294,6 +296,7 @@
import static com.facebook.presto.hive.PartitionUpdate.UpdateMode.APPEND;
import static com.facebook.presto.hive.PartitionUpdate.UpdateMode.NEW;
import static com.facebook.presto.hive.PartitionUpdate.UpdateMode.OVERWRITE;
import static com.facebook.presto.hive.SchemaProperties.getDatabaseProperties;
import static com.facebook.presto.hive.metastore.HivePrivilegeInfo.toHivePrivilege;
import static com.facebook.presto.hive.metastore.MetastoreUtil.AVRO_SCHEMA_URL_KEY;
import static com.facebook.presto.hive.metastore.MetastoreUtil.PRESTO_MATERIALIZED_VIEW_FLAG;
Expand Down Expand Up @@ -502,6 +505,17 @@ public List<String> listSchemaNames(ConnectorSession session)
return metastore.getAllDatabases(getMetastoreContext(session));
}

@Override
public Map<String, Object> getSchemaProperties(ConnectorSession session, CatalogSchemaName schemaName)
{
MetastoreContext metastoreContext = getMetastoreContext(session);
Optional<Database> database = metastore.getDatabase(metastoreContext, schemaName.getSchemaName());
if (database.isPresent()) {
return getDatabaseProperties(database.get());
}
throw new SchemaNotFoundException(schemaName.getSchemaName());
}

@Override
public HiveTableHandle getTableHandle(ConnectorSession session, SchemaTableName tableName)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2781,6 +2781,24 @@ public void testShowCreateTable()
actualResult = computeActual("SHOW CREATE TABLE \"test_show_create_table'2\"");
assertEquals(getOnlyElement(actualResult.getOnlyColumnAsSet()), createTableSql);
}
@Test
public void testShowCreateSchema()
{
String createSchemaSql = "CREATE SCHEMA show_create_hive_schema";
assertUpdate(createSchemaSql);
String expectedShowCreateSchema = "CREATE SCHEMA show_create_hive_schema\n" +
"WITH (\n" +
" location = '.*show_create_hive_schema'\n" +
")";

MaterializedResult actualResult = computeActual("SHOW CREATE SCHEMA show_create_hive_schema");
assertThat(getOnlyElement(actualResult.getOnlyColumnAsSet()).toString().matches(expectedShowCreateSchema));

assertQueryFails(format("SHOW CREATE SCHEMA %s.%s", getSession().getCatalog().get(), ""), ".*mismatched input '.'. Expecting: <EOF>");
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");
assertQueryFails(format("SHOW CREATE SCHEMA %s", "schema_not_exist"), ".*Schema 'hive.schema_not_exist' does not exist");
assertUpdate("DROP SCHEMA show_create_hive_schema");
}

@Test
public void testTextfileAmbiguousTimestamp()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@
package com.facebook.presto.iceberg;

import com.facebook.airlift.json.JsonCodec;
import com.facebook.presto.common.CatalogSchemaName;
import com.facebook.presto.common.type.Type;
import com.facebook.presto.common.type.TypeManager;
import com.facebook.presto.hive.HdfsContext;
Expand Down Expand Up @@ -96,6 +97,7 @@
import static com.facebook.presto.hive.HiveUtil.decodeViewData;
import static com.facebook.presto.hive.HiveUtil.encodeViewData;
import static com.facebook.presto.hive.HiveUtil.hiveColumnHandles;
import static com.facebook.presto.hive.SchemaProperties.getDatabaseProperties;
import static com.facebook.presto.hive.SchemaProperties.getLocation;
import static com.facebook.presto.hive.metastore.HivePrivilegeInfo.HivePrivilege.DELETE;
import static com.facebook.presto.hive.metastore.HivePrivilegeInfo.HivePrivilege.INSERT;
Expand Down Expand Up @@ -232,6 +234,17 @@ public List<String> listSchemaNames(ConnectorSession session)
return metastore.getAllDatabases(getMetastoreContext(session));
}

@Override
public Map<String, Object> getSchemaProperties(ConnectorSession session, CatalogSchemaName schemaName)
{
MetastoreContext metastoreContext = getMetastoreContext(session);
Optional<Database> database = metastore.getDatabase(metastoreContext, schemaName.getSchemaName());
if (database.isPresent()) {
return getDatabaseProperties(database.get());
}
throw new SchemaNotFoundException(schemaName.getSchemaName());
}

@Override
public List<SchemaTableName> listTables(ConnectorSession session, Optional<String> schemaName)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,17 +28,21 @@
import com.facebook.presto.iceberg.IcebergUtil;
import com.facebook.presto.spi.ConnectorSession;
import com.facebook.presto.spi.SchemaTableName;
import com.facebook.presto.testing.MaterializedResult;
import com.facebook.presto.tests.DistributedQueryRunner;
import com.google.common.collect.ImmutableSet;
import org.apache.iceberg.Table;
import org.testng.annotations.Test;

import java.io.File;
import java.nio.file.Path;

import static com.facebook.presto.hive.metastore.InMemoryCachingHiveMetastore.memoizeMetastore;
import static com.facebook.presto.iceberg.CatalogType.HIVE;
import static com.facebook.presto.iceberg.IcebergQueryRunner.getIcebergDataDirectoryPath;
import static com.google.common.collect.Iterables.getOnlyElement;
import static java.lang.String.format;
import static org.assertj.core.api.Assertions.assertThat;

public class TestIcebergSmokeHive
extends IcebergDistributedSmokeTestBase
Expand Down Expand Up @@ -83,4 +87,23 @@ protected Table getIcebergTable(ConnectorSession session, String schema, String
session,
SchemaTableName.valueOf(schema + "." + tableName));
}

@Test
public void testShowCreateSchema()
{
String createSchemaSql = "CREATE SCHEMA show_create_iceberg_schema";
assertUpdate(createSchemaSql);
String expectedShowCreateSchema = "CREATE SCHEMA show_create_iceberg_schema\n" +
"WITH (\n" +
" location = '.*show_create_iceberg_schema'\n" +
")";

MaterializedResult actualResult = computeActual("SHOW CREATE SCHEMA show_create_iceberg_schema");
assertThat(getOnlyElement(actualResult.getOnlyColumnAsSet()).toString().matches(expectedShowCreateSchema));

assertQueryFails(format("SHOW CREATE SCHEMA %s.%s", getSession().getCatalog().get(), ""), ".*mismatched input '.'. Expecting: <EOF>");
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");
assertQueryFails(format("SHOW CREATE SCHEMA %s", "schema_not_exist"), ".*Schema 'iceberg.schema_not_exist' does not exist");
assertUpdate("DROP SCHEMA show_create_iceberg_schema");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -95,6 +95,12 @@ public List<String> listSchemaNames(Session session, String catalogName)
return delegate.listSchemaNames(session, catalogName);
}

@Override
public Map<String, Object> getSchemaProperties(Session session, CatalogSchemaName schemaName)
{
return delegate.getSchemaProperties(session, schemaName);
}

@Override
public Optional<SystemTable> getSystemTable(Session session, QualifiedObjectName tableName)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,6 +74,11 @@ public interface Metadata

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

/**
* Gets the schema properties for the specified schema.
*/
Map<String, Object> getSchemaProperties(Session session, CatalogSchemaName schemaName);

Optional<SystemTable> getSystemTable(Session session, QualifiedObjectName tableName);

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,7 @@
import com.facebook.presto.spi.statistics.TableStatisticsMetadata;
import com.facebook.presto.sql.analyzer.FeaturesConfig;
import com.facebook.presto.sql.analyzer.FunctionsConfig;
import com.facebook.presto.sql.analyzer.SemanticException;
import com.facebook.presto.sql.analyzer.TypeSignatureProvider;
import com.facebook.presto.transaction.TransactionManager;
import com.facebook.presto.type.TypeDeserializer;
Expand Down Expand Up @@ -129,6 +130,7 @@
import static com.facebook.presto.spi.StandardErrorCode.SYNTAX_ERROR;
import static com.facebook.presto.spi.TableLayoutFilterCoverage.NOT_APPLICABLE;
import static com.facebook.presto.spi.analyzer.ViewDefinition.ViewColumn;
import static com.facebook.presto.sql.analyzer.SemanticErrorCode.MISSING_SCHEMA;
import static com.facebook.presto.sql.analyzer.TypeSignatureProvider.fromTypes;
import static com.facebook.presto.transaction.InMemoryTransactionManager.createTestTransactionManager;
import static com.google.common.base.Preconditions.checkArgument;
Expand Down Expand Up @@ -318,6 +320,21 @@ public List<String> listSchemaNames(Session session, String catalogName)
return ImmutableList.copyOf(schemaNames.build());
}

@Override
public Map<String, Object> getSchemaProperties(Session session, CatalogSchemaName schemaName)
{
if (!getMetadataResolver(session).schemaExists(schemaName)) {
throw new SemanticException(MISSING_SCHEMA, format("Schema '%s' does not exist", schemaName));
}

Optional<CatalogMetadata> catalog = getOptionalCatalogMetadata(session, transactionManager, schemaName.getCatalogName());
CatalogMetadata catalogMetadata = catalog.get();
ConnectorSession connectorSession = session.toConnectorSession(catalogMetadata.getConnectorId());
ConnectorMetadata metadata = catalogMetadata.getMetadataFor(catalogMetadata.getConnectorId());

return metadata.getSchemaProperties(connectorSession, schemaName);
}

@Override
public Optional<TableHandle> getTableHandleForStatisticsCollection(Session session, QualifiedObjectName table, Map<String, Object> analyzeProperties)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,7 @@
import com.facebook.presto.sql.tree.ConstraintSpecification;
import com.facebook.presto.sql.tree.CreateFunction;
import com.facebook.presto.sql.tree.CreateMaterializedView;
import com.facebook.presto.sql.tree.CreateSchema;
import com.facebook.presto.sql.tree.CreateTable;
import com.facebook.presto.sql.tree.CreateView;
import com.facebook.presto.sql.tree.DoubleLiteral;
Expand Down Expand Up @@ -113,10 +114,12 @@
import static com.facebook.presto.metadata.MetadataUtil.createCatalogSchemaName;
import static com.facebook.presto.metadata.MetadataUtil.createQualifiedName;
import static com.facebook.presto.metadata.MetadataUtil.createQualifiedObjectName;
import static com.facebook.presto.metadata.MetadataUtil.getConnectorIdOrThrow;
import static com.facebook.presto.metadata.SessionFunctionHandle.SESSION_NAMESPACE;
import static com.facebook.presto.spi.StandardErrorCode.FUNCTION_NOT_FOUND;
import static com.facebook.presto.spi.StandardErrorCode.GENERIC_USER_ERROR;
import static com.facebook.presto.spi.StandardErrorCode.INVALID_COLUMN_PROPERTY;
import static com.facebook.presto.spi.StandardErrorCode.INVALID_SCHEMA_PROPERTY;
import static com.facebook.presto.spi.StandardErrorCode.INVALID_TABLE_PROPERTY;
import static com.facebook.presto.sql.ExpressionUtils.combineConjuncts;
import static com.facebook.presto.sql.QueryUtil.aliased;
Expand Down Expand Up @@ -151,6 +154,7 @@
import static com.facebook.presto.sql.tree.RoutineCharacteristics.Language;
import static com.facebook.presto.sql.tree.RoutineCharacteristics.NullCallClause;
import static com.facebook.presto.sql.tree.ShowCreate.Type.MATERIALIZED_VIEW;
import static com.facebook.presto.sql.tree.ShowCreate.Type.SCHEMA;
import static com.facebook.presto.sql.tree.ShowCreate.Type.TABLE;
import static com.facebook.presto.sql.tree.ShowCreate.Type.VIEW;
import static com.facebook.presto.util.AnalyzerUtil.createParsingOptions;
Expand Down Expand Up @@ -456,6 +460,22 @@ private static Expression toExpression(Object value)
@Override
protected Node visitShowCreate(ShowCreate node, Void context)
{
if (node.getType() == SCHEMA) {
CatalogSchemaName catalogSchemaName = createCatalogSchemaName(session, node, Optional.of(node.getName()));
if (!metadataResolver.schemaExists(catalogSchemaName)) {
throw new SemanticException(MISSING_SCHEMA, node, "Schema '%s' does not exist", catalogSchemaName);
}

Map<String, Object> properties = metadata.getSchemaProperties(session, catalogSchemaName);
Map<String, PropertyMetadata<?>> allSchemaProperties = metadata.getSchemaPropertyManager().getAllProperties().get(getConnectorIdOrThrow(session, metadata, catalogSchemaName.getCatalogName()));
List<Property> propertyNodes = buildProperties("schema " + catalogSchemaName, INVALID_SCHEMA_PROPERTY, properties, allSchemaProperties);
CreateSchema createSchema = new CreateSchema(
node.getName(),
false,
propertyNodes);
return singleValueQuery("Create Schema", formatSql(createSchema, Optional.of(parameters)).trim());
}

QualifiedObjectName objectName = createQualifiedObjectName(session, node, node.getName());
Optional<ViewDefinition> viewDefinition = metadataResolver.getView(objectName);
Optional<MaterializedViewDefinition> materializedViewDefinition = metadataResolver.getMaterializedView(objectName);
Expand Down Expand Up @@ -495,7 +515,7 @@ protected Node visitShowCreate(ShowCreate node, Void context)
ConnectorTableMetadata connectorTableMetadata = metadata.getTableMetadata(session, tableHandle.get()).getMetadata();
Map<String, Object> properties = connectorTableMetadata.getProperties();
Map<String, PropertyMetadata<?>> allTableProperties = metadata.getTablePropertyManager().getAllProperties().get(tableHandle.get().getConnectorId());
List<Property> propertyNodes = buildProperties(objectName, Optional.empty(), INVALID_TABLE_PROPERTY, properties, allTableProperties);
List<Property> propertyNodes = buildProperties("materialized view " + objectName, INVALID_TABLE_PROPERTY, properties, allTableProperties);

CreateMaterializedView createMaterializedView = new CreateMaterializedView(
Optional.empty(),
Expand Down Expand Up @@ -532,7 +552,7 @@ protected Node visitShowCreate(ShowCreate node, Void context)
List<TableElement> columns = connectorTableMetadata.getColumns().stream()
.filter(column -> !column.isHidden())
.map(column -> {
List<Property> propertyNodes = buildProperties(objectName, Optional.of(column.getName()), INVALID_COLUMN_PROPERTY, column.getProperties(), allColumnProperties);
List<Property> propertyNodes = buildProperties(toQualifiedName(objectName, Optional.of(column.getName())), INVALID_COLUMN_PROPERTY, column.getProperties(), allColumnProperties);
return new ColumnDefinition(
QueryUtil.quotedIdentifier(column.getName()),
column.getType().getDisplayName(),
Expand All @@ -544,7 +564,7 @@ protected Node visitShowCreate(ShowCreate node, Void context)

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

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

private List<Property> buildProperties(
Object objectName,
Optional<String> columnName,
StandardErrorCode errorCode,
Map<String, Object> properties,
Map<String, PropertyMetadata<?>> allProperties)
Expand All @@ -661,15 +680,15 @@ private List<Property> buildProperties(
String propertyName = propertyEntry.getKey();
Object value = propertyEntry.getValue();
if (value == null) {
throw new PrestoException(errorCode, format("Property %s for %s cannot have a null value", propertyName, toQualifiedName(objectName, columnName)));
throw new PrestoException(errorCode, format("Property %s for %s cannot have a null value", propertyName, objectName));
}

PropertyMetadata<?> property = allProperties.get(propertyName);
if (!Primitives.wrap(property.getJavaType()).isInstance(value)) {
throw new PrestoException(errorCode, format(
"Property %s for %s should have value of type %s, not %s",
propertyName,
toQualifiedName(objectName, columnName),
objectName,
property.getJavaType().getName(),
value.getClass().getName()));
}
Expand Down
Loading

0 comments on commit 5488360

Please sign in to comment.