Skip to content

Commit

Permalink
Add reference view information to TableInfo
Browse files Browse the repository at this point in the history
  • Loading branch information
xkrogen committed Jan 30, 2024
1 parent e75754e commit 2582208
Show file tree
Hide file tree
Showing 16 changed files with 835 additions and 58 deletions.
64 changes: 53 additions & 11 deletions core/trino-main/src/main/java/io/trino/sql/analyzer/Analysis.java
Original file line number Diff line number Diff line change
Expand Up @@ -40,10 +40,16 @@
import io.trino.spi.connector.ColumnSchema;
import io.trino.spi.connector.ConnectorTableMetadata;
import io.trino.spi.connector.ConnectorTransactionHandle;
import io.trino.spi.eventlistener.BaseViewReferenceInfo;
import io.trino.spi.eventlistener.ColumnDetail;
import io.trino.spi.eventlistener.ColumnInfo;
import io.trino.spi.eventlistener.ColumnMaskReferenceInfo;
import io.trino.spi.eventlistener.MaterializedViewReferenceInfo;
import io.trino.spi.eventlistener.RoutineInfo;
import io.trino.spi.eventlistener.RowFilterReferenceInfo;
import io.trino.spi.eventlistener.TableInfo;
import io.trino.spi.eventlistener.TableReferenceInfo;
import io.trino.spi.eventlistener.ViewReferenceInfo;
import io.trino.spi.function.table.Argument;
import io.trino.spi.function.table.ConnectorTableFunctionHandle;
import io.trino.spi.security.Identity;
Expand Down Expand Up @@ -237,6 +243,8 @@ public class Analysis
// for recursive view detection
private final Deque<Table> tablesForView = new ArrayDeque<>();

private final Deque<TableReferenceInfo> referenceChain = new ArrayDeque<>();

// row id field for update/delete queries
private final Map<NodeRef<Table>, FieldReference> rowIdField = new LinkedHashMap<>();
private final Multimap<Field, SourceColumn> originColumnDetails = ArrayListMultimap.create();
Expand Down Expand Up @@ -638,7 +646,8 @@ public void registerTable(
Optional<TableHandle> handle,
QualifiedObjectName name,
String authorization,
Scope accessControlScope)
Scope accessControlScope,
Optional<String> viewText)
{
tables.put(
NodeRef.of(table),
Expand All @@ -649,7 +658,9 @@ public void registerTable(
accessControlScope,
tablesForView.isEmpty() &&
rowFilterScopes.isEmpty() &&
columnMaskScopes.isEmpty()));
columnMaskScopes.isEmpty(),
viewText,
referenceChain));
}

public Set<ResolvedFunction> getResolvedFunctions()
Expand Down Expand Up @@ -868,14 +879,23 @@ public Optional<Scope> getExpandableBaseScope(Node node)
return Optional.ofNullable(expandableBaseScopes.get(NodeRef.of(node)));
}

public void registerTableForView(Table tableReference)
public void registerTableForView(Table tableReference, QualifiedObjectName name, boolean isMaterializedView)
{
tablesForView.push(requireNonNull(tableReference, "tableReference is null"));
BaseViewReferenceInfo referenceInfo;
if (isMaterializedView) {
referenceInfo = new MaterializedViewReferenceInfo(name.getCatalogName(), name.getSchemaName(), name.getObjectName());
}
else {
referenceInfo = new ViewReferenceInfo(name.getCatalogName(), name.getSchemaName(), name.getObjectName());
}
referenceChain.push(referenceInfo);
}

public void unregisterTableForView()
{
tablesForView.pop();
referenceChain.pop();
}

public boolean hasTableInView(Table tableReference)
Expand Down Expand Up @@ -1089,14 +1109,16 @@ public boolean hasRowFilter(QualifiedObjectName table, String identity)
return rowFilterScopes.contains(new RowFilterScopeEntry(table, identity));
}

public void registerTableForRowFiltering(QualifiedObjectName table, String identity)
public void registerTableForRowFiltering(QualifiedObjectName table, String identity, String filterExpression)
{
rowFilterScopes.add(new RowFilterScopeEntry(table, identity));
referenceChain.push(new RowFilterReferenceInfo(filterExpression, table.getCatalogName(), table.getSchemaName(), table.getObjectName()));
}

