Skip to content

Commit

Permalink
Filter information_schema within engine
Browse files Browse the repository at this point in the history
  • Loading branch information
ebyhr committed Feb 21, 2023
1 parent 577567f commit 0b398f9
Show file tree
Hide file tree
Showing 10 changed files with 53 additions and 50 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -1339,7 +1339,7 @@ public void testGetColumnsMetadataCalls()
.collect(toImmutableList()),
new MetadataCallsCount()
.withListSchemasCount(3)
.withListTablesCount(8)
.withListTablesCount(6)
.withGetTableHandleCount(2)
.withGetColumnsCount(2));

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -112,6 +112,7 @@
import io.trino.transaction.TransactionManager;
import io.trino.type.BlockTypeOperators;

import javax.annotation.Nullable;
import javax.annotation.concurrent.GuardedBy;
import javax.inject.Inject;

Expand Down Expand Up @@ -250,6 +251,7 @@ public List<String> listSchemaNames(Session session, String catalogName)
ConnectorMetadata metadata = catalogMetadata.getMetadataFor(session, catalogHandle);
metadata.listSchemaNames(connectorSession).stream()
.map(schema -> schema.toLowerCase(Locale.ENGLISH))
.filter(schema -> !isExternalInformationSchema(catalogHandle, schema))
.forEach(schemaNames::add);
}
}
Expand Down Expand Up @@ -508,8 +510,12 @@ public List<QualifiedObjectName> listTables(Session session, QualifiedTablePrefi
for (CatalogHandle catalogHandle : catalogMetadata.listCatalogHandles()) {
ConnectorMetadata metadata = catalogMetadata.getMetadataFor(session, catalogHandle);
ConnectorSession connectorSession = session.toConnectorSession(catalogHandle);
if (isExternalInformationSchema(catalogHandle, prefix.getSchemaName())) {
continue;
}
metadata.listTables(connectorSession, prefix.getSchemaName()).stream()
.map(convertFromSchemaTableName(prefix.getCatalogName()))
.filter(table -> !isExternalInformationSchema(catalogHandle, table.getSchemaName()))
.filter(prefix::matches)
.forEach(tables::add);
}
Expand Down Expand Up @@ -554,13 +560,21 @@ public List<TableColumnsMetadata> listTableColumns(Session session, QualifiedTab

SchemaTablePrefix tablePrefix = prefix.asSchemaTablePrefix();
for (CatalogHandle catalogHandle : catalogMetadata.listCatalogHandles()) {
if (isExternalInformationSchema(catalogHandle, prefix.getSchemaName())) {
continue;
}

ConnectorMetadata metadata = catalogMetadata.getMetadataFor(session, catalogHandle);

ConnectorSession connectorSession = session.toConnectorSession(catalogHandle);

// Collect column metadata from tables
metadata.streamTableColumns(connectorSession, tablePrefix)
.forEachRemaining(columnsMetadata -> tableColumns.put(columnsMetadata.getTable(), columnsMetadata.getColumns()));
.forEachRemaining(columnsMetadata -> {
if (!isExternalInformationSchema(catalogHandle, columnsMetadata.getTable().getSchemaName())) {
tableColumns.put(columnsMetadata.getTable(), columnsMetadata.getColumns());
}
});

// Collect column metadata from views. if table and view names overlap, the view wins
for (Entry<QualifiedObjectName, ViewInfo> entry : getViews(session, prefix).entrySet()) {
Expand Down Expand Up @@ -1068,10 +1082,14 @@ public List<QualifiedObjectName> listViews(Session session, QualifiedTablePrefix
CatalogMetadata catalogMetadata = catalog.get();

for (CatalogHandle catalogHandle : catalogMetadata.listCatalogHandles()) {
if (isExternalInformationSchema(catalogHandle, prefix.getSchemaName())) {
continue;
}
ConnectorMetadata metadata = catalogMetadata.getMetadataFor(session, catalogHandle);
ConnectorSession connectorSession = session.toConnectorSession(catalogHandle);
metadata.listViews(connectorSession, prefix.getSchemaName()).stream()
.map(convertFromSchemaTableName(prefix.getCatalogName()))
.filter(view -> !isExternalInformationSchema(catalogHandle, view.getSchemaName()))
.filter(prefix::matches)
.forEach(views::add);
}
Expand All @@ -1092,6 +1110,10 @@ public Map<QualifiedObjectName, ViewInfo> getViews(Session session, QualifiedTab

SchemaTablePrefix tablePrefix = prefix.asSchemaTablePrefix();
for (CatalogHandle catalogHandle : catalogMetadata.listCatalogHandles()) {
if (isExternalInformationSchema(catalogHandle, tablePrefix.getSchema())) {
continue;
}

ConnectorMetadata metadata = catalogMetadata.getMetadataFor(session, catalogHandle);
ConnectorSession connectorSession = session.toConnectorSession(catalogHandle);

Expand All @@ -1106,6 +1128,9 @@ public Map<QualifiedObjectName, ViewInfo> getViews(Session session, QualifiedTab
}

for (Entry<SchemaTableName, ConnectorViewDefinition> entry : viewMap.entrySet()) {
if (isExternalInformationSchema(catalogHandle, entry.getKey().getSchemaName())) {
continue;
}
QualifiedObjectName viewName = new QualifiedObjectName(
prefix.getCatalogName(),
entry.getKey().getSchemaName(),
Expand Down Expand Up @@ -1291,10 +1316,14 @@ public List<QualifiedObjectName> listMaterializedViews(Session session, Qualifie
CatalogMetadata catalogMetadata = catalog.get();

for (CatalogHandle catalogHandle : catalogMetadata.listCatalogHandles()) {
if (isExternalInformationSchema(catalogHandle, prefix.getSchemaName())) {
continue;
}
ConnectorMetadata metadata = catalogMetadata.getMetadataFor(session, catalogHandle);
ConnectorSession connectorSession = session.toConnectorSession(catalogHandle);
metadata.listMaterializedViews(connectorSession, prefix.getSchemaName()).stream()
.map(convertFromSchemaTableName(prefix.getCatalogName()))
.filter(materializedView -> !isExternalInformationSchema(catalogHandle, materializedView.getSchemaName()))
.filter(prefix::matches)
.forEach(materializedViews::add);
}
Expand All @@ -1315,6 +1344,9 @@ public Map<QualifiedObjectName, ViewInfo> getMaterializedViews(Session session,

SchemaTablePrefix tablePrefix = prefix.asSchemaTablePrefix();
for (CatalogHandle catalogHandle : catalogMetadata.listCatalogHandles()) {
if (isExternalInformationSchema(catalogHandle, tablePrefix.getSchema())) {
continue;
}
ConnectorMetadata metadata = catalogMetadata.getMetadataFor(session, catalogHandle);
ConnectorSession connectorSession = session.toConnectorSession(catalogHandle);

Expand All @@ -1329,6 +1361,9 @@ public Map<QualifiedObjectName, ViewInfo> getMaterializedViews(Session session,
}

for (Entry<SchemaTableName, ConnectorMaterializedViewDefinition> entry : materializedViewMap.entrySet()) {
if (isExternalInformationSchema(catalogHandle, entry.getKey().getSchemaName())) {
continue;
}
QualifiedObjectName viewName = new QualifiedObjectName(
prefix.getCatalogName(),
entry.getKey().getSchemaName(),
Expand Down Expand Up @@ -1423,6 +1458,16 @@ public void setMaterializedViewProperties(Session session, QualifiedObjectName v
metadata.setMaterializedViewProperties(session.toConnectorSession(catalogHandle), viewName.asSchemaTableName(), properties);
}

private static boolean isExternalInformationSchema(CatalogHandle catalogHandle, Optional<String> schemaName)
{
return isExternalInformationSchema(catalogHandle, schemaName.orElse(null));
}

private static boolean isExternalInformationSchema(CatalogHandle catalogHandle, @Nullable String schemaName)
{
return !catalogHandle.getType().isInternal() && "information_schema".equalsIgnoreCase(schemaName);
}

@Override
public Optional<TableScanRedirectApplicationResult> applyTableScanRedirect(Session session, TableHandle tableHandle)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -119,7 +119,6 @@ public class BigQueryMetadata

static final int DEFAULT_NUMERIC_TYPE_PRECISION = 38;
static final int DEFAULT_NUMERIC_TYPE_SCALE = 9;
static final String INFORMATION_SCHEMA = "information_schema";
private static final String VIEW_DEFINITION_SYSTEM_TABLE_SUFFIX = "$view_definition";

private final BigQueryClientFactory bigQueryClientFactory;
Expand All @@ -146,7 +145,6 @@ private List<String> listRemoteSchemaNames(ConnectorSession session)

Stream<String> remoteSchemaNames = Streams.stream(client.listDatasets(projectId))
.map(dataset -> dataset.getDatasetId().getDataset())
.filter(schemaName -> !schemaName.equalsIgnoreCase(INFORMATION_SCHEMA))
.distinct();

// filter out all the ambiguous schemas to prevent failures if anyone tries to access the listed schemas
Expand Down Expand Up @@ -179,19 +177,12 @@ public List<SchemaTableName> listTables(ConnectorSession session, Optional<Strin
BigQueryClient client = bigQueryClientFactory.create(session);

log.debug("listTables(session=%s, schemaName=%s)", session, schemaName);
if (schemaName.isPresent() && schemaName.get().equalsIgnoreCase(INFORMATION_SCHEMA)) {
return ImmutableList.of();
}

String projectId = client.getProjectId();

// filter ambiguous schemas
Optional<String> remoteSchema = schemaName.flatMap(schema -> client.toRemoteDataset(projectId, schema)
.filter(dataset -> !dataset.isAmbiguous())
.map(RemoteDatabaseObject::getOnlyRemoteName));
if (remoteSchema.isPresent() && remoteSchema.get().equalsIgnoreCase(INFORMATION_SCHEMA)) {
return ImmutableList.of();
}

Set<String> remoteSchemaNames = remoteSchema.map(ImmutableSet::of)
.orElseGet(() -> ImmutableSet.copyOf(listRemoteSchemaNames(session)));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -251,7 +251,6 @@
import static io.trino.spi.type.VarcharType.VARCHAR;
import static java.lang.String.format;
import static java.time.Instant.EPOCH;
import static java.util.Collections.emptyIterator;
import static java.util.Collections.singletonList;
import static java.util.Collections.unmodifiableMap;
import static java.util.Locale.ENGLISH;
Expand Down Expand Up @@ -365,10 +364,7 @@ public DeltaLakeMetadata(
public List<String> listSchemaNames(ConnectorSession session)
{
return metastore.getAllDatabases().stream()
.filter(schema -> {
String schemaName = schema.toLowerCase(ENGLISH);
return !(schemaName.equals("information_schema") || schemaName.equals("sys"));
})
.filter(schema -> !schema.toLowerCase(ENGLISH).equals("sys"))
.collect(toImmutableList());
}

Expand Down Expand Up @@ -490,10 +486,6 @@ public ConnectorTableMetadata getTableMetadata(ConnectorSession session, Connect
@Override
public List<SchemaTableName> listTables(ConnectorSession session, Optional<String> schemaName)
{
if (schemaName.isPresent() && schemaName.get().equals("information_schema")) {
// TODO https://github.com/trinodb/trino/issues/1559 information_schema should be handled by the engine fully
return ImmutableList.of();
}
return schemaName.map(Collections::singletonList)
.orElseGet(() -> listSchemaNames(session))
.stream()
Expand Down Expand Up @@ -561,11 +553,6 @@ public Map<SchemaTableName, List<ColumnMetadata>> listTableColumns(ConnectorSess
@Override
public Iterator<TableColumnsMetadata> streamTableColumns(ConnectorSession session, SchemaTablePrefix prefix)
{
if (prefix.getSchema().isPresent() && prefix.getSchema().get().equals("information_schema")) {
// TODO https://github.com/trinodb/trino/issues/1559 information_schema should be handled by the engine fully
return emptyIterator();
}

List<SchemaTableName> tables = prefix.getTable()
.map(ignored -> singletonList(prefix.toSchemaTableName()))
.orElseGet(() -> listTables(session, prefix.getSchema()));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1098,11 +1098,6 @@ public static String sortingColumnToString(SortingColumn column)

public static boolean isHiveSystemSchema(String schemaName)
{
if ("information_schema".equals(schemaName)) {
// For things like listing columns in information_schema.columns table, we need to explicitly filter out Hive's own information_schema.
// TODO https://github.com/trinodb/trino/issues/1559 this should be filtered out in engine.
return true;
}
if ("sys".equals(schemaName)) {
// Hive 3's `sys` schema contains no objects we can handle, so there is no point in exposing it.
// Also, exposing it may require proper handling in access control.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -198,10 +198,6 @@ public List<String> listNamespaces(ConnectorSession session)
private List<String> listNamespaces(ConnectorSession session, Optional<String> namespace)
{
if (namespace.isPresent()) {
if (isHiveSystemSchema(namespace.get())) {
// TODO https://github.com/trinodb/trino/issues/1559 information_schema should be handled by the engine fully
return ImmutableList.of();
}
return ImmutableList.of(namespace.get());
}
return listNamespaces(session);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -602,7 +602,6 @@ private List<String> listNamespaces(ConnectorSession session, Optional<String> n
{
if (namespace.isPresent()) {
if (isHiveSystemSchema(namespace.get())) {
// TODO https://github.com/trinodb/trino/issues/1559 information_schema should be handled by the engine fully
return ImmutableList.of();
}
return ImmutableList.of(namespace.get());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -162,10 +162,6 @@ public List<SchemaTableName> listTables(ConnectorSession session, Optional<Strin
private List<String> listNamespaces(ConnectorSession session, Optional<String> namespace)
{
if (namespace.isPresent() && namespaceExists(session, namespace.get())) {
if ("information_schema".equals(namespace.get())) {
// TODO https://github.com/trinodb/trino/issues/1559 this should be filtered out in engine.
return ImmutableList.of();
}
return ImmutableList.of(namespace.get());
}
return listNamespaces(session);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -175,14 +175,9 @@ protected MockConnectorPlugin buildMockConnectorPlugin()
{
MockConnectorFactory connectorFactory = MockConnectorFactory.builder()
.withListSchemaNames(session -> ImmutableList.copyOf(mockTableListings.keySet()))
.withListTables((session, schemaName) -> {
if (schemaName.equals("information_schema")) {
// TODO (https://github.com/trinodb/trino/issues/1559) connector should not be asked about information_schema
return List.of();
}
return verifyNotNull(mockTableListings.get(schemaName), "No listing function registered for [%s]", schemaName)
.apply(session);
})
.withListTables((session, schemaName) ->
verifyNotNull(mockTableListings.get(schemaName), "No listing function registered for [%s]", schemaName)
.apply(session))
.build();
return new MockConnectorPlugin(connectorFactory);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -153,7 +153,7 @@ public void testMetadataCalls()
"VALUES 2",
new MetadataCallsCount()
.withListSchemasCount(1)
.withListTablesCount(3));
.withListTablesCount(2));
assertMetadataCalls(
"SELECT count(*) from test_catalog.information_schema.tables WHERE table_name LIKE 'test_t_ble1' AND table_name IN ('test_table1', 'test_table2')",
"VALUES 2",
Expand Down Expand Up @@ -188,14 +188,13 @@ public void testMetadataCalls()
"VALUES 1",
new MetadataCallsCount()
.withListSchemasCount(1)
.withListTablesCount(1)
.withGetColumnsCount(0));
assertMetadataCalls(
"SELECT count(*) FROM (SELECT * from test_catalog.information_schema.columns LIMIT 1000)",
"VALUES 1000",
new MetadataCallsCount()
.withListSchemasCount(1)
.withListTablesCount(2)
.withListTablesCount(1)
.withGetColumnsCount(1000));

// Empty table schema and table name
Expand Down

0 comments on commit 0b398f9

Please sign in to comment.