From 467dd8b0f884ace9e6767b0403fb59dc9c6f5178 Mon Sep 17 00:00:00 2001 From: Erik Krogen Date: Mon, 11 Sep 2023 10:18:42 -0700 Subject: [PATCH] Add reference view information to TableInfo --- .../java/io/trino/sql/analyzer/Analysis.java | 64 +++- .../trino/sql/analyzer/StatementAnalyzer.java | 29 +- .../eventlistener/BaseViewReferenceInfo.java | 54 ++++ .../ColumnMaskReferenceInfo.java | 39 +++ .../FilterMaskReferenceInfo.java | 60 ++++ .../MaterializedViewReferenceInfo.java | 29 ++ .../eventlistener/RowFilterReferenceInfo.java | 29 ++ .../io/trino/spi/eventlistener/TableInfo.java | 59 +++- .../spi/eventlistener/TableReferenceInfo.java | 34 +++ .../spi/eventlistener/ViewReferenceInfo.java | 29 ++ .../BaseViewReferenceInfoAssert.java | 36 +++ .../FilterMaskReferenceInfoAssert.java | 51 ++++ .../common/assertions/TableInfoAssert.java | 24 ++ .../assertions/TableReferenceInfoAssert.java | 58 ++++ .../common/assertions/TrinoAssertions.java | 18 ++ .../execution/TestEventListenerBasic.java | 280 +++++++++++++++--- 16 files changed, 835 insertions(+), 58 deletions(-) create mode 100644 core/trino-spi/src/main/java/io/trino/spi/eventlistener/BaseViewReferenceInfo.java create mode 100644 core/trino-spi/src/main/java/io/trino/spi/eventlistener/ColumnMaskReferenceInfo.java create mode 100644 core/trino-spi/src/main/java/io/trino/spi/eventlistener/FilterMaskReferenceInfo.java create mode 100644 core/trino-spi/src/main/java/io/trino/spi/eventlistener/MaterializedViewReferenceInfo.java create mode 100644 core/trino-spi/src/main/java/io/trino/spi/eventlistener/RowFilterReferenceInfo.java create mode 100644 core/trino-spi/src/main/java/io/trino/spi/eventlistener/TableReferenceInfo.java create mode 100644 core/trino-spi/src/main/java/io/trino/spi/eventlistener/ViewReferenceInfo.java create mode 100644 testing/trino-tests/src/test/java/io/trino/common/assertions/BaseViewReferenceInfoAssert.java create mode 100644 testing/trino-tests/src/test/java/io/trino/common/assertions/FilterMaskReferenceInfoAssert.java create mode 100644 testing/trino-tests/src/test/java/io/trino/common/assertions/TableReferenceInfoAssert.java diff --git a/core/trino-main/src/main/java/io/trino/sql/analyzer/Analysis.java b/core/trino-main/src/main/java/io/trino/sql/analyzer/Analysis.java index 30b88643bde3..4814c224ad5c 100644 --- a/core/trino-main/src/main/java/io/trino/sql/analyzer/Analysis.java +++ b/core/trino-main/src/main/java/io/trino/sql/analyzer/Analysis.java @@ -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; @@ -237,6 +243,8 @@ public class Analysis // for recursive view detection private final Deque tablesForView = new ArrayDeque<>(); + private final Deque referenceChain = new ArrayDeque<>(); + // row id field for update/delete queries private final Map, FieldReference> rowIdField = new LinkedHashMap<>(); private final Multimap originColumnDetails = ArrayListMultimap.create(); @@ -638,7 +646,8 @@ public void registerTable( Optional handle, QualifiedObjectName name, String authorization, - Scope accessControlScope) + Scope accessControlScope, + Optional viewText) { tables.put( NodeRef.of(table), @@ -649,7 +658,9 @@ public void registerTable( accessControlScope, tablesForView.isEmpty() && rowFilterScopes.isEmpty() && - columnMaskScopes.isEmpty())); + columnMaskScopes.isEmpty(), + viewText, + referenceChain)); } public Set getResolvedFunctions() @@ -868,14 +879,23 @@ public Optional 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) @@ -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) @@ -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) @@ -1178,7 +1202,9 @@ public List getReferencedTables() .map(Expression::toString) .collect(toImmutableList()), columns, - info.isDirectlyReferenced()); + info.isDirectlyReferenced(), + info.getViewText(), + info.getReferenceChain()); }) .collect(toImmutableList()); } @@ -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 viewText; + private final List referenceChain; public TableEntry( Optional handle, QualifiedObjectName name, String authorization, Scope accessControlScope, - boolean directlyReferenced) + boolean directlyReferenced, + Optional viewText, + Iterable 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 getHandle() @@ -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 getViewText() { - return authorization; + return viewText; } - public Scope getAccessControlScope() + public List getReferenceChain() { - return accessControlScope; + return referenceChain; } } diff --git a/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java b/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java index 943af84aa22f..14cd300fbfee 100644 --- a/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java +++ b/core/trino-main/src/main/java/io/trino/sql/analyzer/StatementAnalyzer.java @@ -606,7 +606,7 @@ protected Scope visitInsert(Insert insert, Optional 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 tableColumns = columns.stream() .map(ColumnSchema::getName) @@ -852,7 +852,7 @@ protected Scope visitDelete(Delete node, Optional 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()); @@ -2318,7 +2318,7 @@ protected Scope visitTable(Table table, Optional 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); @@ -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, ViewDefinition view) @@ -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( @@ -2509,7 +2511,8 @@ private Scope createScopeForView( Optional owner, List path, List columns, - Optional storageTable) + Optional storageTable, + boolean isMaterializedView) { Statement statement = analysis.getStatement(); if (statement instanceof CreateView viewStatement) { @@ -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(); @@ -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); @@ -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); @@ -3404,7 +3407,7 @@ protected Scope visitUpdate(Update update, Optional 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 analysesBuilder = ImmutableList.builder(); ImmutableList.Builder expressionTypesBuilder = ImmutableList.builder(); @@ -3524,7 +3527,7 @@ protected Scope visitMerge(Merge merge, Optional 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()) { @@ -5073,7 +5076,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)); @@ -5192,7 +5195,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)); diff --git a/core/trino-spi/src/main/java/io/trino/spi/eventlistener/BaseViewReferenceInfo.java b/core/trino-spi/src/main/java/io/trino/spi/eventlistener/BaseViewReferenceInfo.java new file mode 100644 index 000000000000..019aec26ae95 --- /dev/null +++ b/core/trino-spi/src/main/java/io/trino/spi/eventlistener/BaseViewReferenceInfo.java @@ -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; + } +} diff --git a/core/trino-spi/src/main/java/io/trino/spi/eventlistener/ColumnMaskReferenceInfo.java b/core/trino-spi/src/main/java/io/trino/spi/eventlistener/ColumnMaskReferenceInfo.java new file mode 100644 index 000000000000..fcd3608a8821 --- /dev/null +++ b/core/trino-spi/src/main/java/io/trino/spi/eventlistener/ColumnMaskReferenceInfo.java @@ -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; + } +} diff --git a/core/trino-spi/src/main/java/io/trino/spi/eventlistener/FilterMaskReferenceInfo.java b/core/trino-spi/src/main/java/io/trino/spi/eventlistener/FilterMaskReferenceInfo.java new file mode 100644 index 000000000000..5ad16a0d73ed --- /dev/null +++ b/core/trino-spi/src/main/java/io/trino/spi/eventlistener/FilterMaskReferenceInfo.java @@ -0,0 +1,60 @@ +/* + * 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; + +/** + * This class is JSON serializable for convenience and serialization compatibility is not guaranteed across versions. + */ +public abstract class FilterMaskReferenceInfo + implements TableReferenceInfo +{ + private final String expression; + private final String targetCatalogName; + private final String targetSchemaName; + private final String targetTableName; + + protected FilterMaskReferenceInfo(String expression, String targetCatalogName, String targetSchemaName, String targetTableName) + { + this.expression = expression; + this.targetCatalogName = targetCatalogName; + this.targetSchemaName = targetSchemaName; + this.targetTableName = targetTableName; + } + + @JsonProperty + public String getExpression() + { + return expression; + } + + @JsonProperty + public String getTargetCatalogName() + { + return targetCatalogName; + } + + @JsonProperty + public String getTargetSchemaName() + { + return targetSchemaName; + } + + @JsonProperty + public String getTargetTableName() + { + return targetTableName; + } +} diff --git a/core/trino-spi/src/main/java/io/trino/spi/eventlistener/MaterializedViewReferenceInfo.java b/core/trino-spi/src/main/java/io/trino/spi/eventlistener/MaterializedViewReferenceInfo.java new file mode 100644 index 000000000000..356450a441f1 --- /dev/null +++ b/core/trino-spi/src/main/java/io/trino/spi/eventlistener/MaterializedViewReferenceInfo.java @@ -0,0 +1,29 @@ +/* + * 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; + +/** + * This class is JSON serializable for convenience and serialization compatibility is not guaranteed across versions. + */ +public class MaterializedViewReferenceInfo + extends BaseViewReferenceInfo +{ + @JsonCreator + public MaterializedViewReferenceInfo(String catalogName, String schemaName, String viewName) + { + super(catalogName, schemaName, viewName); + } +} diff --git a/core/trino-spi/src/main/java/io/trino/spi/eventlistener/RowFilterReferenceInfo.java b/core/trino-spi/src/main/java/io/trino/spi/eventlistener/RowFilterReferenceInfo.java new file mode 100644 index 000000000000..11f28abbfd24 --- /dev/null +++ b/core/trino-spi/src/main/java/io/trino/spi/eventlistener/RowFilterReferenceInfo.java @@ -0,0 +1,29 @@ +/* + * 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; + +/** + * This class is JSON serializable for convenience and serialization compatibility is not guaranteed across versions. + */ +public class RowFilterReferenceInfo + extends FilterMaskReferenceInfo +{ + @JsonCreator + public RowFilterReferenceInfo(String filterExpression, String targetCatalogName, String targetSchemaName, String targetTableName) + { + super(filterExpression, targetCatalogName, targetSchemaName, targetTableName); + } +} diff --git a/core/trino-spi/src/main/java/io/trino/spi/eventlistener/TableInfo.java b/core/trino-spi/src/main/java/io/trino/spi/eventlistener/TableInfo.java index 129970ca1227..e9ad7dd2de31 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/eventlistener/TableInfo.java +++ b/core/trino-spi/src/main/java/io/trino/spi/eventlistener/TableInfo.java @@ -18,6 +18,7 @@ import io.trino.spi.Unstable; import java.util.List; +import java.util.Optional; import static java.util.Objects.requireNonNull; @@ -34,10 +35,21 @@ public class TableInfo private final List filters; private final List columns; private final boolean directlyReferenced; + private final Optional viewText; + private final List referenceChain; @JsonCreator @Unstable - public TableInfo(String catalog, String schema, String table, String authorization, List filters, List columns, boolean directlyReferenced) + public TableInfo( + String catalog, + String schema, + String table, + String authorization, + List filters, + List columns, + boolean directlyReferenced, + Optional viewText, + List referenceChain) { this.catalog = requireNonNull(catalog, "catalog is null"); this.schema = requireNonNull(schema, "schema is null"); @@ -46,6 +58,8 @@ public TableInfo(String catalog, String schema, String table, String authorizati this.filters = List.copyOf(requireNonNull(filters, "filters is null")); this.columns = List.copyOf(requireNonNull(columns, "columns is null")); this.directlyReferenced = directlyReferenced; + this.viewText = requireNonNull(viewText, "viewText is null"); + this.referenceChain = List.copyOf(requireNonNull(referenceChain, "referenceChain is null")); } @JsonProperty @@ -89,4 +103,47 @@ public boolean isDirectlyReferenced() { return directlyReferenced; } + + /** + * If this object refers to a view or materialized view, return the SQL text of the view. + * + * @return The SQL text of the view, or empty if this object does not refer to a view. + */ + @JsonProperty + public Optional getViewText() + { + return viewText; + } + + /** + * Get the reference chain for this table. Contains entries for view-like expressions such as views, column masks, and row filters. + * Note that in contrast to {@link #getFilters()} and {@link #getColumns()}, this has information about the reference chain leading + * this table to be included in the query, rather than information about which filters/masks were applied to this table scan. + *

