Skip to content

Commit

Permalink
Rename QualifiedTableName to QualifiedObjectName
Browse files Browse the repository at this point in the history
  • Loading branch information
electrum committed Dec 2, 2015
1 parent 3628005 commit 1bc0676
Show file tree
Hide file tree
Showing 40 changed files with 260 additions and 260 deletions.
Expand Up @@ -15,7 +15,7 @@
package com.facebook.presto.plugin.blackhole; package com.facebook.presto.plugin.blackhole;


import com.facebook.presto.Session; import com.facebook.presto.Session;
import com.facebook.presto.metadata.QualifiedTableName; import com.facebook.presto.metadata.QualifiedObjectName;
import com.facebook.presto.testing.MaterializedResult; import com.facebook.presto.testing.MaterializedResult;
import com.facebook.presto.testing.MaterializedRow; import com.facebook.presto.testing.MaterializedRow;
import com.facebook.presto.testing.QueryRunner; import com.facebook.presto.testing.QueryRunner;
Expand Down Expand Up @@ -105,9 +105,9 @@ public void blackHoleConnectorUsage()
{ {
assertThatQueryReturnsValue("CREATE TABLE nation as SELECT * FROM tpch.tiny.nation", 25L); assertThatQueryReturnsValue("CREATE TABLE nation as SELECT * FROM tpch.tiny.nation", 25L);


List<QualifiedTableName> tableNames = listBlackHoleTables(); List<QualifiedObjectName> tableNames = listBlackHoleTables();
assertTrue(tableNames.size() == 1, "Expected only one table."); assertTrue(tableNames.size() == 1, "Expected only one table.");
assertTrue(tableNames.get(0).getTableName().equals("nation"), "Expected 'nation' table."); assertTrue(tableNames.get(0).getObjectName().equals("nation"), "Expected 'nation' table.");


assertThatQueryReturnsValue("INSERT INTO nation SELECT * FROM tpch.tiny.nation", 25L); assertThatQueryReturnsValue("INSERT INTO nation SELECT * FROM tpch.tiny.nation", 25L);


Expand Down Expand Up @@ -205,7 +205,7 @@ private void assertThatNoBlackHoleTableIsCreated()
assertTrue(listBlackHoleTables().size() == 0, "No blackhole tables expected"); assertTrue(listBlackHoleTables().size() == 0, "No blackhole tables expected");
} }


private List<QualifiedTableName> listBlackHoleTables() private List<QualifiedObjectName> listBlackHoleTables()
{ {
return queryRunner.listTables(createSession(), "blackhole", "default"); return queryRunner.listTables(createSession(), "blackhole", "default");
} }
Expand Down
Expand Up @@ -15,7 +15,7 @@