public void unregisterTableForRowFiltering(QualifiedObjectName table, String identity)
{
rowFilterScopes.remove(new RowFilterScopeEntry(table, identity));
referenceChain.pop();
}

public void addRowFilter(Table table, Expression filter)
Expand Down Expand Up @@ -1126,14 +1148,16 @@ public boolean hasColumnMask(QualifiedObjectName table, String column, String id
return columnMaskScopes.contains(new ColumnMaskScopeEntry(table, column, identity));
}

public void registerTableForColumnMasking(QualifiedObjectName table, String column, String identity)
public void registerTableForColumnMasking(QualifiedObjectName table, String column, String identity, String maskExpression)
{
columnMaskScopes.add(new ColumnMaskScopeEntry(table, column, identity));
referenceChain.push(new ColumnMaskReferenceInfo(maskExpression, table.getCatalogName(), table.getSchemaName(), table.getObjectName(), column));
}

public void unregisterTableForColumnMasking(QualifiedObjectName table, String column, String identity)
{
columnMaskScopes.remove(new ColumnMaskScopeEntry(table, column, identity));
referenceChain.pop();
}

public void addColumnMask(Table table, String column, Expression mask)
Expand Down Expand Up @@ -1178,7 +1202,9 @@ public List<TableInfo> getReferencedTables()
.map(Expression::toString)
.collect(toImmutableList()),
columns,
info.isDirectlyReferenced());
info.isDirectlyReferenced(),
info.getViewText(),
info.getReferenceChain());
})
.collect(toImmutableList());
}
Expand Down Expand Up @@ -1993,19 +2019,25 @@ private static class TableEntry
private final String authorization;
private final Scope accessControlScope; // synthetic scope for analysis of row filters and masks
private final boolean directlyReferenced;
private final Optional<String> viewText;
private final List<TableReferenceInfo> referenceChain;

public TableEntry(
Optional<TableHandle> handle,
QualifiedObjectName name,
String authorization,
Scope accessControlScope,
boolean directlyReferenced)
boolean directlyReferenced,
Optional<String> viewText,
Iterable<TableReferenceInfo> referenceChain)
{
this.handle = requireNonNull(handle, "handle is null");
this.name = requireNonNull(name, "name is null");
this.authorization = requireNonNull(authorization, "authorization is null");
this.accessControlScope = requireNonNull(accessControlScope, "accessControlScope is null");
this.directlyReferenced = directlyReferenced;
this.viewText = requireNonNull(viewText, "viewText is null");
this.referenceChain = ImmutableList.copyOf(requireNonNull(referenceChain, "referenceChain is null"));
}

public Optional<TableHandle> getHandle()
Expand All @@ -2018,19 +2050,29 @@ public QualifiedObjectName getName()
return name;
}

public String getAuthorization()
{
return authorization;
}

public Scope getAccessControlScope()
{
return accessControlScope;
}

public boolean isDirectlyReferenced()
{
return directlyReferenced;
}

public String getAuthorization()
public Optional<String> getViewText()
{
return authorization;
return viewText;
}