+ * Let us assume the following setup, with default catalog "default" and default schema "db": + *

+     * CREATE TABLE t_base (a INT, b INT);
+     * CREATE VIEW v1 AS SELECT * FROM t_base;
+     * CREATE VIEW v2 AS SELECT * FROM v1;
+     *
+     * Row filter for t_base:      a NOT IN (SELECT * FROM t_filter)
+     * Column mask for t_base.b:   IF(b IN (SELECT * FROM t_mask), b, NULL)
+     * 
+ * If we execute {@code SELECT * FROM v2}, we will see the following values for {@code referenceChain}: + *
+     * v2 -> []
+     * v1 -> [{"view", "default", "db", "v2"}]
+     * t_base -> [{"view", "default", "db", "v2"}, {"view", "default", "db", "v1"}]
+     * t_filter -> [{"view", "default", "db", "v2"}, {"view", "default", "db", "v1"}, {"rowFilter", "a NOT IN (SELECT * FROM t_filter)", "default", "db", "t_base"}]
+     * t_mask -> [{"view", "default", "db", "v2"}, {"view", "default", "db", "v1"}, {"columnMask", "IF(b IN SELECT * FROM t_mask, b, NULL)", "default", "db", "t_base"}]
+     * 
+ *

+ * + * @return The reference chain leading to this table, in order. + */ + @JsonProperty + public List getReferenceChain() + { + return referenceChain; + } } diff --git a/core/trino-spi/src/main/java/io/trino/spi/eventlistener/TableReferenceInfo.java b/core/trino-spi/src/main/java/io/trino/spi/eventlistener/TableReferenceInfo.java new file mode 100644 index 000000000000..cb8890541d85 --- /dev/null +++ b/core/trino-spi/src/main/java/io/trino/spi/eventlistener/TableReferenceInfo.java @@ -0,0 +1,34 @@ +/* + * 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.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; + +/** + * This class is JSON serializable for convenience and serialization compatibility is not guaranteed across versions. + * + * @see TableInfo#getReferenceChain() + */ +@JsonTypeInfo( + use = JsonTypeInfo.Id.NAME, + property = "@type") +@JsonSubTypes({ + @JsonSubTypes.Type(value = ViewReferenceInfo.class, name = "view"), + @JsonSubTypes.Type(value = MaterializedViewReferenceInfo.class, name = "materializedView"), + @JsonSubTypes.Type(value = RowFilterReferenceInfo.class, name = "rowFilter"), + @JsonSubTypes.Type(value = ColumnMaskReferenceInfo.class, name = "columnMask")}) +public interface TableReferenceInfo +{ +} diff --git a/core/trino-spi/src/main/java/io/trino/spi/eventlistener/ViewReferenceInfo.java b/core/trino-spi/src/main/java/io/trino/spi/eventlistener/ViewReferenceInfo.java new file mode 100644 index 000000000000..b271f67c334d --- /dev/null +++ b/core/trino-spi/src/main/java/io/trino/spi/eventlistener/ViewReferenceInfo.java @@ -0,0 +1,29 @@ +/* + * 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; + +/** + * This class is JSON serializable for convenience and serialization compatibility is not guaranteed across versions. + */ +public class ViewReferenceInfo + extends BaseViewReferenceInfo +{ + @JsonCreator + public ViewReferenceInfo(String catalogName, String schemaName, String viewName) + { + super(catalogName, schemaName, viewName); + } +} diff --git a/testing/trino-tests/src/test/java/io/trino/common/assertions/BaseViewReferenceInfoAssert.java b/testing/trino-tests/src/test/java/io/trino/common/assertions/BaseViewReferenceInfoAssert.java new file mode 100644 index 000000000000..04ee69c0416e --- /dev/null +++ b/testing/trino-tests/src/test/java/io/trino/common/assertions/BaseViewReferenceInfoAssert.java @@ -0,0 +1,36 @@ +/* + * 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.common.assertions; + +import io.trino.spi.eventlistener.BaseViewReferenceInfo; +import org.assertj.core.api.AbstractAssert; + +import static org.assertj.core.api.Assertions.assertThat; + +public class BaseViewReferenceInfoAssert + extends AbstractAssert +{ + BaseViewReferenceInfoAssert(BaseViewReferenceInfo actual) + { + super(actual, BaseViewReferenceInfoAssert.class); + } + + public BaseViewReferenceInfoAssert hasCatalogSchemaView(String catalogName, String schemaName, String viewName) + { + assertThat(actual.getCatalogName()).isEqualTo(catalogName); + assertThat(actual.getSchemaName()).isEqualTo(schemaName); + assertThat(actual.getViewName()).isEqualTo(viewName); + return this; + } +} diff --git a/testing/trino-tests/src/test/java/io/trino/common/assertions/FilterMaskReferenceInfoAssert.java b/testing/trino-tests/src/test/java/io/trino/common/assertions/FilterMaskReferenceInfoAssert.java new file mode 100644 index 000000000000..20356508cc59 --- /dev/null +++ b/testing/trino-tests/src/test/java/io/trino/common/assertions/FilterMaskReferenceInfoAssert.java @@ -0,0 +1,51 @@ +/* + * 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.common.assertions; + +import io.trino.spi.eventlistener.ColumnMaskReferenceInfo; +import io.trino.spi.eventlistener.FilterMaskReferenceInfo; +import org.assertj.core.api.AbstractAssert; + +import static org.assertj.core.api.Assertions.assertThat; + +public class FilterMaskReferenceInfoAssert + extends AbstractAssert +{ + FilterMaskReferenceInfoAssert(FilterMaskReferenceInfo actual) + { + super(actual, FilterMaskReferenceInfoAssert.class); + } + + public FilterMaskReferenceInfoAssert hasExpression(String expression) + { + assertThat(actual.getExpression()).isEqualToIgnoringWhitespace(expression); + return this; + } + + public FilterMaskReferenceInfoAssert hasTargetCatalogSchemaTable(String catalogName, String schemaName, String tableName) + { + assertThat(actual.getTargetCatalogName()).isEqualTo(catalogName); + assertThat(actual.getTargetSchemaName()).isEqualTo(schemaName); + assertThat(actual.getTargetTableName()).isEqualTo(tableName); + return this; + } + + public FilterMaskReferenceInfoAssert hasTargetColumn(String maskedColumnName) + { + TrinoAssertions.assertThat(actual).isInstanceOfSatisfying( + ColumnMaskReferenceInfo.class, + columnMaskInfo -> assertThat(columnMaskInfo.getTargetColumnName()).isEqualTo(maskedColumnName)); + return this; + } +} diff --git a/testing/trino-tests/src/test/java/io/trino/common/assertions/TableInfoAssert.java b/testing/trino-tests/src/test/java/io/trino/common/assertions/TableInfoAssert.java index 83015743da19..8ba5028e7b62 100644 --- a/testing/trino-tests/src/test/java/io/trino/common/assertions/TableInfoAssert.java +++ b/testing/trino-tests/src/test/java/io/trino/common/assertions/TableInfoAssert.java @@ -15,9 +15,11 @@ import io.trino.spi.eventlistener.ColumnInfo; import io.trino.spi.eventlistener.TableInfo; +import io.trino.spi.eventlistener.TableReferenceInfo; import org.assertj.core.api.AbstractAssert; import java.util.Optional; +import java.util.function.Consumer; import static org.assertj.core.api.Assertions.assertThat; @@ -102,4 +104,26 @@ public TableInfoAssert hasRowFilters(String... filterTexts) } return this; } + + public TableInfoAssert hasViewText(String viewText) + { + assertThat(actual.getViewText()).hasValueSatisfying(sql -> assertThat(sql).isEqualToIgnoringWhitespace(viewText)); + return this; + } + + public TableInfoAssert hasNoTableReferences() + { + assertThat(actual.getReferenceChain()).isEmpty(); + return this; + } + + @SafeVarargs + public final TableInfoAssert hasTableReferencesSatisfying(Consumer... tableReferenceAssertions) + { + assertThat(actual.getReferenceChain()).hasSize(tableReferenceAssertions.length); + for (int i = 0; i < tableReferenceAssertions.length; i++) { + tableReferenceAssertions[i].accept(actual.getReferenceChain().get(i)); + } + return this; + } } diff --git a/testing/trino-tests/src/test/java/io/trino/common/assertions/TableReferenceInfoAssert.java b/testing/trino-tests/src/test/java/io/trino/common/assertions/TableReferenceInfoAssert.java new file mode 100644 index 000000000000..92af951823b3 --- /dev/null +++ b/testing/trino-tests/src/test/java/io/trino/common/assertions/TableReferenceInfoAssert.java @@ -0,0 +1,58 @@ +/* + * 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.common.assertions; + +import io.trino.spi.eventlistener.BaseViewReferenceInfo; +import io.trino.spi.eventlistener.ColumnMaskReferenceInfo; +import io.trino.spi.eventlistener.FilterMaskReferenceInfo; +import io.trino.spi.eventlistener.MaterializedViewReferenceInfo; +import io.trino.spi.eventlistener.RowFilterReferenceInfo; +import io.trino.spi.eventlistener.TableReferenceInfo; +import io.trino.spi.eventlistener.ViewReferenceInfo; +import org.assertj.core.api.AbstractAssert; + +import static io.trino.common.assertions.TrinoAssertions.assertThat; + +public class TableReferenceInfoAssert + extends AbstractAssert +{ + TableReferenceInfoAssert(TableReferenceInfo actual) + { + super(actual, TableReferenceInfoAssert.class); + } + + public BaseViewReferenceInfoAssert asViewInfo() + { + assertThat(actual).isInstanceOf(ViewReferenceInfo.class); + return new BaseViewReferenceInfoAssert((BaseViewReferenceInfo) actual); + } + + public BaseViewReferenceInfoAssert asMaterializedViewInfo() + { + assertThat(actual).isInstanceOf(MaterializedViewReferenceInfo.class); + return new BaseViewReferenceInfoAssert((BaseViewReferenceInfo) actual); + } + + public FilterMaskReferenceInfoAssert asRowFilterInfo() + { + assertThat(actual).isInstanceOf(RowFilterReferenceInfo.class); + return new FilterMaskReferenceInfoAssert((FilterMaskReferenceInfo) actual); + } + + public FilterMaskReferenceInfoAssert asColumnMaskInfo() + { + assertThat(actual).isInstanceOf(ColumnMaskReferenceInfo.class); + return new FilterMaskReferenceInfoAssert((FilterMaskReferenceInfo) actual); + } +} diff --git a/testing/trino-tests/src/test/java/io/trino/common/assertions/TrinoAssertions.java b/testing/trino-tests/src/test/java/io/trino/common/assertions/TrinoAssertions.java index 8c014a5282ac..2e37b15a2c66 100644 --- a/testing/trino-tests/src/test/java/io/trino/common/assertions/TrinoAssertions.java +++ b/testing/trino-tests/src/test/java/io/trino/common/assertions/TrinoAssertions.java @@ -13,7 +13,10 @@ */ package io.trino.common.assertions; +import io.trino.spi.eventlistener.BaseViewReferenceInfo; +import io.trino.spi.eventlistener.FilterMaskReferenceInfo; import io.trino.spi.eventlistener.TableInfo; +import io.trino.spi.eventlistener.TableReferenceInfo; public class TrinoAssertions { @@ -23,4 +26,19 @@ public static TableInfoAssert assertThat(TableInfo actual) { return new TableInfoAssert(actual); } + + public static TableReferenceInfoAssert assertThat(TableReferenceInfo actual) + { + return new TableReferenceInfoAssert(actual); + } + + public static BaseViewReferenceInfoAssert assertThat(BaseViewReferenceInfo actual) + { + return new BaseViewReferenceInfoAssert(actual); + } + + public static FilterMaskReferenceInfoAssert assertThat(FilterMaskReferenceInfo actual) + { + return new FilterMaskReferenceInfoAssert(actual); + } } diff --git a/testing/trino-tests/src/test/java/io/trino/execution/TestEventListenerBasic.java b/testing/trino-tests/src/test/java/io/trino/execution/TestEventListenerBasic.java index 3f47bdab27bb..fddd22253833 100644 --- a/testing/trino-tests/src/test/java/io/trino/execution/TestEventListenerBasic.java +++ b/testing/trino-tests/src/test/java/io/trino/execution/TestEventListenerBasic.java @@ -30,6 +30,7 @@ import io.trino.plugin.tpch.TpchPlugin; import io.trino.spi.Plugin; import io.trino.spi.QueryId; +import io.trino.spi.connector.CatalogSchemaTableName; import io.trino.spi.connector.ColumnMetadata; import io.trino.spi.connector.ConnectorFactory; import io.trino.spi.connector.ConnectorMaterializedViewDefinition; @@ -139,7 +140,7 @@ public Iterable getConnectorFactories() }; }) .withGetColumns(schemaTableName -> { - if (schemaTableName.equals(new SchemaTableName("tiny", "nation"))) { + if (schemaTableName.equals(new SchemaTableName("tiny", "nation")) || schemaTableName.equals(new SchemaTableName("tiny", "nation_storage"))) { return TPCH_NATION_SCHEMA; } return ImmutableList.of( @@ -158,35 +159,74 @@ public Iterable getConnectorFactories() } return Optional.empty(); }) - .withGetViews((connectorSession, prefix) -> { - ConnectorViewDefinition definition = new ConnectorViewDefinition( + .withGetViews((connectorSession, prefix) -> + ImmutableMap.of( + new SchemaTableName("default", "test_view"), new ConnectorViewDefinition( + "SELECT nationkey AS test_column FROM tpch.tiny.nation", + Optional.empty(), + Optional.empty(), + ImmutableList.of(new ConnectorViewDefinition.ViewColumn("test_column", BIGINT.getTypeId(), Optional.empty())), + Optional.empty(), + Optional.empty(), + true, + ImmutableList.of()), + new SchemaTableName("default", "test_view_nesting"), new ConnectorViewDefinition( + "SELECT test_column FROM mock.default.test_view", + Optional.empty(), + Optional.empty(), + ImmutableList.of(new ConnectorViewDefinition.ViewColumn("test_column", BIGINT.getTypeId(), Optional.empty())), + Optional.empty(), + Optional.empty(), + true, + ImmutableList.of()), + new SchemaTableName("default", "test_view_with_row_filter"), new ConnectorViewDefinition( + "SELECT test_varchar AS test_column FROM mock.default.test_table_with_row_filter", + Optional.empty(), + Optional.empty(), + ImmutableList.of(new ConnectorViewDefinition.ViewColumn("test_column", createVarcharType(15).getTypeId(), Optional.empty())), + Optional.empty(), + Optional.empty(), + true, + ImmutableList.of()), + new SchemaTableName("default", "test_view_with_redirect"), new ConnectorViewDefinition( + "SELECT nationkey AS test_column FROM mock.default.nation_redirect", + Optional.empty(), + Optional.empty(), + ImmutableList.of(new ConnectorViewDefinition.ViewColumn("test_column", BIGINT.getTypeId(), Optional.empty())), + Optional.empty(), + Optional.empty(), + true, + ImmutableList.of()))) + .withGetMaterializedViews((connectorSession, prefix) -> { + ConnectorMaterializedViewDefinition definitionStale = new ConnectorMaterializedViewDefinition( "SELECT nationkey AS test_column FROM tpch.tiny.nation", Optional.empty(), Optional.empty(), - ImmutableList.of(new ConnectorViewDefinition.ViewColumn("test_column", BIGINT.getTypeId(), Optional.empty())), Optional.empty(), + ImmutableList.of(new Column("test_column", BIGINT.getTypeId(), Optional.empty())), + Optional.of(Duration.ZERO), Optional.empty(), - true, + Optional.of("alice"), ImmutableList.of()); - SchemaTableName viewName = new SchemaTableName("default", "test_view"); - return ImmutableMap.of(viewName, definition); - }) - .withGetMaterializedViews((connectorSession, prefix) -> { - ConnectorMaterializedViewDefinition definition = new ConnectorMaterializedViewDefinition( - "SELECT nationkey AS test_column FROM tpch.tiny.nation", + ConnectorMaterializedViewDefinition definitionFresh = new ConnectorMaterializedViewDefinition( + "SELECT * FROM tpch.tiny.nation", + Optional.of(new CatalogSchemaTableName("mock", "tiny", "nation")), Optional.empty(), Optional.empty(), - Optional.empty(), - ImmutableList.of(new Column("test_column", BIGINT.getTypeId(), Optional.empty())), - Optional.of(Duration.ZERO), + TPCH_NATION_SCHEMA + .stream() + .map(column -> new Column(column.getName(), column.getType().getTypeId(), Optional.empty())) + .collect(toImmutableList()), + Optional.of(Duration.ofDays(1)), Optional.empty(), Optional.of("alice"), ImmutableList.of()); - SchemaTableName materializedViewName = new SchemaTableName("default", "test_materialized_view"); - return ImmutableMap.of(materializedViewName, definition); + return ImmutableMap.of( + new SchemaTableName("default", "test_materialized_view_stale"), definitionStale, + new SchemaTableName("default", "test_materialized_view_fresh"), definitionFresh); }) .withData(schemaTableName -> { - if (schemaTableName.equals(new SchemaTableName("tiny", "nation"))) { + if (schemaTableName.equals(new SchemaTableName("tiny", "nation")) || schemaTableName.equals(new SchemaTableName("tiny", "nation_storage"))) { return TPCH_NATION_DATA; } return ImmutableList.of(); @@ -219,6 +259,12 @@ public Iterable getConnectorFactories() } return null; }) + .withRedirectTable((session, schemaTableName) -> { + if (schemaTableName.getTableName().equals("nation_redirect")) { + return Optional.of(new CatalogSchemaTableName("tpch", "tiny", "nation")); + } + return Optional.empty(); + }) .build(); return ImmutableList.of(connectorFactory); } @@ -368,7 +414,8 @@ public void testReferencedTablesAndRoutines() .hasAuthorization("user") .isDirectlyReferenced() .hasColumnsWithoutMasking("linenumber") - .hasNoRowFilters(); + .hasNoRowFilters() + .hasNoTableReferences(); RoutineInfo routine = getOnlyElement(event.getMetadata().getRoutines()); assertThat(routine.getRoutine()).isEqualTo("sum"); @@ -392,7 +439,8 @@ public void testReferencedTablesWithViews() .hasAuthorization("user") .isNotDirectlyReferenced() .hasColumnsWithoutMasking("nationkey") - .hasNoRowFilters(); + .hasNoRowFilters() + .hasTableReferencesSatisfying(tableRef -> assertThat(tableRef).asViewInfo().hasCatalogSchemaView("mock", "default", "test_view")); table = tables.get(1); assertThat(table) @@ -400,14 +448,15 @@ public void testReferencedTablesWithViews() .hasAuthorization("user") .isDirectlyReferenced() .hasColumnsWithoutMasking("test_column") - .hasNoRowFilters(); + .hasNoRowFilters() + .hasNoTableReferences(); } @Test - public void testReferencedTablesWithMaterializedViews() + public void testReferencedTablesWithMaterializedViewsStale() throws Exception { - QueryEvents queryEvents = runQueryAndWaitForEvents("SELECT test_column FROM mock.default.test_materialized_view").getQueryEvents(); + QueryEvents queryEvents = runQueryAndWaitForEvents("SELECT test_column FROM mock.default.test_materialized_view_stale").getQueryEvents(); QueryCompletedEvent event = queryEvents.getQueryCompletedEvent(); @@ -419,15 +468,80 @@ public void testReferencedTablesWithMaterializedViews() .hasAuthorization("alice") .isNotDirectlyReferenced() .hasColumnsWithoutMasking("nationkey") - .hasNoRowFilters(); + .hasNoRowFilters() + .hasTableReferencesSatisfying(tableRef -> assertThat(tableRef).asMaterializedViewInfo().hasCatalogSchemaView("mock", "default", "test_materialized_view_stale")); + + table = tables.get(1); + assertThat(table) + .hasCatalogSchemaTable("mock", "default", "test_materialized_view_stale") + .hasAuthorization("user") + .isDirectlyReferenced() + .hasColumnsWithoutMasking("test_column") + .hasNoRowFilters() + .hasNoTableReferences() + .hasViewText("SELECT nationkey AS test_column FROM tpch.tiny.nation"); + } + + // Currently, the storage table for a materialized view is not included anywhere in the set of `tables` in the query event. + // See for more details: https://github.com/trinodb/trino/pull/18871#discussion_r1412247513 + @Test + public void testReferencedTablesWithMaterializedViewsFresh() + throws Exception + { + QueryEvents queryEvents = runQueryAndWaitForEvents("SELECT nationkey FROM mock.default.test_materialized_view_fresh").getQueryEvents(); + + QueryCompletedEvent event = queryEvents.getQueryCompletedEvent(); + + List tables = event.getMetadata().getTables(); + assertThat(tables).hasSize(2); + TableInfo table = tables.get(0); + assertThat(table) + .hasCatalogSchemaTable("tpch", "tiny", "nation") + .hasAuthorization("alice") + .isNotDirectlyReferenced() + .hasColumnsWithoutMasking("nationkey", "regionkey", "name", "comment") + .hasNoRowFilters() + .hasTableReferencesSatisfying(tableRef -> assertThat(tableRef).asMaterializedViewInfo().hasCatalogSchemaView("mock", "default", "test_materialized_view_fresh")); + + table = tables.get(1); + assertThat(table) + .hasCatalogSchemaTable("mock", "default", "test_materialized_view_fresh") + .hasAuthorization("user") + .isDirectlyReferenced() + .hasColumnsWithoutMasking("nationkey") + .hasNoRowFilters() + .hasNoTableReferences() + .hasViewText("SELECT * FROM tpch.tiny.nation"); + } + + @Test + public void testReferencedTablesWithViewsAndRedirection() + throws Exception + { + QueryEvents queryEvents = runQueryAndWaitForEvents("SELECT test_column FROM mock.default.test_view_with_redirect").getQueryEvents(); + + QueryCompletedEvent event = queryEvents.getQueryCompletedEvent(); + + List tables = event.getMetadata().getTables(); + assertThat(tables).hasSize(2); + + TableInfo table = tables.get(0); + assertThat(table) + .hasCatalogSchemaTable("tpch", "tiny", "nation") + .hasAuthorization("user") + .isNotDirectlyReferenced() + .hasColumnsWithoutMasking("nationkey") + .hasNoRowFilters() + .hasTableReferencesSatisfying(tableRef -> assertThat(tableRef).asViewInfo().hasCatalogSchemaView("mock", "default", "test_view_with_redirect")); table = tables.get(1); assertThat(table) - .hasCatalogSchemaTable("mock", "default", "test_materialized_view") + .hasCatalogSchemaTable("mock", "default", "test_view_with_redirect") .hasAuthorization("user") .isDirectlyReferenced() .hasColumnsWithoutMasking("test_column") - .hasNoRowFilters(); + .hasNoRowFilters() + .hasNoTableReferences(); } @Test @@ -454,7 +568,8 @@ public void testReferencedTablesInCreateView() .hasAuthorization("user") .isDirectlyReferenced() .hasColumnsWithoutMasking("nationkey", "regionkey", "name", "comment") - .hasNoRowFilters(); + .hasNoRowFilters() + .hasNoTableReferences(); } @Test @@ -481,7 +596,8 @@ public void testReferencedTablesInCreateMaterializedView() .hasAuthorization("user") .isDirectlyReferenced() .hasColumnsWithoutMasking("nationkey", "regionkey", "name", "comment") - .hasNoRowFilters(); + .hasNoRowFilters() + .hasNoTableReferences(); } @Test @@ -501,7 +617,12 @@ public void testReferencedTablesWithRowFilter() .hasAuthorization("user") .isNotDirectlyReferenced() .hasColumnsWithoutMasking("name") - .hasNoRowFilters(); + .hasNoRowFilters() + .hasTableReferencesSatisfying(tableRef -> + assertThat(tableRef) + .asRowFilterInfo() + .hasTargetCatalogSchemaTable("mock", "default", "test_table_with_row_filter") + .hasExpression("(EXISTS (SELECT 1 FROM nation WHERE (name = test_varchar)))")); table = tables.get(1); assertThat(table) @@ -509,7 +630,93 @@ public void testReferencedTablesWithRowFilter() .hasAuthorization("user") .isDirectlyReferenced() .hasColumnsWithoutMasking("test_varchar") - .hasRowFilters("(EXISTS (SELECT 1 FROM nation WHERE (name = test_varchar)))"); + .hasRowFilters("(EXISTS (SELECT 1 FROM nation WHERE (name = test_varchar)))") + .hasNoTableReferences(); + } + + @Test + public void testReferencedTablesWithNestedView() + throws Exception + { + QueryEvents queryEvents = runQueryAndWaitForEvents("SELECT 1 FROM mock.default.test_view_nesting").getQueryEvents(); + + QueryCompletedEvent event = queryEvents.getQueryCompletedEvent(); + + List tables = event.getMetadata().getTables(); + assertThat(tables).hasSize(3); + + TableInfo table = tables.get(0); + assertThat(table) + .hasCatalogSchemaTable("tpch", "tiny", "nation") + .hasAuthorization("user") + .isNotDirectlyReferenced() + .hasColumnsWithoutMasking("nationkey") + .hasNoRowFilters() + .hasTableReferencesSatisfying( + tableRef -> assertThat(tableRef).asViewInfo().hasCatalogSchemaView("mock", "default", "test_view"), + tableRef -> assertThat(tableRef).asViewInfo().hasCatalogSchemaView("mock", "default", "test_view_nesting")); + + table = tables.get(1); + assertThat(table) + .hasCatalogSchemaTable("mock", "default", "test_view") + .hasAuthorization("user") + .isNotDirectlyReferenced() + .hasColumnsWithoutMasking("test_column") + .hasNoRowFilters() + .hasTableReferencesSatisfying(tableRef -> assertThat(tableRef).asViewInfo().hasCatalogSchemaView("mock", "default", "test_view_nesting")); + + table = tables.get(2); + assertThat(table) + .hasCatalogSchemaTable("mock", "default", "test_view_nesting") + .hasAuthorization("user") + .isDirectlyReferenced() + .hasColumnsWithoutMasking() + .hasNoRowFilters() + .hasNoTableReferences(); + } + + @Test + public void testReferencedTablesWithRowFilterAndView() + throws Exception + { + QueryEvents queryEvents = runQueryAndWaitForEvents("SELECT 1 FROM mock.default.test_view_with_row_filter").getQueryEvents(); + + QueryCompletedEvent event = queryEvents.getQueryCompletedEvent(); + + List tables = event.getMetadata().getTables(); + assertThat(tables).hasSize(3); + + TableInfo table = tables.get(0); + assertThat(table) + .hasCatalogSchemaTable("tpch", "tiny", "nation") + .hasAuthorization("user") + .isNotDirectlyReferenced() + .hasColumnsWithoutMasking("name") + .hasNoRowFilters() + .hasTableReferencesSatisfying( + tableRef -> assertThat(tableRef) + .asRowFilterInfo() + .hasTargetCatalogSchemaTable("mock", "default", "test_table_with_row_filter") + .hasExpression("(EXISTS (SELECT 1 FROM nation WHERE (name = test_varchar)))"), + tableRef -> assertThat(tableRef).asViewInfo().hasCatalogSchemaView("mock", "default", "test_view_with_row_filter")); + + table = tables.get(1); + assertThat(table) + .hasCatalogSchemaTable("mock", "default", "test_table_with_row_filter") + .hasAuthorization("user") + .isNotDirectlyReferenced() + .hasColumnsWithoutMasking("test_varchar") + .hasRowFilters("(EXISTS (SELECT 1 FROM nation WHERE (name = test_varchar)))") + .hasTableReferencesSatisfying(tableRef -> assertThat(tableRef).asViewInfo().hasCatalogSchemaView("mock", "default", "test_view_with_row_filter")); + + table = tables.get(2); + assertThat(table) + .hasCatalogSchemaTable("mock", "default", "test_view_with_row_filter") + .hasAuthorization("user") + .isDirectlyReferenced() + .hasColumnsWithoutMasking() + .hasNoRowFilters() + .hasNoTableReferences(); } @Test @@ -539,7 +746,13 @@ public void testReferencedTablesWithColumnMask() .hasAuthorization("user") .isNotDirectlyReferenced() .hasColumnsWithoutMasking("orderkey") - .hasNoRowFilters(); + .hasNoRowFilters() + .hasTableReferencesSatisfying(tableRef -> + assertThat(tableRef) + .asColumnMaskInfo() + .hasTargetCatalogSchemaTable("mock", "default", "test_table_with_column_mask") + .hasExpression("(SELECT CAST(max(orderkey) AS varchar(15)) FROM orders)") + .hasTargetColumn("test_varchar")); table = tables.get(1); assertThat(table) @@ -548,7 +761,8 @@ public void testReferencedTablesWithColumnMask() .isDirectlyReferenced() .hasColumnNames("test_varchar", "test_bigint") .hasColumnMasks("(SELECT CAST(max(orderkey) AS varchar(15)) FROM orders)", null) - .hasNoRowFilters(); + .hasNoRowFilters() + .hasNoTableReferences(); } @Test @@ -741,11 +955,11 @@ public void testOutputColumnsForCreateTableAsSelectAllFromView() public void testOutputColumnsForCreateTableAsSelectAllFromMaterializedView() throws Exception { - QueryEvents queryEvents = runQueryAndWaitForEvents("CREATE TABLE mock.default.create_new_table AS SELECT * FROM mock.default.test_materialized_view").getQueryEvents(); + QueryEvents queryEvents = runQueryAndWaitForEvents("CREATE TABLE mock.default.create_new_table AS SELECT * FROM mock.default.test_materialized_view_stale").getQueryEvents(); QueryCompletedEvent event = queryEvents.getQueryCompletedEvent(); assertThat(event.getIoMetadata().getOutput().get().getColumns().get()) .containsExactly( - new OutputColumnMetadata("test_column", BIGINT_TYPE, ImmutableSet.of(new ColumnDetail("mock", "default", "test_materialized_view", "test_column")))); + new OutputColumnMetadata("test_column", BIGINT_TYPE, ImmutableSet.of(new ColumnDetail("mock", "default", "test_materialized_view_stale", "test_column")))); } @Test