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 13, 2025
1 parent 0d22e37 commit 68ef2bb
Show file tree
Hide file tree
Showing 19 changed files with 199 additions and 0 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 @@ -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 @@ -201,6 +203,7 @@
import static com.facebook.presto.hive.HivePartition.UNPARTITIONED_ID;
import static com.facebook.presto.hive.HivePartitioningHandle.createHiveCompatiblePartitioningHandle;
import static com.facebook.presto.hive.HivePartitioningHandle.createPrestoNativePartitioningHandle;
import static com.facebook.presto.hive.HiveSchemaProperties.getDatabaseProperties;
import static com.facebook.presto.hive.HiveSessionProperties.HIVE_STORAGE_FORMAT;
import static com.facebook.presto.hive.HiveSessionProperties.RESPECT_TABLE_FORMAT;
import static com.facebook.presto.hive.HiveSessionProperties.getBucketFunctionTypeForExchange;
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 @@ -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 @@ -2781,6 +2781,20 @@ 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));
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 @@ -87,6 +88,7 @@
import java.util.TimeZone;
import java.util.stream.Stream;

import static com.facebook.presto.hive.HiveSchemaProperties.getDatabaseProperties;
import static com.facebook.presto.hive.HiveStatisticsUtil.createPartitionStatistics;
import static com.facebook.presto.hive.HiveStatisticsUtil.updatePartitionStatistics;
import static com.facebook.presto.hive.HiveUtil.decodeViewData;
Expand Down Expand Up @@ -208,6 +210,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,19 @@ 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));
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 @@ -581,6 +585,22 @@ else if (constraint instanceof UniqueConstraint) {
return singleValueQuery("Create Table", formatSql(createTable, Optional.of(parameters)).trim());
}

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<?>> allTableProperties = metadata.getSchemaPropertyManager().getAllProperties().get(getConnectorIdOrThrow(session, metadata, catalogSchemaName.getCatalogName()));
List<Property> propertyNodes = buildProperties(objectName, Optional.empty(), INVALID_SCHEMA_PROPERTY, properties, allTableProperties);
CreateSchema createSchema = new CreateSchema(
node.getName(),
false,
propertyNodes);
return singleValueQuery("Create Schema", formatSql(createSchema, Optional.of(parameters)).trim());
}

throw new UnsupportedOperationException("SHOW CREATE only supported for tables and views");
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -140,6 +140,12 @@ public List<String> listSchemaNames(Session session, String catalogName)
throw new UnsupportedOperationException();
}

@Override
public Map<String, Object> getSchemaProperties(Session session, CatalogSchemaName schemaName)
{
throw new UnsupportedOperationException();
}

@Override
public Optional<TableHandle> getTableHandleForStatisticsCollection(Session session, QualifiedObjectName tableName, Map<String, Object> analyzeProperties)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -118,6 +118,7 @@ statement
| EXPLAIN ANALYZE? VERBOSE?
('(' explainOption (',' explainOption)* ')')? statement #explain
| SHOW CREATE TABLE qualifiedName #showCreateTable
| SHOW CREATE SCHEMA qualifiedName #showCreateSchema
| SHOW CREATE VIEW qualifiedName #showCreateView
| SHOW CREATE MATERIALIZED VIEW qualifiedName #showCreateMaterializedView
| SHOW CREATE FUNCTION qualifiedName types? #showCreateFunction
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -858,6 +858,10 @@ protected Void visitShowCreate(ShowCreate node, Integer context)
builder.append("SHOW CREATE TABLE ")
.append(formatName(node.getName()));
}
else if (node.getType() == ShowCreate.Type.SCHEMA) {
builder.append("SHOW CREATE SCHEMA ")
.append(formatName(node.getName()));
}
else if (node.getType() == ShowCreate.Type.VIEW) {
builder.append("SHOW CREATE VIEW ")
.append(formatName(node.getName()));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -375,6 +375,12 @@ public Node visitShowCreateTable(SqlBaseParser.ShowCreateTableContext context)
return new ShowCreate(getLocation(context), ShowCreate.Type.TABLE, getQualifiedName(context.qualifiedName()));
}

@Override
public Node visitShowCreateSchema(SqlBaseParser.ShowCreateSchemaContext context)
{
return new ShowCreate(getLocation(context), ShowCreate.Type.SCHEMA, getQualifiedName(context.qualifiedName()));
}

@Override
public Node visitShowCreateFunction(SqlBaseParser.ShowCreateFunctionContext context)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ public class ShowCreate
public enum Type
{
TABLE,
SCHEMA,
VIEW,
MATERIALIZED_VIEW
}
Expand Down
Loading

0 comments on commit 68ef2bb

Please sign in to comment.