public Scope getAccessControlScope()
public List<TableReferenceInfo> getReferenceChain()
{
return accessControlScope;
return referenceChain;
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -606,7 +606,7 @@ protected Scope visitInsert(Insert insert, Optional<Scope> scope)
.build();
analyzeFiltersAndMasks(insert.getTable(), targetTable, new RelationType(tableFields), accessControlScope);
analyzeCheckConstraints(insert.getTable(), targetTable, accessControlScope, checkConstraints);
analysis.registerTable(insert.getTable(), targetTableHandle, targetTable, session.getIdentity().getUser(), accessControlScope);
analysis.registerTable(insert.getTable(), targetTableHandle, targetTable, session.getIdentity().getUser(), accessControlScope, Optional.empty());

List<String> tableColumns = columns.stream()
.map(ColumnSchema::getName)
Expand Down Expand Up @@ -852,7 +852,7 @@ protected Scope visitDelete(Delete node, Optional<Scope> scope)
.build();
analyzeFiltersAndMasks(table, tableName, analysis.getScope(table).getRelationType(), accessControlScope);
analyzeCheckConstraints(table, tableName, accessControlScope, tableSchema.getTableSchema().getCheckConstraints());
analysis.registerTable(table, Optional.of(handle), tableName, session.getIdentity().getUser(), accessControlScope);
analysis.registerTable(table, Optional.of(handle), tableName, session.getIdentity().getUser(), accessControlScope, Optional.empty());

createMergeAnalysis(table, handle, tableSchema, tableScope, tableScope, ImmutableList.of());

Expand Down Expand Up @@ -2318,7 +2318,7 @@ protected Scope visitTable(Table table, Optional<Scope> scope)
.withRelationType(RelationId.anonymous(), new RelationType(outputFields))
.build();
analyzeFiltersAndMasks(table, targetTableName, new RelationType(outputFields), accessControlScope);
analysis.registerTable(table, tableHandle, targetTableName, session.getIdentity().getUser(), accessControlScope);
analysis.registerTable(table, tableHandle, targetTableName, session.getIdentity().getUser(), accessControlScope, Optional.empty());

Scope tableScope = createAndAssignScope(table, scope, outputFields);

Expand Down Expand Up @@ -2482,7 +2482,8 @@ private Scope createScopeForMaterializedView(Table table, QualifiedObjectName na
view.getRunAsIdentity(),
view.getPath(),
view.getColumns(),
storageTable);
storageTable,
true);
}

private Scope createScopeForView(Table table, QualifiedObjectName name, Optional<Scope> scope, ViewDefinition view)
Expand All @@ -2496,7 +2497,8 @@ private Scope createScopeForView(Table table, QualifiedObjectName name, Optional
view.getRunAsIdentity(),
view.getPath(),
view.getColumns(),
Optional.empty());
Optional.empty(),
false);
}