import com.facebook.presto.Session; import com.facebook.presto.Session;
import com.facebook.presto.metadata.Metadata; import com.facebook.presto.metadata.Metadata;
import com.facebook.presto.metadata.QualifiedTableName; import com.facebook.presto.metadata.QualifiedObjectName;
import com.facebook.presto.metadata.TableHandle; import com.facebook.presto.metadata.TableHandle;
import com.facebook.presto.metadata.TableLayout; import com.facebook.presto.metadata.TableLayout;
import com.facebook.presto.metadata.TableLayoutResult; import com.facebook.presto.metadata.TableLayoutResult;
Expand Down Expand Up @@ -424,7 +424,7 @@ private TableMetadata getTableMetadata(String tableName)
{ {
Session session = getSession(); Session session = getSession();
Metadata metadata = ((DistributedQueryRunner) queryRunner).getCoordinator().getMetadata(); Metadata metadata = ((DistributedQueryRunner) queryRunner).getCoordinator().getMetadata();
Optional<TableHandle> tableHandle = metadata.getTableHandle(session, new QualifiedTableName(HIVE_CATALOG, TPCH_SCHEMA, tableName)); Optional<TableHandle> tableHandle = metadata.getTableHandle(session, new QualifiedObjectName(HIVE_CATALOG, TPCH_SCHEMA, tableName));
assertTrue(tableHandle.isPresent()); assertTrue(tableHandle.isPresent());
return metadata.getTableMetadata(session, tableHandle.get()); return metadata.getTableMetadata(session, tableHandle.get());
} }
Expand All @@ -433,7 +433,7 @@ private List<HivePartition> getPartitions(String tableName)
{ {
Session session = getSession(); Session session = getSession();
Metadata metadata = ((DistributedQueryRunner) queryRunner).getCoordinator().getMetadata(); Metadata metadata = ((DistributedQueryRunner) queryRunner).getCoordinator().getMetadata();
Optional<TableHandle> tableHandle = metadata.getTableHandle(session, new QualifiedTableName(HIVE_CATALOG, TPCH_SCHEMA, tableName)); Optional<TableHandle> tableHandle = metadata.getTableHandle(session, new QualifiedObjectName(HIVE_CATALOG, TPCH_SCHEMA, tableName));
assertTrue(tableHandle.isPresent()); assertTrue(tableHandle.isPresent());


List<TableLayoutResult> layouts = metadata.getLayouts(session, tableHandle.get(), Constraint.alwaysTrue(), Optional.empty()); List<TableLayoutResult> layouts = metadata.getLayouts(session, tableHandle.get(), Constraint.alwaysTrue(), Optional.empty());
Expand Down
Expand Up @@ -18,7 +18,7 @@
import com.facebook.presto.kafka.util.EmbeddedKafka; import com.facebook.presto.kafka.util.EmbeddedKafka;
import com.facebook.presto.kafka.util.TestUtils; import com.facebook.presto.kafka.util.TestUtils;
import com.facebook.presto.metadata.Metadata; import com.facebook.presto.metadata.Metadata;
import com.facebook.presto.metadata.QualifiedTableName; import com.facebook.presto.metadata.QualifiedObjectName;
import com.facebook.presto.spi.SchemaTableName; import com.facebook.presto.spi.SchemaTableName;
import com.facebook.presto.tests.DistributedQueryRunner; import com.facebook.presto.tests.DistributedQueryRunner;
import com.facebook.presto.tests.TestingPrestoClient; import com.facebook.presto.tests.TestingPrestoClient;
Expand Down Expand Up @@ -96,7 +96,7 @@ private static void loadTpchTopic(EmbeddedKafka embeddedKafka, TestingPrestoClie
{ {
long start = System.nanoTime(); long start = System.nanoTime();
log.info("Running import for %s", table.getTableName()); log.info("Running import for %s", table.getTableName());
TestUtils.loadTpchTopic(embeddedKafka, prestoClient, kafkaTopicName(table), new QualifiedTableName("tpch", TINY_SCHEMA_NAME, table.getTableName().toLowerCase(ENGLISH))); TestUtils.loadTpchTopic(embeddedKafka, prestoClient, kafkaTopicName(table), new QualifiedObjectName("tpch", TINY_SCHEMA_NAME, table.getTableName().toLowerCase(ENGLISH)));
log.info("Imported %s in %s", 0, table.getTableName(), nanosSince(start).convertToMostSuccinctTimeUnit()); log.info("Imported %s in %s", 0, table.getTableName(), nanosSince(start).convertToMostSuccinctTimeUnit());
} }


Expand Down
Expand Up @@ -16,7 +16,7 @@
import com.facebook.presto.Session; import com.facebook.presto.Session;
import com.facebook.presto.kafka.util.EmbeddedKafka; import com.facebook.presto.kafka.util.EmbeddedKafka;
import com.facebook.presto.kafka.util.TestUtils; import com.facebook.presto.kafka.util.TestUtils;
import com.facebook.presto.metadata.QualifiedTableName; import com.facebook.presto.metadata.QualifiedObjectName;
import com.facebook.presto.metadata.TableHandle; import com.facebook.presto.metadata.TableHandle;
import com.facebook.presto.spi.SchemaTableName; import com.facebook.presto.spi.SchemaTableName;
import com.facebook.presto.spi.type.BigintType; import com.facebook.presto.spi.type.BigintType;
Expand Down Expand Up @@ -105,7 +105,7 @@ private void createMessages(String topicName, int count)
public void testTopicExists() public void testTopicExists()
throws Exception throws Exception
{ {
QualifiedTableName name = new QualifiedTableName("kafka", "default", topicName); QualifiedObjectName name = new QualifiedObjectName("kafka", "default", topicName);
Optional<TableHandle> handle = queryRunner.getServer().getMetadata().getTableHandle(SESSION, name); Optional<TableHandle> handle = queryRunner.getServer().getMetadata().getTableHandle(SESSION, name);
assertTrue(handle.isPresent()); assertTrue(handle.isPresent());
} }
Expand Down
Expand Up @@ -15,7 +15,7 @@


import com.facebook.presto.kafka.KafkaPlugin; import com.facebook.presto.kafka.KafkaPlugin;
import com.facebook.presto.kafka.KafkaTopicDescription; import com.facebook.presto.kafka.KafkaTopicDescription;
import com.facebook.presto.metadata.QualifiedTableName; import com.facebook.presto.metadata.QualifiedObjectName;
import com.facebook.presto.spi.SchemaTableName; import com.facebook.presto.spi.SchemaTableName;
import com.facebook.presto.testing.QueryRunner; import com.facebook.presto.testing.QueryRunner;
import com.facebook.presto.tests.TestingPrestoClient; import com.facebook.presto.tests.TestingPrestoClient;
Expand Down Expand Up @@ -68,7 +68,7 @@ public static void installKafkaPlugin(EmbeddedKafka embeddedKafka, QueryRunner q
queryRunner.createCatalog("kafka", "kafka", kafkaConfig); queryRunner.createCatalog("kafka", "kafka", kafkaConfig);
} }


public static void loadTpchTopic(EmbeddedKafka embeddedKafka, TestingPrestoClient prestoClient, String topicName, QualifiedTableName tpchTableName) public static void loadTpchTopic(EmbeddedKafka embeddedKafka, TestingPrestoClient prestoClient, String topicName, QualifiedObjectName tpchTableName)
{ {
try (CloseableProducer<Long, Object> producer = embeddedKafka.createProducer(); try (CloseableProducer<Long, Object> producer = embeddedKafka.createProducer();
KafkaLoader tpchLoader = new KafkaLoader(producer, topicName, prestoClient.getServer(), prestoClient.getDefaultSession())) { KafkaLoader tpchLoader = new KafkaLoader(producer, topicName, prestoClient.getServer(), prestoClient.getDefaultSession())) {
Expand Down
Expand Up @@ -18,7 +18,7 @@
import com.facebook.presto.metadata.InternalTable; import com.facebook.presto.metadata.InternalTable;
import com.facebook.presto.metadata.Metadata; import com.facebook.presto.metadata.Metadata;
import com.facebook.presto.metadata.OperatorNotFoundException; import com.facebook.presto.metadata.OperatorNotFoundException;
import com.facebook.presto.metadata.QualifiedTableName; import com.facebook.presto.metadata.QualifiedObjectName;
import com.facebook.presto.metadata.QualifiedTablePrefix; import com.facebook.presto.metadata.QualifiedTablePrefix;
import com.facebook.presto.metadata.Signature; import com.facebook.presto.metadata.Signature;
import com.facebook.presto.metadata.TableHandle; import com.facebook.presto.metadata.TableHandle;
Expand Down Expand Up @@ -151,8 +151,8 @@ public InternalTable getInformationSchemaTable(Session session, String catalog,
private InternalTable buildColumns(Session session, String catalogName, Map<String, NullableValue> filters) private InternalTable buildColumns(Session session, String catalogName, Map<String, NullableValue> filters)
{ {
InternalTable.Builder table = InternalTable.builder(informationSchemaTableColumns(TABLE_COLUMNS)); InternalTable.Builder table = InternalTable.builder(informationSchemaTableColumns(TABLE_COLUMNS));
for (Entry<QualifiedTableName, List<ColumnMetadata>> entry : getColumnsList(session, catalogName, filters).entrySet()) { for (Entry<QualifiedObjectName, List<ColumnMetadata>> entry : getColumnsList(session, catalogName, filters).entrySet()) {
QualifiedTableName tableName = entry.getKey(); QualifiedObjectName tableName = entry.getKey();
int ordinalPosition = 1; int ordinalPosition = 1;
for (ColumnMetadata column : entry.getValue()) { for (ColumnMetadata column : entry.getValue()) {
if (column.isHidden()) { if (column.isHidden()) {
Expand All @@ -161,7 +161,7 @@ private InternalTable buildColumns(Session session, String catalogName, Map<Stri
table.add( table.add(
tableName.getCatalogName(), tableName.getCatalogName(),
tableName.getSchemaName(), tableName.getSchemaName(),
tableName.getTableName(), tableName.getObjectName(),
column.getName(), column.getName(),
ordinalPosition, ordinalPosition,
null, null,
Expand All @@ -175,53 +175,53 @@ private InternalTable buildColumns(Session session, String catalogName, Map<Stri
return table.build(); return table.build();
} }


private Map<QualifiedTableName, List<ColumnMetadata>> getColumnsList(Session session, String catalogName, Map<String, NullableValue> filters) private Map<QualifiedObjectName, List<ColumnMetadata>> getColumnsList(Session session, String catalogName, Map<String, NullableValue> filters)
{ {
return metadata.listTableColumns(session, extractQualifiedTablePrefix(catalogName, filters)); return metadata.listTableColumns(session, extractQualifiedTablePrefix(catalogName, filters));
} }


private InternalTable buildTables(Session session, String catalogName, Map<String, NullableValue> filters) private InternalTable buildTables(Session session, String catalogName, Map<String, NullableValue> filters)
{ {
Set<QualifiedTableName> tables = ImmutableSet.copyOf(getTablesList(session, catalogName, filters)); Set<QualifiedObjectName> tables = ImmutableSet.copyOf(getTablesList(session, catalogName, filters));
Set<QualifiedTableName> views = ImmutableSet.copyOf(getViewsList(session, catalogName, filters)); Set<QualifiedObjectName> views = ImmutableSet.copyOf(getViewsList(session, catalogName, filters));


InternalTable.Builder table = InternalTable.builder(informationSchemaTableColumns(TABLE_TABLES)); InternalTable.Builder table = InternalTable.builder(informationSchemaTableColumns(TABLE_TABLES));
for (QualifiedTableName name : union(tables, views)) { for (QualifiedObjectName name : union(tables, views)) {
// if table and view names overlap, the view wins // if table and view names overlap, the view wins
String type = views.contains(name) ? "VIEW" : "BASE TABLE"; String type = views.contains(name) ? "VIEW" : "BASE TABLE";
table.add( table.add(
name.getCatalogName(), name.getCatalogName(),
name.getSchemaName(), name.getSchemaName(),
name.getTableName(), name.getObjectName(),
type); type);
} }
return table.build(); return table.build();
} }


private List<QualifiedTableName> getTablesList(Session session, String catalogName, Map<String, NullableValue> filters) private List<QualifiedObjectName> getTablesList(Session session, String catalogName, Map<String, NullableValue> filters)
{ {
return metadata.listTables(session, extractQualifiedTablePrefix(catalogName, filters)); return metadata.listTables(session, extractQualifiedTablePrefix(catalogName, filters));
} }


private List<QualifiedTableName> getViewsList(Session session, String catalogName, Map<String, NullableValue> filters) private List<QualifiedObjectName> getViewsList(Session session, String catalogName, Map<String, NullableValue> filters)
{ {
return metadata.listViews(session, extractQualifiedTablePrefix(catalogName, filters)); return metadata.listViews(session, extractQualifiedTablePrefix(catalogName, filters));
} }


private InternalTable buildViews(Session session, String catalogName, Map<String, NullableValue> filters) private InternalTable buildViews(Session session, String catalogName, Map<String, NullableValue> filters)
{ {
InternalTable.Builder table = InternalTable.builder(informationSchemaTableColumns(TABLE_VIEWS)); InternalTable.Builder table = InternalTable.builder(informationSchemaTableColumns(TABLE_VIEWS));
for (Entry<QualifiedTableName, ViewDefinition> entry : getViews(session, catalogName, filters).entrySet()) { for (Entry<QualifiedObjectName, ViewDefinition> entry : getViews(session, catalogName, filters).entrySet()) {
table.add( table.add(
entry.getKey().getCatalogName(), entry.getKey().getCatalogName(),
entry.getKey().getSchemaName(), entry.getKey().getSchemaName(),
entry.getKey().getTableName(), entry.getKey().getObjectName(),
entry.getValue().getOriginalSql()); entry.getValue().getOriginalSql());
} }
return table.build(); return table.build();
} }


private Map<QualifiedTableName, ViewDefinition> getViews(Session session, String catalogName, Map<String, NullableValue> filters) private Map<QualifiedObjectName, ViewDefinition> getViews(Session session, String catalogName, Map<String, NullableValue> filters)
{ {
return metadata.getViews(session, extractQualifiedTablePrefix(catalogName, filters)); return metadata.getViews(session, extractQualifiedTablePrefix(catalogName, filters));
} }
Expand All @@ -237,7 +237,7 @@ private InternalTable buildSchemata(Session session, String catalogName)


private InternalTable buildPartitions(Session session, String catalogName, Map<String, NullableValue> filters) private InternalTable buildPartitions(Session session, String catalogName, Map<String, NullableValue> filters)
{ {
QualifiedTableName tableName = extractQualifiedTableName(catalogName, filters); QualifiedObjectName tableName = extractQualifiedTableName(catalogName, filters);


InternalTable.Builder table = InternalTable.builder(informationSchemaTableColumns(TABLE_INTERNAL_PARTITIONS)); InternalTable.Builder table = InternalTable.builder(informationSchemaTableColumns(TABLE_INTERNAL_PARTITIONS));


Expand Down Expand Up @@ -286,7 +286,7 @@ private InternalTable buildPartitions(Session session, String catalogName, Map<S
table.add( table.add(
catalogName, catalogName,
tableName.getSchemaName(), tableName.getSchemaName(),
tableName.getTableName(), tableName.getObjectName(),
partitionNumber, partitionNumber,
columnName, columnName,
value); value);
Expand All @@ -298,13 +298,13 @@ private InternalTable buildPartitions(Session session, String catalogName, Map<S
return table.build(); return table.build();
} }


private static QualifiedTableName extractQualifiedTableName(String catalogName, Map<String, NullableValue> filters) private static QualifiedObjectName extractQualifiedTableName(String catalogName, Map<String, NullableValue> filters)
{ {
Optional<String> schemaName = getFilterColumn(filters, "table_schema"); Optional<String> schemaName = getFilterColumn(filters, "table_schema");
checkArgument(schemaName.isPresent(), "filter is required for column: %s.%s", TABLE_INTERNAL_PARTITIONS, "table_schema"); checkArgument(schemaName.isPresent(), "filter is required for column: %s.%s", TABLE_INTERNAL_PARTITIONS, "table_schema");
Optional<String> tableName = getFilterColumn(filters, "table_name"); Optional<String> tableName = getFilterColumn(filters, "table_name");
checkArgument(tableName.isPresent(), "filter is required for column: %s.%s", TABLE_INTERNAL_PARTITIONS, "table_name"); checkArgument(tableName.isPresent(), "filter is required for column: %s.%s", TABLE_INTERNAL_PARTITIONS, "table_name");
return new QualifiedTableName(catalogName, schemaName.get(), tableName.get()); return new QualifiedObjectName(catalogName, schemaName.get(), tableName.get());
} }


private static QualifiedTablePrefix extractQualifiedTablePrefix(String catalogName, Map<String, NullableValue> filters) private static QualifiedTablePrefix extractQualifiedTablePrefix(String catalogName, Map<String, NullableValue> filters)
Expand Down
Expand Up @@ -15,7 +15,7 @@


import com.facebook.presto.Session; import com.facebook.presto.Session;
import com.facebook.presto.metadata.Metadata; import com.facebook.presto.metadata.Metadata;
import com.facebook.presto.metadata.QualifiedTableName; import com.facebook.presto.metadata.QualifiedObjectName;
import com.facebook.presto.metadata.QualifiedTablePrefix; import com.facebook.presto.metadata.QualifiedTablePrefix;
import com.facebook.presto.spi.ColumnMetadata; import com.facebook.presto.spi.ColumnMetadata;
import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.ConnectorSession;
Expand Down Expand Up @@ -109,14 +109,14 @@ public RecordCursor cursor(ConnectorSession connectorSession, TupleDomain<Intege
Builder table = InMemoryRecordSet.builder(METADATA); Builder table = InMemoryRecordSet.builder(METADATA);
for (String catalog : filter(metadata.getCatalogNames().keySet(), catalogFilter)) { for (String catalog : filter(metadata.getCatalogNames().keySet(), catalogFilter)) {
QualifiedTablePrefix prefix = FilterUtil.tablePrefix(catalog, schemaFilter, tableFilter); QualifiedTablePrefix prefix = FilterUtil.tablePrefix(catalog, schemaFilter, tableFilter);
for (Entry<QualifiedTableName, List<ColumnMetadata>> entry : metadata.listTableColumns(session, prefix).entrySet()) { for (Entry<QualifiedObjectName, List<ColumnMetadata>> entry : metadata.listTableColumns(session, prefix).entrySet()) {
addColumnRows(table, entry.getKey(), entry.getValue()); addColumnRows(table, entry.getKey(), entry.getValue());
} }
} }
return table.build().cursor(); return table.build().cursor();
} }


private static void addColumnRows(Builder builder, QualifiedTableName tableName, List<ColumnMetadata> columns) private static void addColumnRows(Builder builder, QualifiedObjectName tableName, List<ColumnMetadata> columns)
{ {
int ordinalPosition = 1; int ordinalPosition = 1;
for (ColumnMetadata column : columns) { for (ColumnMetadata column : columns) {
Expand All @@ -126,7 +126,7 @@ private static void addColumnRows(Builder builder, QualifiedTableName tableName,
builder.addRow( builder.addRow(
tableName.getCatalogName(), tableName.getCatalogName(),
tableName.getSchemaName(), tableName.getSchemaName(),
tableName.getTableName(), tableName.getObjectName(),
column.getName(), column.getName(),
jdbcDataType(column.getType()), jdbcDataType(column.getType()),
column.getType().getDisplayName(), column.getType().getDisplayName(),
Expand Down
Expand Up @@ -15,7 +15,7 @@


import com.facebook.presto.Session; import com.facebook.presto.Session;
import com.facebook.presto.metadata.Metadata; import com.facebook.presto.metadata.Metadata;
import com.facebook.presto.metadata.QualifiedTableName; import com.facebook.presto.metadata.QualifiedObjectName;
import com.facebook.presto.metadata.QualifiedTablePrefix; import com.facebook.presto.metadata.QualifiedTablePrefix;
import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.ConnectorSession;
import com.facebook.presto.spi.ConnectorTableMetadata; import com.facebook.presto.spi.ConnectorTableMetadata;
Expand Down Expand Up @@ -83,23 +83,23 @@ public RecordCursor cursor(ConnectorSession connectorSession, TupleDomain<Intege
QualifiedTablePrefix prefix = tablePrefix(catalog, schemaFilter, tableFilter); QualifiedTablePrefix prefix = tablePrefix(catalog, schemaFilter, tableFilter);


if (FilterUtil.emptyOrEquals(typeFilter, "TABLE")) { if (FilterUtil.emptyOrEquals(typeFilter, "TABLE")) {
for (QualifiedTableName name : metadata.listTables(session, prefix)) { for (QualifiedObjectName name : metadata.listTables(session, prefix)) {
table.addRow(tableRow(name, "TABLE")); table.addRow(tableRow(name, "TABLE"));
} }
} }


if (FilterUtil.emptyOrEquals(typeFilter, "VIEW")) { if (FilterUtil.emptyOrEquals(typeFilter, "VIEW")) {
for (QualifiedTableName name : metadata.listViews(session, prefix)) { for (QualifiedObjectName name : metadata.listViews(session, prefix)) {
table.addRow(tableRow(name, "VIEW")); table.addRow(tableRow(name, "VIEW"));
} }
} }
} }
return table.build().cursor(); return table.build().cursor();
} }


private static Object[] tableRow(QualifiedTableName name, String type) private static Object[] tableRow(QualifiedObjectName name, String type)
{ {
return new Object[] {name.getCatalogName(), name.getSchemaName(), name.getTableName(), type, return new Object[] {name.getCatalogName(), name.getSchemaName(), name.getObjectName(), type,
null, null, null, null, null, null}; null, null, null, null, null, null};
} }
} }
Expand Up @@ -15,7 +15,7 @@


import com.facebook.presto.Session; import com.facebook.presto.Session;
import com.facebook.presto.metadata.Metadata; import com.facebook.presto.metadata.Metadata;
import com.facebook.presto.metadata.QualifiedTableName; import com.facebook.presto.metadata.QualifiedObjectName;
import com.facebook.presto.metadata.TableHandle; import com.facebook.presto.metadata.TableHandle;
import com.facebook.presto.security.AccessControl; import com.facebook.presto.security.AccessControl;
import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.ColumnHandle;
Expand All @@ -28,7 +28,7 @@
import java.util.Map; import java.util.Map;
import java.util.Optional; import java.util.Optional;


import static com.facebook.presto.metadata.MetadataUtil.createQualifiedTableName; import static com.facebook.presto.metadata.MetadataUtil.createQualifiedObjectName;
import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature; import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature;
import static com.facebook.presto.sql.analyzer.SemanticErrorCode.COLUMN_ALREADY_EXISTS; import static com.facebook.presto.sql.analyzer.SemanticErrorCode.COLUMN_ALREADY_EXISTS;
import static com.facebook.presto.sql.analyzer.SemanticErrorCode.MISSING_TABLE; import static com.facebook.presto.sql.analyzer.SemanticErrorCode.MISSING_TABLE;
Expand All @@ -47,7 +47,7 @@ public String getName()
@Override @Override
public void execute(AddColumn statement, Session session, Metadata metadata, AccessControl accessControl, QueryStateMachine stateMachine) public void execute(AddColumn statement, Session session, Metadata metadata, AccessControl accessControl, QueryStateMachine stateMachine)
{ {
QualifiedTableName tableName = createQualifiedTableName(session, statement, statement.getName()); QualifiedObjectName tableName = createQualifiedObjectName(session, statement, statement.getName());
Optional<TableHandle> tableHandle = metadata.getTableHandle(session, tableName); Optional<TableHandle> tableHandle = metadata.getTableHandle(session, tableName);
if (!tableHandle.isPresent()) { if (!tableHandle.isPresent()) {
throw new SemanticException(MISSING_TABLE, statement, "Table '%s' does not exist", tableName); throw new SemanticException(MISSING_TABLE, statement, "Table '%s' does not exist", tableName);
Expand Down

0 comments on commit 1bc0676

Please sign in to comment.