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