private Scope createScopeForView(
Expand All @@ -2509,7 +2511,8 @@ private Scope createScopeForView(
Optional<Identity> owner,
List<CatalogSchemaName> path,
List<ViewColumn> columns,
Optional<TableHandle> storageTable)
Optional<TableHandle> storageTable,
boolean isMaterializedView)
{
Statement statement = analysis.getStatement();
if (statement instanceof CreateView viewStatement) {
Expand All @@ -2534,7 +2537,7 @@ private Scope createScopeForView(
throw semanticException(NOT_SUPPORTED, table, "View contains inline function: %s", name);
}

analysis.registerTableForView(table);
analysis.registerTableForView(table, name, isMaterializedView);
RelationType descriptor = analyzeView(query, name, catalog, schema, owner, path, table);
analysis.unregisterTableForView();

Expand All @@ -2561,7 +2564,7 @@ private Scope createScopeForView(
.withRelationType(RelationId.anonymous(), new RelationType(viewFields))
.build();
analyzeFiltersAndMasks(table, name, new RelationType(viewFields), accessControlScope);
analysis.registerTable(table, storageTable, name, session.getIdentity().getUser(), accessControlScope);
analysis.registerTable(table, storageTable, name, session.getIdentity().getUser(), accessControlScope, Optional.of(originalSql));
analysis.addRelationCoercion(table, viewFields.stream().map(Field::getType).toArray(Type[]::new));
// use storage table output fields as they contain ColumnHandles
return createAndAssignScope(table, scope, storageTableFields);
Expand All @@ -2571,7 +2574,7 @@ private Scope createScopeForView(
.withRelationType(RelationId.anonymous(), new RelationType(viewFields))
.build();
analyzeFiltersAndMasks(table, name, new RelationType(viewFields), accessControlScope);
analysis.registerTable(table, storageTable, name, session.getIdentity().getUser(), accessControlScope);
analysis.registerTable(table, storageTable, name, session.getIdentity().getUser(), accessControlScope, Optional.of(originalSql));
viewFields.forEach(field -> analysis.addSourceColumns(field, ImmutableSet.of(new SourceColumn(name, field.getName().orElseThrow()))));
analysis.registerNamedQuery(table, query);
return createAndAssignScope(table, scope, viewFields);
Expand Down Expand Up @@ -3402,7 +3405,7 @@ protected Scope visitUpdate(Update update, Optional<Scope> scope)
Scope tableScope = analyzer.analyzeForUpdate(table, scope, UpdateKind.UPDATE);
update.getWhere().ifPresent(where -> analyzeWhere(update, tableScope, where));
analyzeCheckConstraints(table, tableName, tableScope, tableSchema.getTableSchema().getCheckConstraints());
analysis.registerTable(table, redirection.tableHandle(), tableName, session.getIdentity().getUser(), tableScope);
analysis.registerTable(table, redirection.tableHandle(), tableName, session.getIdentity().getUser(), tableScope, Optional.empty());

ImmutableList.Builder<ExpressionAnalysis> analysesBuilder = ImmutableList.builder();
ImmutableList.Builder<Type> expressionTypesBuilder = ImmutableList.builder();
Expand Down Expand Up @@ -3522,7 +3525,7 @@ protected Scope visitMerge(Merge merge, Optional<Scope> scope)
Scope sourceTableScope = process(merge.getSource(), mergeScope);
Scope joinScope = createAndAssignScope(merge, Optional.of(mergeScope), targetTableScope.getRelationType().joinWith(sourceTableScope.getRelationType()));
analyzeCheckConstraints(table, tableName, targetTableScope, tableSchema.getTableSchema().getCheckConstraints());
analysis.registerTable(table, redirection.tableHandle(), tableName, session.getIdentity().getUser(), targetTableScope);
analysis.registerTable(table, redirection.tableHandle(), tableName, session.getIdentity().getUser(), targetTableScope, Optional.empty());

for (ColumnSchema column : dataColumnSchemas) {
if (accessControl.getColumnMask(session.toSecurityContext(), tableName, column.getName(), column.getType()).isPresent()) {
Expand Down Expand Up @@ -5071,7 +5074,7 @@ private void analyzeRowFilter(String currentIdentity, Table table, QualifiedObje
throw new TrinoException(INVALID_ROW_FILTER, extractLocation(table), format("Invalid row filter for '%s': %s", name, e.getErrorMessage()), e);
}

analysis.registerTableForRowFiltering(name, currentIdentity);
analysis.registerTableForRowFiltering(name, currentIdentity, expression.toString());

verifyNoAggregateWindowOrGroupingFunctions(session, functionResolver, accessControl, expression, format("Row filter for '%s'", name));

Expand Down Expand Up @@ -5190,7 +5193,7 @@ private void analyzeColumnMask(String currentIdentity, Table table, QualifiedObj
}

ExpressionAnalysis expressionAnalysis;
analysis.registerTableForColumnMasking(tableName, column, currentIdentity);
analysis.registerTableForColumnMasking(tableName, column, currentIdentity, expression.toString());

verifyNoAggregateWindowOrGroupingFunctions(session, functionResolver, accessControl, expression, format("Column mask for '%s.%s'", table.getName(), column));

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,54 @@
/*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.trino.spi.eventlistener;

import com.fasterxml.jackson.annotation.JsonProperty;

import static java.util.Objects.requireNonNull;

/**
* This class is JSON serializable for convenience and serialization compatibility is not guaranteed across versions.
*/
public abstract class BaseViewReferenceInfo
implements TableReferenceInfo
{
private final String catalogName;
private final String schemaName;
private final String viewName;

protected BaseViewReferenceInfo(String catalogName, String schemaName, String viewName)
{
this.catalogName = requireNonNull(catalogName, "catalogName is null");
this.schemaName = requireNonNull(schemaName, "schemaName is null");
this.viewName = requireNonNull(viewName, "viewName is null");
}

@JsonProperty
public String getCatalogName()
{
return catalogName;
}

@JsonProperty
public String getSchemaName()
{
return schemaName;
}

@JsonProperty
public String getViewName()
{
return viewName;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
/*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.trino.spi.eventlistener;

import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;

/**
* This class is JSON serializable for convenience and serialization compatibility is not guaranteed across versions.
*/
public class ColumnMaskReferenceInfo
extends FilterMaskReferenceInfo
{
private final String columnName;

@JsonCreator
public ColumnMaskReferenceInfo(String maskExpression, String targetCatalogName, String targetSchemaName, String targetTableName, String targetColumnName)
{
super(maskExpression, targetCatalogName, targetSchemaName, targetTableName);
this.columnName = targetColumnName;
}

@JsonProperty
public String getTargetColumnName()
{
return columnName;
}
}
Loading

0 comments on commit 2582208

Please sign in to comment.