From 348d11449da2bf749e20748867a5a04b2c3b9e8e Mon Sep 17 00:00:00 2001 From: Tim Meehan Date: Wed, 12 Nov 2025 16:31:11 -0500 Subject: [PATCH 1/2] feat(plugin-iceberg): Add support for Iceberg materialized views --- .../src/main/sphinx/connector/iceberg.rst | 61 +- presto-iceberg/pom.xml | 7 + .../iceberg/IcebergAbstractMetadata.java | 413 ++++- .../presto/iceberg/IcebergCommonModule.java | 2 + .../presto/iceberg/IcebergConfig.java | 17 + .../presto/iceberg/IcebergErrorCode.java | 4 +- .../presto/iceberg/IcebergHiveMetadata.java | 31 +- .../iceberg/IcebergHiveMetadataFactory.java | 7 + .../iceberg/IcebergInsertTableHandle.java | 8 +- .../presto/iceberg/IcebergNativeMetadata.java | 104 +- .../iceberg/IcebergNativeMetadataFactory.java | 9 +- .../iceberg/IcebergSessionProperties.java | 13 + .../presto/iceberg/IcebergTableHandle.java | 20 +- .../iceberg/IcebergTableProperties.java | 22 + .../facebook/presto/iceberg/IcebergUtil.java | 11 +- .../iceberg/IcebergWritableTableHandle.java | 26 + .../IcebergEqualityDeleteAsJoin.java | 6 +- .../presto/iceberg/TestIcebergConfig.java | 7 +- .../TestIcebergMaterializedViewMetadata.java | 532 ++++++ .../TestIcebergMaterializedViewOptimizer.java | 152 ++ .../iceberg/TestIcebergMaterializedViews.java | 1514 +++++++++++++++++ .../presto/iceberg/TestStatisticsUtil.java | 2 +- .../TestRenameTableOnFragileFileSystem.java | 5 +- .../sql/analyzer/StatementAnalyzer.java | 2 +- .../iceberg/presto_protocol_iceberg.cpp | 28 + .../iceberg/presto_protocol_iceberg.h | 2 + .../special/IcebergInsertTableHandle.hpp.inc | 1 + 27 files changed, 2971 insertions(+), 35 deletions(-) create mode 100644 presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergMaterializedViewMetadata.java create mode 100644 presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergMaterializedViewOptimizer.java create mode 100644 presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergMaterializedViews.java diff --git a/presto-docs/src/main/sphinx/connector/iceberg.rst b/presto-docs/src/main/sphinx/connector/iceberg.rst index 4339f54b6c61a..3e2a6d5977208 100644 --- a/presto-docs/src/main/sphinx/connector/iceberg.rst +++ b/presto-docs/src/main/sphinx/connector/iceberg.rst @@ -553,6 +553,10 @@ Property Name Description See :ref:`develop/connectors:Node Selection Strategy`. ``iceberg.parquet_dereference_pushdown_enabled`` Overrides the behavior of the connector property Yes No ``iceberg.enable-parquet-dereference-pushdown`` in the current session. +``materialized_view_storage_table_name_prefix`` Prefix for automatically generated materialized view storage table Yes No + names. Default: ``__mv_storage__`` +``materialized_view_missing_base_table_behavior`` Behavior when a base table referenced by a materialized view is Yes No + missing. Valid values: ``FAIL``, ``IGNORE``. Default: ``FAIL`` ===================================================== ======================================================================= =================== ============================================= Caching Support @@ -2261,4 +2265,59 @@ For example:: ) If a user creates a table externally with non-identity sort columns and then inserts data, the following warning message will be shown. -``Iceberg table sort order has sort fields of , , ... which are not currently supported by Presto`` \ No newline at end of file +``Iceberg table sort order has sort fields of , , ... which are not currently supported by Presto`` + +Materialized Views +------------------ + +The Iceberg connector supports materialized views. See :doc:`/admin/materialized-views` for general information and :doc:`/sql/create-materialized-view` for SQL syntax. + +Storage +^^^^^^^ + +Materialized views use a dedicated Iceberg storage table to persist the pre-computed results. By default, the storage table is created with the prefix ``__mv_storage__`` followed by the materialized view name in the same schema as the view. + +Table Properties +^^^^^^^^^^^^^^^^ + +The following table properties can be specified when creating a materialized view: + +========================================================== ============================================================================ +Property Name Description +========================================================== ============================================================================ +``materialized_view_storage_schema`` Schema name for the storage table. Defaults to the materialized view's + schema. + +``materialized_view_storage_table_name`` Custom name for the storage table. Defaults to the prefix plus the + materialized view name. +========================================================== ============================================================================ + +The storage table inherits standard Iceberg table properties for partitioning, sorting, and file format. + +Freshness and Refresh +^^^^^^^^^^^^^^^^^^^^^^ + +Materialized views track the snapshot IDs of their base tables to determine staleness. When base tables are modified, the materialized view becomes stale and returns results by querying the base tables directly. After running ``REFRESH MATERIALIZED VIEW``, queries read from the pre-computed storage table. + +The refresh operation uses a full refresh strategy, replacing all data in the storage table with the current query results. + +Limitations +^^^^^^^^^^^ + +- All refreshes recompute the entire result set +- REFRESH does not provide snapshot isolation across multiple base tables +- Querying materialized views at specific snapshots or timestamps is not supported + +Example +^^^^^^^ + +Create a materialized view with custom storage configuration: + +.. code-block:: sql + + CREATE MATERIALIZED VIEW regional_sales + WITH ( + materialized_view_storage_schema = 'analytics', + materialized_view_storage_table_name = 'sales_summary' + ) + AS SELECT region, SUM(amount) as total FROM orders GROUP BY region; \ No newline at end of file diff --git a/presto-iceberg/pom.xml b/presto-iceberg/pom.xml index c7a57d800cb67..f01df336890e2 100644 --- a/presto-iceberg/pom.xml +++ b/presto-iceberg/pom.xml @@ -609,6 +609,13 @@ test + + com.facebook.presto + presto-common + test-jar + test + + com.facebook.airlift http-server diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergAbstractMetadata.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergAbstractMetadata.java index 1a5db48f80051..8c7736205548d 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergAbstractMetadata.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergAbstractMetadata.java @@ -46,6 +46,9 @@ import com.facebook.presto.spi.ConnectorTableMetadata; import com.facebook.presto.spi.Constraint; import com.facebook.presto.spi.DiscretePredicates; +import com.facebook.presto.spi.MaterializedViewDefinition; +import com.facebook.presto.spi.MaterializedViewDefinition.ColumnMapping; +import com.facebook.presto.spi.MaterializedViewStatus; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.PrestoWarning; import com.facebook.presto.spi.SchemaTableName; @@ -61,6 +64,7 @@ import com.facebook.presto.spi.plan.FilterStatsCalculatorService; import com.facebook.presto.spi.relation.RowExpression; import com.facebook.presto.spi.relation.RowExpressionService; +import com.facebook.presto.spi.security.ViewSecurity; import com.facebook.presto.spi.statistics.ColumnStatisticMetadata; import com.facebook.presto.spi.statistics.ComputedStatistics; import com.facebook.presto.spi.statistics.TableStatisticType; @@ -109,6 +113,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -140,6 +145,7 @@ import static com.facebook.presto.iceberg.IcebergColumnHandle.PATH_COLUMN_HANDLE; import static com.facebook.presto.iceberg.IcebergColumnHandle.PATH_COLUMN_METADATA; import static com.facebook.presto.iceberg.IcebergErrorCode.ICEBERG_COMMIT_ERROR; +import static com.facebook.presto.iceberg.IcebergErrorCode.ICEBERG_INVALID_MATERIALIZED_VIEW; import static com.facebook.presto.iceberg.IcebergErrorCode.ICEBERG_INVALID_SNAPSHOT_ID; import static com.facebook.presto.iceberg.IcebergMetadataColumn.DATA_SEQUENCE_NUMBER; import static com.facebook.presto.iceberg.IcebergMetadataColumn.DELETE_FILE_PATH; @@ -148,10 +154,13 @@ import static com.facebook.presto.iceberg.IcebergMetadataColumn.UPDATE_ROW_DATA; import static com.facebook.presto.iceberg.IcebergPartitionType.ALL; import static com.facebook.presto.iceberg.IcebergSessionProperties.getCompressionCodec; +import static com.facebook.presto.iceberg.IcebergSessionProperties.getMaterializedViewStoragePrefix; import static com.facebook.presto.iceberg.IcebergSessionProperties.isPushdownFilterEnabled; import static com.facebook.presto.iceberg.IcebergTableProperties.LOCATION_PROPERTY; import static com.facebook.presto.iceberg.IcebergTableProperties.PARTITIONING_PROPERTY; import static com.facebook.presto.iceberg.IcebergTableProperties.SORTED_BY_PROPERTY; +import static com.facebook.presto.iceberg.IcebergTableProperties.getMaterializedViewStorageSchema; +import static com.facebook.presto.iceberg.IcebergTableProperties.getMaterializedViewStorageTableName; import static com.facebook.presto.iceberg.IcebergTableType.CHANGELOG; import static com.facebook.presto.iceberg.IcebergTableType.DATA; import static com.facebook.presto.iceberg.IcebergTableType.EQUALITY_DELETES; @@ -193,16 +202,24 @@ import static com.facebook.presto.iceberg.optimizer.IcebergPlanOptimizer.getEnforcedColumns; import static com.facebook.presto.iceberg.util.StatisticsUtil.calculateBaseTableStatistics; import static com.facebook.presto.iceberg.util.StatisticsUtil.calculateStatisticsConsideringLayout; +import static com.facebook.presto.spi.MaterializedViewStatus.MaterializedViewState.FULLY_MATERIALIZED; +import static com.facebook.presto.spi.MaterializedViewStatus.MaterializedViewState.NOT_MATERIALIZED; +import static com.facebook.presto.spi.MaterializedViewStatus.MaterializedViewState.PARTIALLY_MATERIALIZED; +import static com.facebook.presto.spi.StandardErrorCode.ALREADY_EXISTS; import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED; import static com.facebook.presto.spi.statistics.TableStatisticType.ROW_COUNT; +import static com.google.common.base.Preconditions.checkState; import static com.google.common.base.Strings.isNullOrEmpty; import static com.google.common.base.Verify.verify; import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableMap.toImmutableMap; import static com.google.common.collect.ImmutableSet.toImmutableSet; +import static java.lang.Long.parseLong; import static java.lang.String.format; +import static java.util.Arrays.stream; import static java.util.Collections.singletonList; import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.joining; import static org.apache.iceberg.MetadataColumns.ROW_POSITION; import static org.apache.iceberg.RowLevelOperationMode.MERGE_ON_READ; import static org.apache.iceberg.SnapshotSummary.DELETED_RECORDS_PROP; @@ -211,6 +228,7 @@ import static org.apache.iceberg.TableProperties.DELETE_ISOLATION_LEVEL; import static org.apache.iceberg.TableProperties.DELETE_ISOLATION_LEVEL_DEFAULT; import static org.apache.iceberg.TableProperties.WRITE_DATA_LOCATION; +import static org.apache.iceberg.expressions.Expressions.alwaysTrue; public abstract class IcebergAbstractMetadata implements ConnectorMetadata @@ -218,8 +236,20 @@ public abstract class IcebergAbstractMetadata private static final Logger log = Logger.get(IcebergAbstractMetadata.class); protected static final String INFORMATION_SCHEMA = "information_schema"; + // Materialized view metadata property keys + protected static final String PRESTO_MATERIALIZED_VIEW_ORIGINAL_SQL = "presto.materialized_view.original_sql"; + protected static final String PRESTO_MATERIALIZED_VIEW_BASE_TABLES = "presto.materialized_view.base_tables"; + protected static final String PRESTO_MATERIALIZED_VIEW_BASE_SNAPSHOT_PREFIX = "presto.materialized_view.base_snapshot."; + protected static final String PRESTO_MATERIALIZED_VIEW_LAST_REFRESH_SNAPSHOT_ID = "presto.materialized_view.last_refresh_snapshot_id"; + protected static final String PRESTO_MATERIALIZED_VIEW_STORAGE_SCHEMA = "presto.materialized_view.storage_schema"; + protected static final String PRESTO_MATERIALIZED_VIEW_STORAGE_TABLE_NAME = "presto.materialized_view.storage_table_name"; + protected static final String PRESTO_MATERIALIZED_VIEW_COLUMN_MAPPINGS = "presto.materialized_view.column_mappings"; + protected static final String PRESTO_MATERIALIZED_VIEW_OWNER = "presto.materialized_view.owner"; + protected static final String PRESTO_MATERIALIZED_VIEW_SECURITY_MODE = "presto.materialized_view.security_mode"; + protected final TypeManager typeManager; protected final JsonCodec commitTaskCodec; + protected final JsonCodec> columnMappingsCodec; protected final NodeVersion nodeVersion; protected final RowExpressionService rowExpressionService; protected final FilterStatsCalculatorService filterStatsCalculatorService; @@ -235,6 +265,7 @@ public IcebergAbstractMetadata( StandardFunctionResolution functionResolution, RowExpressionService rowExpressionService, JsonCodec commitTaskCodec, + JsonCodec> columnMappingsCodec, NodeVersion nodeVersion, FilterStatsCalculatorService filterStatsCalculatorService, StatisticsFileCache statisticsFileCache, @@ -242,6 +273,7 @@ public IcebergAbstractMetadata( { this.typeManager = requireNonNull(typeManager, "typeManager is null"); this.commitTaskCodec = requireNonNull(commitTaskCodec, "commitTaskCodec is null"); + this.columnMappingsCodec = requireNonNull(columnMappingsCodec, "columnMappingsCodec is null"); this.functionResolution = requireNonNull(functionResolution, "functionResolution is null"); this.rowExpressionService = requireNonNull(rowExpressionService, "rowExpressionService is null"); this.nodeVersion = requireNonNull(nodeVersion, "nodeVersion is null"); @@ -261,6 +293,20 @@ protected final Table getIcebergTable(ConnectorSession session, SchemaTableName protected abstract View getIcebergView(ConnectorSession session, SchemaTableName schemaTableName); + protected abstract void createIcebergView( + ConnectorSession session, + SchemaTableName viewName, + List columns, + String viewSql, + Map properties); + + protected abstract void dropIcebergView(ConnectorSession session, SchemaTableName viewName); + + protected abstract void updateIcebergViewProperties( + ConnectorSession session, + SchemaTableName viewName, + Map properties); + protected abstract boolean tableExists(ConnectorSession session, SchemaTableName schemaTableName); public abstract void registerTable(ConnectorSession clientSession, SchemaTableName schemaTableName, Path metadataLocation); @@ -524,7 +570,8 @@ protected ConnectorInsertTableHandle beginIcebergTableInsert(ConnectorSession se getFileFormat(icebergTable), getCompressionCodec(session), icebergTable.properties(), - getSupportedSortFields(icebergTable.schema(), icebergTable.sortOrder())); + getSupportedSortFields(icebergTable.schema(), icebergTable.sortOrder()), + Optional.empty()); } public static List getSupportedSortFields(Schema schema, SortOrder sortOrder) @@ -723,8 +770,8 @@ protected List getColumnMetadata(ConnectorSession session, Table .setComment(column.doc()) .setHidden(false) .setExtraInfo(partitionFields.containsKey(column.name()) ? - columnExtraInfo(partitionFields.get(column.name())) : - null) + columnExtraInfo(partitionFields.get(column.name())) : + null) .build()) .collect(toImmutableList()); } @@ -980,11 +1027,38 @@ public IcebergTableHandle getTableHandle(ConnectorSession session, SchemaTableNa verify(name.getTableType() == DATA || name.getTableType() == CHANGELOG || name.getTableType() == EQUALITY_DELETES, "Wrong table type: " + name.getTableType()); if (!tableExists(session, tableName)) { - return null; + // If table doesn't exist, check if it's a materialized view + return getMaterializedView(session, tableName).map(definition -> { + SchemaTableName storageTableName = new SchemaTableName(definition.getSchema(), definition.getTable()); + Table storageTable = getIcebergTable(session, storageTableName); + + // TODO: time travel for materialized views doesn't work properly + Optional tableSnapshotId = tableVersion + .map(version -> { + long tableVersionSnapshotId = getSnapshotIdForTableVersion(storageTable, version); + return Optional.of(tableVersionSnapshotId); + }) + .orElseGet(() -> resolveSnapshotIdByName(storageTable, name)); + + return new IcebergTableHandle( + storageTableName.getSchemaName(), + new IcebergTableName(storageTableName.getTableName(), name.getTableType(), tableSnapshotId, name.getChangelogEndSnapshot()), + name.getSnapshotId().isPresent(), + tryGetLocation(storageTable), + tryGetProperties(storageTable), + tryGetSchema(storageTable).map(SchemaParser::toJson), + Optional.empty(), + Optional.empty(), + getSortFields(storageTable), + ImmutableList.of(), + Optional.of(tableName)); + }) + // null indicates table not found + .orElse(null); } - // use a new schema table name that omits the table type - Table table = getIcebergTable(session, new SchemaTableName(tableName.getSchemaName(), name.getTableName())); + SchemaTableName tableNameToLoad = new SchemaTableName(tableName.getSchemaName(), name.getTableName()); + Table table = getIcebergTable(session, tableNameToLoad); Optional tableSnapshotId = tableVersion .map(version -> { @@ -1000,8 +1074,8 @@ public IcebergTableHandle getTableHandle(ConnectorSession session, SchemaTableNa Optional tableSchemaJson = tableSchema.map(SchemaParser::toJson); return new IcebergTableHandle( - tableName.getSchemaName(), - new IcebergTableName(name.getTableName(), name.getTableType(), tableSnapshotId, name.getChangelogEndSnapshot()), + tableNameToLoad.getSchemaName(), + new IcebergTableName(tableNameToLoad.getTableName(), name.getTableType(), tableSnapshotId, name.getChangelogEndSnapshot()), name.getSnapshotId().isPresent(), tryGetLocation(table), tryGetProperties(table), @@ -1009,7 +1083,8 @@ public IcebergTableHandle getTableHandle(ConnectorSession session, SchemaTableNa Optional.empty(), Optional.empty(), getSortFields(table), - ImmutableList.of()); + ImmutableList.of(), + Optional.empty()); } @Override @@ -1232,9 +1307,9 @@ private OptionalLong removeScanFiles(Table icebergTable, TupleDomain summary = icebergTable.currentSnapshot().summary(); - long deletedRecords = Long.parseLong(summary.getOrDefault(DELETED_RECORDS_PROP, "0")); - long removedPositionDeletes = Long.parseLong(summary.getOrDefault(REMOVED_POS_DELETES_PROP, "0")); - long removedEqualityDeletes = Long.parseLong(summary.getOrDefault(REMOVED_EQ_DELETES_PROP, "0")); + long deletedRecords = parseLong(summary.getOrDefault(DELETED_RECORDS_PROP, "0")); + long removedPositionDeletes = parseLong(summary.getOrDefault(REMOVED_POS_DELETES_PROP, "0")); + long removedEqualityDeletes = parseLong(summary.getOrDefault(REMOVED_EQ_DELETES_PROP, "0")); // Removed rows count is inaccurate when existing equality delete files return OptionalLong.of(deletedRecords - removedPositionDeletes - removedEqualityDeletes); } @@ -1361,4 +1436,318 @@ public Optional getInfo(ConnectorTableLayoutHandle tableHandle) icebergTableHandle.getTable().getIcebergTableName().getSnapshotId(), outputPath.get())); } + + @Override + public void createMaterializedView( + ConnectorSession session, + ConnectorTableMetadata viewMetadata, + MaterializedViewDefinition viewDefinition, + boolean ignoreExisting) + { + try { + SchemaTableName viewName = viewMetadata.getTable(); + Map materializedViewProperties = viewMetadata.getProperties(); + + SchemaTableName storageTableName = getStorageTableName(session, viewName, materializedViewProperties); + + if (viewExists(session, viewMetadata)) { + if (ignoreExisting) { + return; + } + throw new PrestoException(ALREADY_EXISTS, "Materialized view " + viewName + " already exists"); + } + + ConnectorTableMetadata storageTableMetadata = new ConnectorTableMetadata( + storageTableName, + viewMetadata.getColumns(), + materializedViewProperties, + viewMetadata.getComment()); + createTable(session, storageTableMetadata, false); + + Map properties = new HashMap<>(); + properties.put(PRESTO_MATERIALIZED_VIEW_ORIGINAL_SQL, viewDefinition.getOriginalSql()); + properties.put(PRESTO_MATERIALIZED_VIEW_STORAGE_SCHEMA, storageTableName.getSchemaName()); + properties.put(PRESTO_MATERIALIZED_VIEW_STORAGE_TABLE_NAME, storageTableName.getTableName()); + + String baseTablesStr = viewDefinition.getBaseTables().stream() + .map(table -> table.getSchemaName() + "." + table.getTableName()) + .collect(joining(",")); + properties.put(PRESTO_MATERIALIZED_VIEW_BASE_TABLES, baseTablesStr); + properties.put(PRESTO_MATERIALIZED_VIEW_COLUMN_MAPPINGS, serializeColumnMappings(viewDefinition.getColumnMappings())); + checkState(viewDefinition.getOwner().isPresent(), "Materialized view owner is required"); + properties.put(PRESTO_MATERIALIZED_VIEW_OWNER, viewDefinition.getOwner().get()); + checkState(viewDefinition.getSecurityMode().isPresent(), "Materialized view security mode is required"); + properties.put(PRESTO_MATERIALIZED_VIEW_SECURITY_MODE, viewDefinition.getSecurityMode().get().name()); + + for (SchemaTableName baseTable : viewDefinition.getBaseTables()) { + Table baseIcebergTable = getIcebergTable(session, baseTable); + long baseSnapshotId = baseIcebergTable.currentSnapshot() != null + ? baseIcebergTable.currentSnapshot().snapshotId() + : 0L; + properties.put(getBaseTableViewPropertyName(baseTable), baseSnapshotId + ""); + } + + createIcebergView(session, viewName, viewMetadata.getColumns(), viewDefinition.getOriginalSql(), properties); + } + catch (PrestoException e) { + if (e.getErrorCode() == NOT_SUPPORTED.toErrorCode()) { + throw new PrestoException(NOT_SUPPORTED, "Materialized views are not supported with this catalog type", e); + } + throw e; + } + } + + @Override + public Optional getMaterializedView(ConnectorSession session, SchemaTableName viewName) + { + try { + View icebergView = getIcebergView(session, viewName); + + Map viewProperties = icebergView.properties(); + String originalSql = viewProperties.get(PRESTO_MATERIALIZED_VIEW_ORIGINAL_SQL); + + if (originalSql == null) { + return Optional.empty(); + } + + String baseTablesStr = getRequiredMaterializedViewProperty(viewProperties, PRESTO_MATERIALIZED_VIEW_BASE_TABLES); + List baseTables; + if (baseTablesStr.isEmpty()) { + baseTables = ImmutableList.of(); + } + else { + baseTables = stream(baseTablesStr.split("\\s*,\\s*")) + .map(tableStr -> { + String[] parts = tableStr.split("\\."); + if (parts.length != 2 || parts[0].isEmpty() || parts[1].isEmpty()) { + throw new PrestoException(ICEBERG_INVALID_MATERIALIZED_VIEW, + format("Invalid base table name format: %s. Expected format: schema.table", tableStr)); + } + return new SchemaTableName(parts[0], parts[1]); + }) + .collect(toImmutableList()); + } + + String columnMappingsJson = getRequiredMaterializedViewProperty(viewProperties, PRESTO_MATERIALIZED_VIEW_COLUMN_MAPPINGS); + List columnMappings = deserializeColumnMappings(columnMappingsJson); + + String storageSchema = getRequiredMaterializedViewProperty(viewProperties, PRESTO_MATERIALIZED_VIEW_STORAGE_SCHEMA); + String storageTableName = getRequiredMaterializedViewProperty(viewProperties, PRESTO_MATERIALIZED_VIEW_STORAGE_TABLE_NAME); + + String owner = getRequiredMaterializedViewProperty(viewProperties, PRESTO_MATERIALIZED_VIEW_OWNER); + ViewSecurity securityMode; + try { + securityMode = ViewSecurity.valueOf(getRequiredMaterializedViewProperty(viewProperties, PRESTO_MATERIALIZED_VIEW_SECURITY_MODE)); + } + catch (IllegalArgumentException | NullPointerException e) { + throw new PrestoException(ICEBERG_INVALID_MATERIALIZED_VIEW, "Invalid or missing materialized view security mode"); + } + + return Optional.of(new MaterializedViewDefinition( + originalSql, + storageSchema, + storageTableName, + baseTables, + Optional.of(owner), + Optional.of(securityMode), + columnMappings, + ImmutableList.of(), + Optional.empty())); + } + catch (NoSuchViewException e) { + return Optional.empty(); + } + catch (PrestoException e) { + if (e.getErrorCode() == NOT_SUPPORTED.toErrorCode()) { + return Optional.empty(); + } + throw e; + } + } + + @Override + public void dropMaterializedView(ConnectorSession session, SchemaTableName viewName) + { + Optional definition = getMaterializedView(session, viewName); + + dropIcebergView(session, viewName); + + if (definition.isPresent()) { + SchemaTableName storageTableName = new SchemaTableName( + definition.get().getSchema(), + definition.get().getTable()); + ConnectorTableHandle storageTableHandle = getTableHandle(session, storageTableName); + if (storageTableHandle != null) { + dropTable(session, storageTableHandle); + } + } + } + + @Override + public MaterializedViewStatus getMaterializedViewStatus( + ConnectorSession session, + SchemaTableName materializedViewName, + TupleDomain baseQueryDomain) + { + Optional definition = getMaterializedView(session, materializedViewName); + if (definition.isEmpty()) { + return new MaterializedViewStatus(NOT_MATERIALIZED, ImmutableMap.of()); + } + + View icebergView = getIcebergView(session, materializedViewName); + Map props = icebergView.properties(); + String lastRefreshSnapshotStr = props.get(PRESTO_MATERIALIZED_VIEW_LAST_REFRESH_SNAPSHOT_ID); + if (lastRefreshSnapshotStr == null) { + return new MaterializedViewStatus(NOT_MATERIALIZED, ImmutableMap.of()); + } + + for (SchemaTableName baseTable : definition.get().getBaseTables()) { + Table baseIcebergTable = getIcebergTable(session, baseTable); + long currentSnapshotId = baseIcebergTable.currentSnapshot() != null + ? baseIcebergTable.currentSnapshot().snapshotId() + : 0L; + + String key = getBaseTableViewPropertyName(baseTable); + String recordedSnapshotStr = props.get(key); + long recordedSnapshotId = recordedSnapshotStr != null ? parseLong(recordedSnapshotStr) : -1L; + + if (currentSnapshotId != recordedSnapshotId) { + return new MaterializedViewStatus(PARTIALLY_MATERIALIZED, ImmutableMap.of()); + } + } + + return new MaterializedViewStatus(FULLY_MATERIALIZED, ImmutableMap.of()); + } + + @Override + public ConnectorInsertTableHandle beginRefreshMaterializedView( + ConnectorSession session, + ConnectorTableHandle tableHandle) + { + IcebergTableHandle mvHandle = (IcebergTableHandle) tableHandle; + + if (mvHandle.getMaterializedViewName().isEmpty()) { + throw new IllegalStateException(format( + "beginRefreshMaterializedView called on non-materialized view table: %s", + mvHandle.getSchemaTableName())); + } + + SchemaTableName storageTableName = mvHandle.getSchemaTableName(); + IcebergTableHandle storageTableHandle = getTableHandle(session, storageTableName); + Table storageTable = getIcebergTable(session, storageTableName); + + transaction = storageTable.newTransaction(); + + transaction.newDelete().deleteFromRowFilter(alwaysTrue()).commit(); + + SchemaTableName mvName = mvHandle.getMaterializedViewName() + .orElseThrow(() -> new PrestoException(ICEBERG_INVALID_MATERIALIZED_VIEW, "materializedViewName not set in handle for MV refresh")); + + return new IcebergInsertTableHandle( + storageTableHandle.getSchemaName(), + storageTableHandle.getIcebergTableName(), + toPrestoSchema(storageTable.schema(), typeManager), + toPrestoPartitionSpec(storageTable.spec(), typeManager), + getColumns(storageTable.schema(), storageTable.spec(), typeManager), + storageTable.location(), + getFileFormat(storageTable), + getCompressionCodec(session), + storageTable.properties(), + getSupportedSortFields(storageTable.schema(), storageTable.sortOrder()), + Optional.of(mvName)); + } + + @Override + public Optional finishRefreshMaterializedView( + ConnectorSession session, + ConnectorInsertTableHandle insertHandle, + Collection fragments, + Collection computedStatistics) + { + Optional result = finishInsert(session, insertHandle, fragments, computedStatistics); + + IcebergInsertTableHandle icebergInsertHandle = (IcebergInsertTableHandle) insertHandle; + + icebergInsertHandle.getMaterializedViewName().ifPresent(materializedViewName -> { + SchemaTableName storageTableName = new SchemaTableName( + icebergInsertHandle.getSchemaName(), + icebergInsertHandle.getTableName().getTableName()); + + Table storageTable = getIcebergTable(session, storageTableName); + long newSnapshotId = storageTable.currentSnapshot() != null + ? storageTable.currentSnapshot().snapshotId() + : 0L; + + Optional definition = getMaterializedView(session, materializedViewName); + Map properties = new HashMap<>(); + properties.put(PRESTO_MATERIALIZED_VIEW_LAST_REFRESH_SNAPSHOT_ID, String.valueOf(newSnapshotId)); + + if (definition.isPresent()) { + for (SchemaTableName baseTable : definition.get().getBaseTables()) { + try { + // TODO: this is not transactional - base table may change between refresh start and end + Table baseIcebergTable = getIcebergTable(session, baseTable); + long baseSnapshotId = baseIcebergTable.currentSnapshot() != null + ? baseIcebergTable.currentSnapshot().snapshotId() + : 0L; + String key = getBaseTableViewPropertyName(baseTable); + properties.put(key, baseSnapshotId + ""); + } + catch (Exception e) { + log.warn(e, "Failed to capture snapshot for base table %s during refresh of materialized view %s", baseTable, materializedViewName); + } + } + } + + updateIcebergViewProperties(session, materializedViewName, properties); + }); + + return result; + } + + private SchemaTableName getStorageTableName(ConnectorSession session, SchemaTableName viewName, Map properties) + { + String tableName = getMaterializedViewStorageTableName(properties).orElseGet(() -> { + // Generate default storage table name using prefix + return getMaterializedViewStoragePrefix(session) + viewName.getTableName(); + }); + String schema = getMaterializedViewStorageSchema(properties) + .orElse(viewName.getSchemaName()); + return new SchemaTableName(schema, tableName); + } + + private String serializeColumnMappings(List columnMappings) + { + return columnMappingsCodec.toJson(columnMappings); + } + + private List deserializeColumnMappings(String json) + { + return columnMappingsCodec.fromJson(json); + } + + private static String getBaseTableViewPropertyName(SchemaTableName baseTable) + { + return format("%s%s.%s", PRESTO_MATERIALIZED_VIEW_BASE_SNAPSHOT_PREFIX, baseTable.getSchemaName(), baseTable.getTableName()); + } + + private static String getRequiredMaterializedViewProperty(Map viewProperties, String propertyKey) + { + String value = viewProperties.get(propertyKey); + if (value == null) { + throw new PrestoException(ICEBERG_INVALID_MATERIALIZED_VIEW, format("Materialized view missing required property: %s", propertyKey)); + } + return value; + } + + private boolean viewExists(ConnectorSession session, ConnectorTableMetadata viewMetadata) + { + try { + getIcebergView(session, viewMetadata.getTable()); + return true; + } + catch (NoSuchViewException e) { + return false; + } + } } diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergCommonModule.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergCommonModule.java index d0e3049e0f5ad..497ed5b529593 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergCommonModule.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergCommonModule.java @@ -77,6 +77,7 @@ import com.facebook.presto.parquet.cache.ParquetCacheConfig; import com.facebook.presto.parquet.cache.ParquetFileMetadata; import com.facebook.presto.parquet.cache.ParquetMetadataSource; +import com.facebook.presto.spi.MaterializedViewDefinition; import com.facebook.presto.spi.connector.ConnectorNodePartitioningProvider; import com.facebook.presto.spi.connector.ConnectorPageSinkProvider; import com.facebook.presto.spi.connector.ConnectorPageSourceProvider; @@ -166,6 +167,7 @@ protected void setup(Binder binder) configBinder(binder).bindConfig(ParquetFileWriterConfig.class); jsonCodecBinder(binder).bindJsonCodec(CommitTaskData.class); + jsonCodecBinder(binder).bindListJsonCodec(MaterializedViewDefinition.ColumnMapping.class); binder.bind(FileFormatDataSourceStats.class).in(Scopes.SINGLETON); newExporter(binder).export(FileFormatDataSourceStats.class).withGeneratedName(); diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergConfig.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergConfig.java index 720bdd1e02151..432612bb3366b 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergConfig.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergConfig.java @@ -76,6 +76,7 @@ public class IcebergConfig private DataSize manifestCacheMaxChunkSize = succinctDataSize(2, MEGABYTE); private int splitManagerThreads = Runtime.getRuntime().availableProcessors(); private DataSize maxStatisticsFileCacheSize = succinctDataSize(256, MEGABYTE); + private String materializedViewStoragePrefix = "__mv_storage__"; @NotNull public FileFormat getFileFormat() @@ -479,4 +480,20 @@ public IcebergConfig setStatisticsKllSketchKParameter(int kllSketchKParameter) this.statisticsKllSketchKParameter = kllSketchKParameter; return this; } + + @NotNull + public String getMaterializedViewStoragePrefix() + { + return materializedViewStoragePrefix; + } + + @Config("iceberg.materialized-view-storage-prefix") + @ConfigDescription("Default prefix for generated materialized view storage table names. " + + "This is only used when the materialized_view_storage_table_name table property is not explicitly set. " + + "When a custom table name is provided, it takes precedence over this prefix.") + public IcebergConfig setMaterializedViewStoragePrefix(String materializedViewStoragePrefix) + { + this.materializedViewStoragePrefix = materializedViewStoragePrefix; + return this; + } } diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergErrorCode.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergErrorCode.java index 2d78252763f7f..c7a4b98291771 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergErrorCode.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergErrorCode.java @@ -39,7 +39,9 @@ public enum IcebergErrorCode ICEBERG_INVALID_FORMAT_VERSION(14, USER_ERROR), ICEBERG_UNKNOWN_MANIFEST_TYPE(15, EXTERNAL), ICEBERG_COMMIT_ERROR(16, EXTERNAL), - ICEBERG_MISSING_COLUMN(17, EXTERNAL); + ICEBERG_MISSING_COLUMN(17, EXTERNAL), + ICEBERG_INVALID_MATERIALIZED_VIEW(18, EXTERNAL), + /**/; private final ErrorCode errorCode; diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergHiveMetadata.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergHiveMetadata.java index 3737247d9c595..5fb9569b5222e 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergHiveMetadata.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergHiveMetadata.java @@ -37,6 +37,7 @@ import com.facebook.presto.hive.metastore.Table; import com.facebook.presto.iceberg.statistics.StatisticsFileCache; import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.ColumnMetadata; import com.facebook.presto.spi.ConnectorNewTableLayout; import com.facebook.presto.spi.ConnectorOutputTableHandle; import com.facebook.presto.spi.ConnectorSession; @@ -46,6 +47,7 @@ import com.facebook.presto.spi.ConnectorTableMetadata; import com.facebook.presto.spi.ConnectorViewDefinition; import com.facebook.presto.spi.Constraint; +import com.facebook.presto.spi.MaterializedViewDefinition; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.SchemaNotFoundException; import com.facebook.presto.spi.SchemaTableName; @@ -178,6 +180,7 @@ public IcebergHiveMetadata( StandardFunctionResolution functionResolution, RowExpressionService rowExpressionService, JsonCodec commitTaskCodec, + JsonCodec> columnMappingsCodec, NodeVersion nodeVersion, FilterStatsCalculatorService filterStatsCalculatorService, IcebergHiveTableOperationsConfig hiveTableOperationsConfig, @@ -186,7 +189,7 @@ public IcebergHiveMetadata( IcebergTableProperties tableProperties, ConnectorSystemConfig connectorSystemConfig) { - super(typeManager, functionResolution, rowExpressionService, commitTaskCodec, nodeVersion, filterStatsCalculatorService, statisticsFileCache, tableProperties); + super(typeManager, functionResolution, rowExpressionService, commitTaskCodec, columnMappingsCodec, nodeVersion, filterStatsCalculatorService, statisticsFileCache, tableProperties); this.catalogName = requireNonNull(catalogName, "catalogName is null"); this.metastore = requireNonNull(metastore, "metastore is null"); this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); @@ -692,4 +695,30 @@ public void unregisterTable(ConnectorSession clientSession, SchemaTableName sche MetastoreContext metastoreContext = getMetastoreContext(clientSession); metastore.dropTableFromMetastore(metastoreContext, schemaTableName.getSchemaName(), schemaTableName.getTableName()); } + + @Override + protected void createIcebergView( + ConnectorSession session, + SchemaTableName viewName, + List columns, + String viewSql, + Map properties) + { + throw new PrestoException(NOT_SUPPORTED, "Iceberg Hive catalog does not support native Iceberg views for materialized views."); + } + + @Override + protected void dropIcebergView(ConnectorSession session, SchemaTableName schemaTableName) + { + throw new PrestoException(NOT_SUPPORTED, "Iceberg Hive catalog does not support native Iceberg views for materialized views."); + } + + @Override + protected void updateIcebergViewProperties( + ConnectorSession session, + SchemaTableName viewName, + Map properties) + { + throw new PrestoException(NOT_SUPPORTED, "Iceberg Hive catalog does not support native Iceberg views for materialized views."); + } } diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergHiveMetadataFactory.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergHiveMetadataFactory.java index df801b6da3c88..d01ae7a33dab7 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergHiveMetadataFactory.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergHiveMetadataFactory.java @@ -26,6 +26,9 @@ import com.facebook.presto.spi.relation.RowExpressionService; import jakarta.inject.Inject; +import java.util.List; + +import static com.facebook.presto.spi.MaterializedViewDefinition.ColumnMapping; import static java.util.Objects.requireNonNull; public class IcebergHiveMetadataFactory @@ -36,6 +39,7 @@ public class IcebergHiveMetadataFactory final HdfsEnvironment hdfsEnvironment; final TypeManager typeManager; final JsonCodec commitTaskCodec; + final JsonCodec> columnMappingsCodec; final StandardFunctionResolution functionResolution; final RowExpressionService rowExpressionService; final NodeVersion nodeVersion; @@ -55,6 +59,7 @@ public IcebergHiveMetadataFactory( StandardFunctionResolution functionResolution, RowExpressionService rowExpressionService, JsonCodec commitTaskCodec, + JsonCodec> columnMappingsCodec, NodeVersion nodeVersion, FilterStatsCalculatorService filterStatsCalculatorService, IcebergHiveTableOperationsConfig operationsConfig, @@ -70,6 +75,7 @@ public IcebergHiveMetadataFactory( this.functionResolution = requireNonNull(functionResolution, "functionResolution is null"); this.rowExpressionService = requireNonNull(rowExpressionService, "rowExpressionService is null"); this.commitTaskCodec = requireNonNull(commitTaskCodec, "commitTaskCodec is null"); + this.columnMappingsCodec = requireNonNull(columnMappingsCodec, "columnMappingsCodec is null"); this.nodeVersion = requireNonNull(nodeVersion, "nodeVersion is null"); this.filterStatsCalculatorService = requireNonNull(filterStatsCalculatorService, "filterStatsCalculatorService is null"); this.operationsConfig = requireNonNull(operationsConfig, "operationsConfig is null"); @@ -89,6 +95,7 @@ public ConnectorMetadata create() functionResolution, rowExpressionService, commitTaskCodec, + columnMappingsCodec, nodeVersion, filterStatsCalculatorService, operationsConfig, diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergInsertTableHandle.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergInsertTableHandle.java index 0a2ad488a88d4..de6a31503c7b8 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergInsertTableHandle.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergInsertTableHandle.java @@ -15,11 +15,13 @@ import com.facebook.presto.hive.HiveCompressionCodec; import com.facebook.presto.spi.ConnectorInsertTableHandle; +import com.facebook.presto.spi.SchemaTableName; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; import java.util.Map; +import java.util.Optional; public class IcebergInsertTableHandle extends IcebergWritableTableHandle @@ -36,7 +38,8 @@ public IcebergInsertTableHandle( @JsonProperty("fileFormat") FileFormat fileFormat, @JsonProperty("compressionCodec") HiveCompressionCodec compressionCodec, @JsonProperty("storageProperties") Map storageProperties, - @JsonProperty("sortOrder") List sortOrder) + @JsonProperty("sortOrder") List sortOrder, + @JsonProperty("materializedViewName") Optional materializedViewName) { super( schemaName, @@ -48,6 +51,7 @@ public IcebergInsertTableHandle( fileFormat, compressionCodec, storageProperties, - sortOrder); + sortOrder, + materializedViewName); } } diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergNativeMetadata.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergNativeMetadata.java index 766f0bf389528..52d59d7b367f7 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergNativeMetadata.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergNativeMetadata.java @@ -19,12 +19,14 @@ import com.facebook.presto.hive.TableAlreadyExistsException; import com.facebook.presto.iceberg.statistics.StatisticsFileCache; import com.facebook.presto.iceberg.util.IcebergPrestoModelConverters; +import com.facebook.presto.spi.ColumnMetadata; import com.facebook.presto.spi.ConnectorNewTableLayout; import com.facebook.presto.spi.ConnectorOutputTableHandle; import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.ConnectorTableHandle; import com.facebook.presto.spi.ConnectorTableMetadata; import com.facebook.presto.spi.ConnectorViewDefinition; +import com.facebook.presto.spi.MaterializedViewDefinition; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.SchemaTableName; import com.facebook.presto.spi.SchemaTablePrefix; @@ -48,9 +50,11 @@ import org.apache.iceberg.exceptions.NamespaceNotEmptyException; import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.exceptions.NoSuchViewException; import org.apache.iceberg.view.View; import org.apache.iceberg.view.ViewBuilder; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -82,6 +86,7 @@ import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED; import static com.facebook.presto.spi.StandardErrorCode.SCHEMA_NOT_EMPTY; import static com.google.common.base.Strings.isNullOrEmpty; +import static com.google.common.base.Throwables.getRootCause; import static com.google.common.base.Verify.verify; import static com.google.common.collect.ImmutableList.toImmutableList; import static java.lang.String.format; @@ -107,13 +112,14 @@ public IcebergNativeMetadata( StandardFunctionResolution functionResolution, RowExpressionService rowExpressionService, JsonCodec commitTaskCodec, + JsonCodec> columnMappingsCodec, CatalogType catalogType, NodeVersion nodeVersion, FilterStatsCalculatorService filterStatsCalculatorService, StatisticsFileCache statisticsFileCache, IcebergTableProperties tableProperties) { - super(typeManager, functionResolution, rowExpressionService, commitTaskCodec, nodeVersion, filterStatsCalculatorService, statisticsFileCache, tableProperties); + super(typeManager, functionResolution, rowExpressionService, commitTaskCodec, columnMappingsCodec, nodeVersion, filterStatsCalculatorService, statisticsFileCache, tableProperties); this.catalogFactory = requireNonNull(catalogFactory, "catalogFactory is null"); this.catalogType = requireNonNull(catalogType, "catalogType is null"); this.warehouseDataDir = Optional.ofNullable(catalogFactory.getCatalogWarehouseDataDir()); @@ -128,9 +134,18 @@ protected Table getRawIcebergTable(ConnectorSession session, SchemaTableName sch @Override protected View getIcebergView(ConnectorSession session, SchemaTableName schemaTableName) { - return icebergViews.computeIfAbsent( - schemaTableName, - ignored -> getNativeIcebergView(catalogFactory, session, schemaTableName)); + try { + return icebergViews.computeIfAbsent( + schemaTableName, + ignored -> getNativeIcebergView(catalogFactory, session, schemaTableName)); + } + catch (RuntimeException e) { + Throwable rootCause = getRootCause(e); + if (rootCause instanceof NoSuchViewException) { + throw (NoSuchViewException) rootCause; + } + throw e; + } } @Override @@ -285,12 +300,20 @@ public Map getViews(ConnectorSession s TableIdentifier viewIdentifier = toIcebergTableIdentifier(schemaTableName, catalogFactory.isNestedNamespaceEnabled()); if (((ViewCatalog) catalog).viewExists(viewIdentifier)) { View view = ((ViewCatalog) catalog).loadView(viewIdentifier); + // Skip materialized views + if (view.properties().containsKey(PRESTO_MATERIALIZED_VIEW_ORIGINAL_SQL)) { + continue; + } verifyAndPopulateViews(view, schemaTableName, view.sqlFor(VIEW_DIALECT).sql(), views); } } catch (IllegalArgumentException e) { // Ignore illegal view names } + catch (Exception e) { + // Ignore views that can't be loaded (e.g., if listViews returned a table name by mistake) + // This can happen if the catalog's listViews() implementation is buggy + } } } return views.build(); @@ -432,4 +455,77 @@ protected Optional getDataLocationBasedOnWarehouseDataDir(SchemaTableNam } return warehouseDataDir.map(base -> base + schemaTableName.getSchemaName() + "/" + schemaTableName.getTableName()); } + + @Override + protected void createIcebergView( + ConnectorSession session, + SchemaTableName viewName, + List columns, + String viewSql, + Map properties) + { + Catalog catalog = catalogFactory.getCatalog(session); + if (!(catalog instanceof ViewCatalog)) { + throw new PrestoException(NOT_SUPPORTED, "This connector does not support creating Iceberg views for materialized views"); + } + ViewCatalog viewCatalog = (ViewCatalog) catalog; + + Schema schema = toIcebergSchema(columns); + + viewCatalog.buildView(toIcebergTableIdentifier(viewName, catalogFactory.isNestedNamespaceEnabled())) + .withSchema(schema) + .withDefaultNamespace(toIcebergNamespace(Optional.ofNullable(viewName.getSchemaName()), catalogFactory.isNestedNamespaceEnabled())) + .withQuery(VIEW_DIALECT, viewSql) + .withProperties(properties) + .create(); + + icebergViews.remove(viewName); + + if (!viewCatalog.viewExists(toIcebergTableIdentifier(viewName, catalogFactory.isNestedNamespaceEnabled()))) { + throw new PrestoException(ICEBERG_COMMIT_ERROR, "Failed to create Iceberg view for materialized view: " + viewName); + } + } + + @Override + protected void dropIcebergView(ConnectorSession session, SchemaTableName schemaTableName) + { + Catalog catalog = catalogFactory.getCatalog(session); + if (!(catalog instanceof ViewCatalog)) { + throw new PrestoException(NOT_SUPPORTED, "This connector does not support dropping Iceberg views for materialized views"); + } + ViewCatalog viewCatalog = (ViewCatalog) catalog; + viewCatalog.dropView(toIcebergTableIdentifier(schemaTableName, catalogFactory.isNestedNamespaceEnabled())); + + icebergViews.remove(schemaTableName); + } + + @Override + protected void updateIcebergViewProperties( + ConnectorSession session, + SchemaTableName viewName, + Map properties) + { + Catalog catalog = catalogFactory.getCatalog(session); + if (!(catalog instanceof ViewCatalog)) { + throw new PrestoException(NOT_SUPPORTED, "This connector does not support updating Iceberg views for materialized views"); + } + ViewCatalog viewCatalog = (ViewCatalog) catalog; + + TableIdentifier viewIdentifier = toIcebergTableIdentifier(viewName, catalogFactory.isNestedNamespaceEnabled()); + View existingView = viewCatalog.loadView(viewIdentifier); + + Map tempProperties = new HashMap<>(existingView.properties()); + tempProperties.putAll(properties); + Map mergedProperties = ImmutableMap.copyOf(tempProperties); + + ViewBuilder viewBuilder = viewCatalog.buildView(viewIdentifier) + .withSchema(existingView.schema()) + .withDefaultNamespace(toIcebergNamespace(Optional.ofNullable(viewName.getSchemaName()), catalogFactory.isNestedNamespaceEnabled())) + .withQuery(VIEW_DIALECT, existingView.sqlFor(VIEW_DIALECT).sql()) + .withProperties(mergedProperties); + + viewBuilder.createOrReplace(); + + icebergViews.remove(viewName); + } } diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergNativeMetadataFactory.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergNativeMetadataFactory.java index 59b203c623303..b421a823efe2c 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergNativeMetadataFactory.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergNativeMetadataFactory.java @@ -17,12 +17,15 @@ import com.facebook.presto.common.type.TypeManager; import com.facebook.presto.hive.NodeVersion; import com.facebook.presto.iceberg.statistics.StatisticsFileCache; +import com.facebook.presto.spi.MaterializedViewDefinition; import com.facebook.presto.spi.connector.ConnectorMetadata; import com.facebook.presto.spi.function.StandardFunctionResolution; import com.facebook.presto.spi.plan.FilterStatsCalculatorService; import com.facebook.presto.spi.relation.RowExpressionService; import jakarta.inject.Inject; +import java.util.List; + import static java.util.Objects.requireNonNull; public class IcebergNativeMetadataFactory @@ -30,6 +33,7 @@ public class IcebergNativeMetadataFactory { final TypeManager typeManager; final JsonCodec commitTaskCodec; + final JsonCodec> columnMappingsCodec; final IcebergNativeCatalogFactory catalogFactory; final CatalogType catalogType; final StandardFunctionResolution functionResolution; @@ -47,6 +51,7 @@ public IcebergNativeMetadataFactory( StandardFunctionResolution functionResolution, RowExpressionService rowExpressionService, JsonCodec commitTaskCodec, + JsonCodec> columnMappingsCodec, NodeVersion nodeVersion, FilterStatsCalculatorService filterStatsCalculatorService, StatisticsFileCache statisticsFileCache, @@ -57,8 +62,8 @@ public IcebergNativeMetadataFactory( this.functionResolution = requireNonNull(functionResolution, "functionResolution is null"); this.rowExpressionService = requireNonNull(rowExpressionService, "rowExpressionService is null"); this.commitTaskCodec = requireNonNull(commitTaskCodec, "commitTaskCodec is null"); + this.columnMappingsCodec = requireNonNull(columnMappingsCodec, "columnMappingsCodec is null"); this.nodeVersion = requireNonNull(nodeVersion, "nodeVersion is null"); - requireNonNull(config, "config is null"); this.catalogType = config.getCatalogType(); this.filterStatsCalculatorService = requireNonNull(filterStatsCalculatorService, "filterStatsCalculatorService is null"); this.statisticsFileCache = requireNonNull(statisticsFileCache, "statisticsFileCache is null"); @@ -67,6 +72,6 @@ public IcebergNativeMetadataFactory( public ConnectorMetadata create() { - return new IcebergNativeMetadata(catalogFactory, typeManager, functionResolution, rowExpressionService, commitTaskCodec, catalogType, nodeVersion, filterStatsCalculatorService, statisticsFileCache, tableProperties); + return new IcebergNativeMetadata(catalogFactory, typeManager, functionResolution, rowExpressionService, commitTaskCodec, columnMappingsCodec, catalogType, nodeVersion, filterStatsCalculatorService, statisticsFileCache, tableProperties); } } diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergSessionProperties.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergSessionProperties.java index 38bf69a01c8a6..5f741b7bfb741 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergSessionProperties.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergSessionProperties.java @@ -71,6 +71,7 @@ public final class IcebergSessionProperties public static final String ROWS_FOR_METADATA_OPTIMIZATION_THRESHOLD = "rows_for_metadata_optimization_threshold"; public static final String STATISTICS_KLL_SKETCH_K_PARAMETER = "statistics_kll_sketch_k_parameter"; public static final String TARGET_SPLIT_SIZE_BYTES = "target_split_size_bytes"; + public static final String MATERIALIZED_VIEW_STORAGE_PREFIX = "materialized_view_storage_prefix"; private final List> sessionProperties; @@ -217,6 +218,13 @@ public IcebergSessionProperties( TARGET_SPLIT_SIZE_BYTES, "The target split size. Set to 0 to use the iceberg table's read.split.target-size property", 0L, + false)) + .add(stringProperty( + MATERIALIZED_VIEW_STORAGE_PREFIX, + "Default prefix for generated materialized view storage table names. " + + "This is only used when the materialized_view_storage_table_name table property is not explicitly set. " + + "When a custom table name is provided, it takes precedence over this prefix.", + icebergConfig.getMaterializedViewStoragePrefix(), false)); nessieConfig.ifPresent((config) -> propertiesBuilder @@ -361,4 +369,9 @@ public static Long getTargetSplitSize(ConnectorSession session) { return session.getProperty(TARGET_SPLIT_SIZE_BYTES, Long.class); } + + public static String getMaterializedViewStoragePrefix(ConnectorSession session) + { + return session.getProperty(MATERIALIZED_VIEW_STORAGE_PREFIX, String.class); + } } diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergTableHandle.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergTableHandle.java index 633f80d51eb33..fbff48e18d1f4 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergTableHandle.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergTableHandle.java @@ -15,6 +15,7 @@ import com.facebook.presto.hive.BaseHiveTableHandle; import com.facebook.presto.spi.ConnectorDeleteTableHandle; +import com.facebook.presto.spi.SchemaTableName; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; @@ -40,6 +41,7 @@ public class IcebergTableHandle private final Optional> equalityFieldIds; private final List sortOrder; private final List updatedColumns; + private final Optional materializedViewName; @JsonCreator public IcebergTableHandle( @@ -52,7 +54,8 @@ public IcebergTableHandle( @JsonProperty("partitionFieldIds") Optional> partitionFieldIds, @JsonProperty("equalityFieldIds") Optional> equalityFieldIds, @JsonProperty("sortOrder") List sortOrder, - @JsonProperty("updatedColumns") List updatedColumns) + @JsonProperty("updatedColumns") List updatedColumns, + @JsonProperty("materializedViewName") Optional materializedViewName) { super(schemaName, icebergTableName.getTableName()); @@ -65,6 +68,7 @@ public IcebergTableHandle( this.equalityFieldIds = requireNonNull(equalityFieldIds, "equalityFieldIds is null"); this.sortOrder = ImmutableList.copyOf(requireNonNull(sortOrder, "sortOrder is null")); this.updatedColumns = requireNonNull(updatedColumns, "updatedColumns is null"); + this.materializedViewName = requireNonNull(materializedViewName, "materializedViewName is null"); } @JsonProperty @@ -121,6 +125,12 @@ public List getUpdatedColumns() return updatedColumns; } + @JsonProperty + public Optional getMaterializedViewName() + { + return materializedViewName; + } + public IcebergTableHandle withUpdatedColumns(List updatedColumns) { return new IcebergTableHandle( @@ -133,7 +143,8 @@ public IcebergTableHandle withUpdatedColumns(List updatedCo partitionFieldIds, equalityFieldIds, sortOrder, - updatedColumns); + updatedColumns, + materializedViewName); } @Override @@ -152,13 +163,14 @@ public boolean equals(Object o) snapshotSpecified == that.snapshotSpecified && Objects.equals(sortOrder, that.sortOrder) && Objects.equals(tableSchemaJson, that.tableSchemaJson) && - Objects.equals(equalityFieldIds, that.equalityFieldIds); + Objects.equals(equalityFieldIds, that.equalityFieldIds) && + Objects.equals(materializedViewName, that.materializedViewName); } @Override public int hashCode() { - return Objects.hash(getSchemaName(), icebergTableName, sortOrder, snapshotSpecified, tableSchemaJson, equalityFieldIds); + return Objects.hash(getSchemaName(), icebergTableName, sortOrder, snapshotSpecified, tableSchemaJson, equalityFieldIds, materializedViewName); } @Override diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergTableProperties.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergTableProperties.java index 1af10c5f31316..5fb4b669a8382 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergTableProperties.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergTableProperties.java @@ -61,6 +61,8 @@ public class IcebergTableProperties public static final String PARTITIONING_PROPERTY = "partitioning"; public static final String SORTED_BY_PROPERTY = "sorted_by"; public static final String LOCATION_PROPERTY = "location"; + public static final String MATERIALIZED_VIEW_STORAGE_SCHEMA = "materialized_view_storage_schema"; + public static final String MATERIALIZED_VIEW_STORAGE_TABLE_NAME = "materialized_view_storage_table_name"; /** * Please use {@link TableProperties#FORMAT_VERSION} @@ -219,6 +221,16 @@ public IcebergTableProperties(IcebergConfig icebergConfig) "Desired size of split to generate during query scan planning", TableProperties.SPLIT_SIZE_DEFAULT, false)) + .add(stringProperty( + MATERIALIZED_VIEW_STORAGE_SCHEMA, + "Schema for the materialized view storage table (defaults to same schema as the materialized view)", + null, + true)) + .add(stringProperty( + MATERIALIZED_VIEW_STORAGE_TABLE_NAME, + "Custom name for the materialized view storage table (defaults to generated name)", + null, + true)) .build(); deprecatedPropertyMetadata = properties.stream() @@ -351,6 +363,16 @@ public static Long getTargetSplitSize(Map tableProperties) return (Long) tableProperties.get(TableProperties.SPLIT_SIZE); } + public static Optional getMaterializedViewStorageSchema(Map tableProperties) + { + return Optional.ofNullable((String) tableProperties.get(MATERIALIZED_VIEW_STORAGE_SCHEMA)); + } + + public static Optional getMaterializedViewStorageTableName(Map tableProperties) + { + return Optional.ofNullable((String) tableProperties.get(MATERIALIZED_VIEW_STORAGE_TABLE_NAME)); + } + @VisibleForTesting protected Object getTablePropertyWithDeprecationWarning(ConnectorSession session, Map tableProperties, String keyName) { diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergUtil.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergUtil.java index 377500f81a69b..f9db39adfe3be 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergUtil.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergUtil.java @@ -276,7 +276,16 @@ public static View getNativeIcebergView(IcebergNativeCatalogFactory catalogFacto if (!(catalog instanceof ViewCatalog)) { throw new PrestoException(NOT_SUPPORTED, "This connector does not support get views"); } - return ((ViewCatalog) catalog).loadView(toIcebergTableIdentifier(table, catalogFactory.isNestedNamespaceEnabled())); + return ((ViewCatalog) catalog).loadView(toIcebergTableIdentifier(getBaseSchemaTableName(table), catalogFactory.isNestedNamespaceEnabled())); + } + + /** + * Removes Iceberg-specific suffixes from the table name + */ + private static SchemaTableName getBaseSchemaTableName(SchemaTableName table) + { + IcebergTableName icebergTableName = IcebergTableName.from(table.getTableName()); + return new SchemaTableName(table.getSchemaName(), icebergTableName.getTableName()); } public static List getPartitionKeyColumnHandles(IcebergTableHandle tableHandle, Table table, TypeManager typeManager) diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergWritableTableHandle.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergWritableTableHandle.java index b518103106421..91ae4a0d91f2a 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergWritableTableHandle.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergWritableTableHandle.java @@ -14,11 +14,13 @@ package com.facebook.presto.iceberg; import com.facebook.presto.hive.HiveCompressionCodec; +import com.facebook.presto.spi.SchemaTableName; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; import java.util.List; import java.util.Map; +import java.util.Optional; import static java.util.Objects.requireNonNull; @@ -34,6 +36,7 @@ public class IcebergWritableTableHandle private final HiveCompressionCodec compressionCodec; private final Map storageProperties; private final List sortOrder; + private final Optional materializedViewName; public IcebergWritableTableHandle( String schemaName, @@ -46,6 +49,22 @@ public IcebergWritableTableHandle( HiveCompressionCodec compressionCodec, Map storageProperties, List sortOrder) + { + this(schemaName, tableName, schema, partitionSpec, inputColumns, outputPath, fileFormat, compressionCodec, storageProperties, sortOrder, Optional.empty()); + } + + public IcebergWritableTableHandle( + String schemaName, + IcebergTableName tableName, + PrestoIcebergSchema schema, + PrestoIcebergPartitionSpec partitionSpec, + List inputColumns, + String outputPath, + FileFormat fileFormat, + HiveCompressionCodec compressionCodec, + Map storageProperties, + List sortOrder, + Optional materializedViewName) { this.schemaName = requireNonNull(schemaName, "schemaName is null"); this.tableName = requireNonNull(tableName, "tableName is null"); @@ -57,6 +76,7 @@ public IcebergWritableTableHandle( this.compressionCodec = requireNonNull(compressionCodec, "compressionCodec is null"); this.storageProperties = requireNonNull(storageProperties, "storageProperties is null"); this.sortOrder = ImmutableList.copyOf(requireNonNull(sortOrder, "sortOrder is null")); + this.materializedViewName = requireNonNull(materializedViewName, "materializedViewName is null"); } @JsonProperty @@ -124,4 +144,10 @@ public List getSortOrder() { return sortOrder; } + + @JsonProperty + public Optional getMaterializedViewName() + { + return materializedViewName; + } } diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/optimizer/IcebergEqualityDeleteAsJoin.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/optimizer/IcebergEqualityDeleteAsJoin.java index fa542cd2972da..36b41a3e3c5ff 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/optimizer/IcebergEqualityDeleteAsJoin.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/optimizer/IcebergEqualityDeleteAsJoin.java @@ -354,7 +354,8 @@ private TableScanNode createDeletesTableScan(ImmutableMap assignmentsBuilder = ImmutableMap.builder() diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergConfig.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergConfig.java index 4a8650175e553..bf25891c38d57 100644 --- a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergConfig.java +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergConfig.java @@ -73,7 +73,8 @@ public void testDefaults() .setMetricsMaxInferredColumn(METRICS_MAX_INFERRED_COLUMN_DEFAULTS_DEFAULT) .setManifestCacheMaxChunkSize(succinctDataSize(2, MEGABYTE)) .setMaxStatisticsFileCacheSize(succinctDataSize(256, MEGABYTE)) - .setStatisticsKllSketchKParameter(1024)); + .setStatisticsKllSketchKParameter(1024) + .setMaterializedViewStoragePrefix("__mv_storage__")); } @Test @@ -109,6 +110,7 @@ public void testExplicitPropertyMappings() .put("iceberg.metrics-max-inferred-column", "16") .put("iceberg.max-statistics-file-cache-size", "512MB") .put("iceberg.statistics-kll-sketch-k-parameter", "4096") + .put("iceberg.materialized-view-storage-prefix", "custom_mv_prefix") .build(); IcebergConfig expected = new IcebergConfig() @@ -140,7 +142,8 @@ public void testExplicitPropertyMappings() .setMetadataDeleteAfterCommit(true) .setMetricsMaxInferredColumn(16) .setMaxStatisticsFileCacheSize(succinctDataSize(512, MEGABYTE)) - .setStatisticsKllSketchKParameter(4096); + .setStatisticsKllSketchKParameter(4096) + .setMaterializedViewStoragePrefix("custom_mv_prefix"); assertFullMapping(properties, expected); } diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergMaterializedViewMetadata.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergMaterializedViewMetadata.java new file mode 100644 index 0000000000000..d54ea608c2d8c --- /dev/null +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergMaterializedViewMetadata.java @@ -0,0 +1,532 @@ +/* + * 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 com.facebook.presto.iceberg; + +import com.facebook.airlift.http.server.testing.TestingHttpServer; +import com.facebook.presto.testing.QueryRunner; +import com.facebook.presto.tests.AbstractTestQueryFramework; +import com.google.common.collect.ImmutableMap; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.rest.RESTCatalog; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.view.View; +import org.assertj.core.util.Files; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import java.io.File; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +import static com.facebook.presto.iceberg.CatalogType.REST; +import static com.facebook.presto.iceberg.IcebergAbstractMetadata.PRESTO_MATERIALIZED_VIEW_BASE_TABLES; +import static com.facebook.presto.iceberg.IcebergAbstractMetadata.PRESTO_MATERIALIZED_VIEW_COLUMN_MAPPINGS; +import static com.facebook.presto.iceberg.IcebergAbstractMetadata.PRESTO_MATERIALIZED_VIEW_LAST_REFRESH_SNAPSHOT_ID; +import static com.facebook.presto.iceberg.IcebergAbstractMetadata.PRESTO_MATERIALIZED_VIEW_ORIGINAL_SQL; +import static com.facebook.presto.iceberg.IcebergAbstractMetadata.PRESTO_MATERIALIZED_VIEW_OWNER; +import static com.facebook.presto.iceberg.IcebergAbstractMetadata.PRESTO_MATERIALIZED_VIEW_SECURITY_MODE; +import static com.facebook.presto.iceberg.IcebergAbstractMetadata.PRESTO_MATERIALIZED_VIEW_STORAGE_SCHEMA; +import static com.facebook.presto.iceberg.IcebergAbstractMetadata.PRESTO_MATERIALIZED_VIEW_STORAGE_TABLE_NAME; +import static com.facebook.presto.iceberg.rest.IcebergRestTestUtil.getRestServer; +import static com.facebook.presto.iceberg.rest.IcebergRestTestUtil.restConnectorProperties; +import static com.google.common.io.MoreFiles.deleteRecursively; +import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; + +@Test(singleThreaded = true) +public class TestIcebergMaterializedViewMetadata + extends AbstractTestQueryFramework +{ + private File warehouseLocation; + private TestingHttpServer restServer; + private String serverUri; + + @BeforeClass + @Override + public void init() + throws Exception + { + warehouseLocation = Files.newTemporaryFolder(); + + restServer = getRestServer(warehouseLocation.getAbsolutePath()); + restServer.start(); + + serverUri = restServer.getBaseUrl().toString(); + super.init(); + } + + @AfterClass(alwaysRun = true) + public void tearDown() + throws Exception + { + if (restServer != null) { + restServer.stop(); + } + deleteRecursively(warehouseLocation.toPath(), ALLOW_INSECURE); + } + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + return IcebergQueryRunner.builder() + .setCatalogType(REST) + .setExtraConnectorProperties(restConnectorProperties(serverUri)) + .setDataDirectory(Optional.of(warehouseLocation.toPath())) + .setSchemaName("test_schema") + .setCreateTpchTables(false) + .setExtraProperties(ImmutableMap.of("experimental.legacy-materialized-views", "false")) + .build().getQueryRunner(); + } + + @Test + public void testMaterializedViewSnapshotTracking() + throws Exception + { + assertUpdate("CREATE TABLE test_snapshot_base (id BIGINT, value BIGINT)"); + assertUpdate("INSERT INTO test_snapshot_base VALUES (1, 100)", 1); + + assertUpdate("CREATE MATERIALIZED VIEW test_snapshot_mv AS SELECT id, value FROM test_snapshot_base"); + + RESTCatalog catalog = new RESTCatalog(); + Map catalogProps = new HashMap<>(); + catalogProps.put("uri", serverUri); + catalogProps.put("warehouse", warehouseLocation.getAbsolutePath()); + catalog.initialize("test_catalog", catalogProps); + + try { + TableIdentifier viewId = TableIdentifier.of(Namespace.of("test_schema"), "test_snapshot_mv"); + + View viewBeforeRefresh = catalog.loadView(viewId); + String lastRefreshBefore = viewBeforeRefresh.properties().get(PRESTO_MATERIALIZED_VIEW_LAST_REFRESH_SNAPSHOT_ID); + assertNull(lastRefreshBefore, "Expected last_refresh_snapshot_id to be null before refresh"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_snapshot_mv", 1); + + View viewAfterRefresh = catalog.loadView(viewId); + String lastRefreshAfter = viewAfterRefresh.properties().get(PRESTO_MATERIALIZED_VIEW_LAST_REFRESH_SNAPSHOT_ID); + assertNotNull(lastRefreshAfter, "Expected last_refresh_snapshot_id to be set after refresh"); + + String baseSnapshot = viewAfterRefresh.properties().get("presto.materialized_view.base_snapshot.test_schema.test_snapshot_base"); + assertNotNull(baseSnapshot, "Expected base table snapshot ID to be tracked"); + + assertUpdate("INSERT INTO test_snapshot_base VALUES (2, 200)", 1); + assertUpdate("REFRESH MATERIALIZED VIEW test_snapshot_mv", 2); + + View viewAfterSecondRefresh = catalog.loadView(viewId); + String lastRefreshAfter2 = viewAfterSecondRefresh.properties().get(PRESTO_MATERIALIZED_VIEW_LAST_REFRESH_SNAPSHOT_ID); + String baseSnapshot2 = viewAfterSecondRefresh.properties().get("presto.materialized_view.base_snapshot.test_schema.test_snapshot_base"); + + assertNotEquals(lastRefreshAfter2, lastRefreshAfter, "Expected last_refresh_snapshot_id to change after second refresh"); + assertNotEquals(baseSnapshot2, baseSnapshot, "Expected base table snapshot ID to change after INSERT"); + + assertUpdate("CREATE TABLE test_empty_snapshot (id BIGINT)"); + assertUpdate("CREATE MATERIALIZED VIEW test_empty_mv AS SELECT id FROM test_empty_snapshot"); + assertUpdate("REFRESH MATERIALIZED VIEW test_empty_mv", 0); + + TableIdentifier emptyViewId = TableIdentifier.of(Namespace.of("test_schema"), "test_empty_mv"); + View emptyView = catalog.loadView(emptyViewId); + String emptySnapshot = emptyView.properties().get("presto.materialized_view.base_snapshot.test_schema.test_empty_snapshot"); + + assertEquals(emptySnapshot, "0", "Expected empty base table snapshot ID to be 0"); + + assertUpdate("DROP MATERIALIZED VIEW test_empty_mv"); + assertUpdate("DROP TABLE test_empty_snapshot"); + } + finally { + catalog.close(); + } + + assertUpdate("DROP MATERIALIZED VIEW test_snapshot_mv"); + assertUpdate("DROP TABLE test_snapshot_base"); + } + + @Test + public void testMaterializedViewWithComplexColumnMappings() + throws Exception + { + assertUpdate("CREATE TABLE test_mapping_orders (order_id BIGINT, customer_id BIGINT, product_id BIGINT, order_date DATE, amount BIGINT)"); + assertUpdate("CREATE TABLE test_mapping_customers (customer_id BIGINT, customer_name VARCHAR, region VARCHAR)"); + assertUpdate("CREATE TABLE test_mapping_products (product_id BIGINT, product_name VARCHAR, category VARCHAR)"); + + assertUpdate("INSERT INTO test_mapping_orders VALUES (1, 100, 1, DATE '2024-01-01', 500), (2, 200, 2, DATE '2024-01-02', 750)", 2); + assertUpdate("INSERT INTO test_mapping_customers VALUES (100, 'Alice', 'US'), (200, 'Bob', 'EU')", 2); + assertUpdate("INSERT INTO test_mapping_products VALUES (1, 'Widget', 'Electronics'), (2, 'Gadget', 'Electronics')", 2); + + assertUpdate("CREATE MATERIALIZED VIEW test_mapping_mv AS " + + "SELECT o.order_id, c.customer_name, c.region, o.order_date, o.amount, p.product_name, p.category " + + "FROM test_mapping_orders o " + + "JOIN test_mapping_customers c ON o.customer_id = c.customer_id " + + "JOIN test_mapping_products p ON o.product_id = p.product_id"); + + assertQuery("SELECT COUNT(*) FROM test_mapping_mv", "SELECT 2"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_mapping_mv", 2); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_mapping_mv\"", "SELECT 2"); + + assertQuery("SELECT order_id, customer_name, region FROM test_mapping_mv WHERE order_id = 1", + "VALUES (1, 'Alice', 'US')"); + + RESTCatalog catalog = new RESTCatalog(); + Map catalogProps = new HashMap<>(); + catalogProps.put("uri", serverUri); + catalogProps.put("warehouse", warehouseLocation.getAbsolutePath()); + catalog.initialize("test_catalog", catalogProps); + + try { + TableIdentifier viewId = TableIdentifier.of(Namespace.of("test_schema"), "test_mapping_mv"); + View view = catalog.loadView(viewId); + + String columnMappings = view.properties().get("presto.materialized_view.column_mappings"); + assertNotNull(columnMappings, "Expected column_mappings property to be set"); + + assertFalse(columnMappings.isEmpty() || columnMappings.equals("[]"), "Expected non-empty column mappings for multi-table join"); + } + finally { + catalog.close(); + } + + assertUpdate("INSERT INTO test_mapping_orders VALUES (3, 100, 1, DATE '2024-01-03', 1000)", 1); + assertUpdate("REFRESH MATERIALIZED VIEW test_mapping_mv", 3); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_mapping_mv\"", "SELECT 3"); + + assertUpdate("DROP MATERIALIZED VIEW test_mapping_mv"); + assertUpdate("DROP TABLE test_mapping_products"); + assertUpdate("DROP TABLE test_mapping_customers"); + assertUpdate("DROP TABLE test_mapping_orders"); + } + + @Test + public void testMaterializedViewWithSpecialCharactersInTableNames() + throws Exception + { + assertUpdate("CREATE TABLE test_base_123 (id BIGINT, value_1 BIGINT)"); + assertUpdate("CREATE TABLE test_base_456_special (id BIGINT, value_2 BIGINT)"); + + assertUpdate("INSERT INTO test_base_123 VALUES (1, 100), (2, 200)", 2); + assertUpdate("INSERT INTO test_base_456_special VALUES (1, 300), (2, 400)", 2); + + assertUpdate("CREATE MATERIALIZED VIEW test_special_chars_mv AS " + + "SELECT a.id, a.value_1, b.value_2 " + + "FROM test_base_123 a " + + "JOIN test_base_456_special b ON a.id = b.id"); + + assertQuery("SELECT COUNT(*) FROM test_special_chars_mv", "SELECT 2"); + assertQuery("SELECT * FROM test_special_chars_mv ORDER BY id", + "VALUES (1, 100, 300), (2, 200, 400)"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_special_chars_mv", 2); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_special_chars_mv\"", "SELECT 2"); + assertQuery("SELECT * FROM \"__mv_storage__test_special_chars_mv\" ORDER BY id", + "VALUES (1, 100, 300), (2, 200, 400)"); + + RESTCatalog catalog = new RESTCatalog(); + Map catalogProps = new HashMap<>(); + catalogProps.put("uri", serverUri); + catalogProps.put("warehouse", warehouseLocation.getAbsolutePath()); + catalog.initialize("test_catalog", catalogProps); + + try { + TableIdentifier viewId = TableIdentifier.of(Namespace.of("test_schema"), "test_special_chars_mv"); + View view = catalog.loadView(viewId); + + String baseTables = view.properties().get("presto.materialized_view.base_tables"); + assertNotNull(baseTables, "Expected base_tables property to be set"); + + assertTrue(baseTables.contains("test_base_123"), "Expected base_tables to contain 'test_base_123', got: " + baseTables); + assertTrue(baseTables.contains("test_base_456_special"), "Expected base_tables to contain 'test_base_456_special', got: " + baseTables); + + String snapshot1 = view.properties().get("presto.materialized_view.base_snapshot.test_schema.test_base_123"); + String snapshot2 = view.properties().get("presto.materialized_view.base_snapshot.test_schema.test_base_456_special"); + + assertNotNull(snapshot1, "Expected snapshot for test_base_123 to be tracked"); + assertNotNull(snapshot2, "Expected snapshot for test_base_456_special to be tracked"); + } + finally { + catalog.close(); + } + + assertUpdate("INSERT INTO test_base_123 VALUES (3, 500)", 1); + assertUpdate("INSERT INTO test_base_456_special VALUES (3, 600)", 1); + + assertQuery("SELECT COUNT(*) FROM test_special_chars_mv", "SELECT 3"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_special_chars_mv", 3); + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_special_chars_mv\"", "SELECT 3"); + + assertUpdate("DROP MATERIALIZED VIEW test_special_chars_mv"); + assertUpdate("DROP TABLE test_base_456_special"); + assertUpdate("DROP TABLE test_base_123"); + } + + @Test + public void testMaterializedViewOtherValidationErrors() + throws Exception + { + assertUpdate("CREATE TABLE test_other_validation_base (id BIGINT, name VARCHAR)"); + assertUpdate("INSERT INTO test_other_validation_base VALUES (1, 'Alice')", 1); + + RESTCatalog catalog = new RESTCatalog(); + Map catalogProps = new HashMap<>(); + catalogProps.put("uri", serverUri); + catalogProps.put("warehouse", warehouseLocation.getAbsolutePath()); + catalog.initialize("test_catalog", catalogProps); + + try { + TableIdentifier viewId1 = TableIdentifier.of(Namespace.of("test_schema"), "test_mv_empty_base_tables"); + Map properties = new HashMap<>(); + properties.put(PRESTO_MATERIALIZED_VIEW_ORIGINAL_SQL, "SELECT 1 as id"); + properties.put(PRESTO_MATERIALIZED_VIEW_STORAGE_SCHEMA, "test_schema"); + properties.put(PRESTO_MATERIALIZED_VIEW_STORAGE_TABLE_NAME, "storage_empty_base"); + properties.put(PRESTO_MATERIALIZED_VIEW_COLUMN_MAPPINGS, "[]"); + properties.put(PRESTO_MATERIALIZED_VIEW_BASE_TABLES, ""); + properties.put(PRESTO_MATERIALIZED_VIEW_OWNER, "test_user"); + properties.put(PRESTO_MATERIALIZED_VIEW_SECURITY_MODE, "DEFINER"); + + assertUpdate("CREATE TABLE storage_empty_base (id BIGINT)"); + + catalog.buildView(viewId1) + .withSchema(new org.apache.iceberg.Schema( + Types.NestedField.required(1, "id", Types.LongType.get()))) + .withQuery("spark", "SELECT 1 as id") + .withDefaultNamespace(Namespace.of("test_schema")) + .withProperties(properties) + .create(); + + assertQuery("SELECT * FROM test_mv_empty_base_tables", "SELECT 1"); + + catalog.dropView(viewId1); + assertUpdate("DROP TABLE storage_empty_base"); + + TableIdentifier viewId2 = TableIdentifier.of(Namespace.of("test_schema"), "test_mv_invalid_json"); + Map properties2 = new HashMap<>(); + properties2.put(PRESTO_MATERIALIZED_VIEW_ORIGINAL_SQL, "SELECT id FROM test_other_validation_base"); + properties2.put(PRESTO_MATERIALIZED_VIEW_STORAGE_SCHEMA, "test_schema"); + properties2.put(PRESTO_MATERIALIZED_VIEW_STORAGE_TABLE_NAME, "storage_invalid_json"); + properties2.put(PRESTO_MATERIALIZED_VIEW_BASE_TABLES, "test_schema.test_other_validation_base"); + properties2.put(PRESTO_MATERIALIZED_VIEW_COLUMN_MAPPINGS, "{invalid json here"); + properties2.put(PRESTO_MATERIALIZED_VIEW_OWNER, "test_user"); + properties2.put(PRESTO_MATERIALIZED_VIEW_SECURITY_MODE, "DEFINER"); + + assertUpdate("CREATE TABLE storage_invalid_json (id BIGINT)"); + + catalog.buildView(viewId2) + .withSchema(new org.apache.iceberg.Schema( + Types.NestedField.required(1, "id", Types.LongType.get()))) + .withQuery("spark", "SELECT id FROM test_other_validation_base") + .withDefaultNamespace(Namespace.of("test_schema")) + .withProperties(properties2) + .create(); + + assertQueryFails("SELECT * FROM test_mv_invalid_json", + ".*Invalid JSON string.*"); + + catalog.dropView(viewId2); + assertUpdate("DROP TABLE storage_invalid_json"); + + TableIdentifier viewId3 = TableIdentifier.of(Namespace.of("test_schema"), "test_mv_nonexistent_base"); + Map properties3 = new HashMap<>(); + properties3.put(PRESTO_MATERIALIZED_VIEW_ORIGINAL_SQL, "SELECT id FROM nonexistent_table"); + properties3.put(PRESTO_MATERIALIZED_VIEW_STORAGE_SCHEMA, "test_schema"); + properties3.put(PRESTO_MATERIALIZED_VIEW_STORAGE_TABLE_NAME, "storage_nonexistent_base"); + properties3.put(PRESTO_MATERIALIZED_VIEW_BASE_TABLES, "test_schema.nonexistent_table"); + properties3.put(PRESTO_MATERIALIZED_VIEW_COLUMN_MAPPINGS, "[]"); + properties3.put(PRESTO_MATERIALIZED_VIEW_OWNER, "test_user"); + properties3.put(PRESTO_MATERIALIZED_VIEW_SECURITY_MODE, "DEFINER"); + + assertUpdate("CREATE TABLE storage_nonexistent_base (id BIGINT)"); + + catalog.buildView(viewId3) + .withSchema(new org.apache.iceberg.Schema( + Types.NestedField.required(1, "id", Types.LongType.get()))) + .withQuery("spark", "SELECT id FROM nonexistent_table") + .withDefaultNamespace(Namespace.of("test_schema")) + .withProperties(properties3) + .create(); + + assertQueryFails("SELECT * FROM test_mv_nonexistent_base", + ".*(does not exist|not found).*"); + + catalog.dropView(viewId3); + assertUpdate("DROP TABLE storage_nonexistent_base"); + } + finally { + catalog.close(); + } + + assertUpdate("CREATE TABLE existing_storage_table (id BIGINT, value BIGINT)"); + assertUpdate("INSERT INTO existing_storage_table VALUES (1, 100)", 1); + + assertQueryFails("CREATE MATERIALIZED VIEW test_mv_duplicate_storage " + + "WITH (materialized_view_storage_table_name = 'existing_storage_table') " + + "AS SELECT id, name FROM test_other_validation_base", + ".*already exists.*"); + + assertUpdate("DROP TABLE existing_storage_table"); + assertUpdate("DROP TABLE test_other_validation_base"); + } + + @Test + public void testMaterializedViewInvalidBaseTableNameFormat() + throws Exception + { + assertUpdate("CREATE TABLE test_format_base (id BIGINT, name VARCHAR)"); + assertUpdate("INSERT INTO test_format_base VALUES (1, 'Alice')", 1); + + assertUpdate("CREATE TABLE storage_table_1 (id BIGINT)"); + assertUpdate("CREATE TABLE storage_table_2 (id BIGINT)"); + assertUpdate("CREATE TABLE storage_table_3 (id BIGINT)"); + assertUpdate("CREATE TABLE storage_table_4 (id BIGINT)"); + assertUpdate("CREATE TABLE storage_table_5 (id BIGINT)"); + assertUpdate("CREATE TABLE storage_table_6 (id BIGINT)"); + + RESTCatalog catalog = new RESTCatalog(); + Map catalogProps = new HashMap<>(); + catalogProps.put("uri", serverUri); + catalogProps.put("warehouse", warehouseLocation.getAbsolutePath()); + catalog.initialize("test_catalog", catalogProps); + + try { + TableIdentifier viewId1 = TableIdentifier.of(Namespace.of("test_schema"), "test_mv_no_schema"); + Map properties1 = createValidMvProperties("storage_table_1"); + properties1.put(PRESTO_MATERIALIZED_VIEW_BASE_TABLES, "table_only"); + + catalog.buildView(viewId1) + .withSchema(new org.apache.iceberg.Schema( + Types.NestedField.required(1, "id", Types.LongType.get()))) + .withQuery("spark", "SELECT id FROM table_only") + .withDefaultNamespace(Namespace.of("test_schema")) + .withProperties(properties1) + .create(); + + assertQueryFails("SELECT * FROM test_mv_no_schema", + ".*Invalid base table name format: table_only.*"); + + catalog.dropView(viewId1); + + TableIdentifier viewId2 = TableIdentifier.of(Namespace.of("test_schema"), "test_mv_empty_schema"); + Map properties2 = createValidMvProperties("storage_table_2"); + properties2.put(PRESTO_MATERIALIZED_VIEW_BASE_TABLES, "schema."); + + catalog.buildView(viewId2) + .withSchema(new org.apache.iceberg.Schema( + Types.NestedField.required(1, "id", Types.LongType.get()))) + .withQuery("spark", "SELECT id FROM schema.") + .withDefaultNamespace(Namespace.of("test_schema")) + .withProperties(properties2) + .create(); + + assertQueryFails("SELECT * FROM test_mv_empty_schema", + ".*Invalid base table name format: schema\\..*"); + + catalog.dropView(viewId2); + + TableIdentifier viewId3 = TableIdentifier.of(Namespace.of("test_schema"), "test_mv_empty_table"); + Map properties3 = createValidMvProperties("storage_table_3"); + properties3.put(PRESTO_MATERIALIZED_VIEW_BASE_TABLES, ".table"); + + catalog.buildView(viewId3) + .withSchema(new org.apache.iceberg.Schema( + Types.NestedField.required(1, "id", Types.LongType.get()))) + .withQuery("spark", "SELECT id FROM .table") + .withDefaultNamespace(Namespace.of("test_schema")) + .withProperties(properties3) + .create(); + + assertQueryFails("SELECT * FROM test_mv_empty_table", + ".*Invalid base table name format: \\.table.*"); + + catalog.dropView(viewId3); + + TableIdentifier viewId4 = TableIdentifier.of(Namespace.of("test_schema"), "test_mv_double_dots"); + Map properties4 = createValidMvProperties("storage_table_4"); + properties4.put(PRESTO_MATERIALIZED_VIEW_BASE_TABLES, "schema..table"); + + catalog.buildView(viewId4) + .withSchema(new org.apache.iceberg.Schema( + Types.NestedField.required(1, "id", Types.LongType.get()))) + .withQuery("spark", "SELECT id FROM schema..table") + .withDefaultNamespace(Namespace.of("test_schema")) + .withProperties(properties4) + .create(); + + assertQueryFails("SELECT * FROM test_mv_double_dots", + ".*Invalid base table name format: schema\\.\\.table.*"); + + catalog.dropView(viewId4); + + TableIdentifier viewId5 = TableIdentifier.of(Namespace.of("test_schema"), "test_mv_too_many_parts"); + Map properties5 = createValidMvProperties("storage_table_5"); + properties5.put(PRESTO_MATERIALIZED_VIEW_BASE_TABLES, "a.b.c"); + + catalog.buildView(viewId5) + .withSchema(new org.apache.iceberg.Schema( + Types.NestedField.required(1, "id", Types.LongType.get()))) + .withQuery("spark", "SELECT id FROM a.b.c") + .withDefaultNamespace(Namespace.of("test_schema")) + .withProperties(properties5) + .create(); + + assertQueryFails("SELECT * FROM test_mv_too_many_parts", + ".*Invalid base table name format: a\\.b\\.c.*"); + + catalog.dropView(viewId5); + + TableIdentifier viewId6 = TableIdentifier.of(Namespace.of("test_schema"), "test_mv_no_separator"); + Map properties6 = createValidMvProperties("storage_table_6"); + properties6.put(PRESTO_MATERIALIZED_VIEW_BASE_TABLES, "schema_table"); + + catalog.buildView(viewId6) + .withSchema(new org.apache.iceberg.Schema( + Types.NestedField.required(1, "id", Types.LongType.get()))) + .withQuery("spark", "SELECT id FROM test_schema.test_format_base") + .withDefaultNamespace(Namespace.of("test_schema")) + .withProperties(properties6) + .create(); + + assertQueryFails("SELECT * FROM test_mv_no_separator", + ".*Invalid base table name format: schema_table.*"); + + catalog.dropView(viewId6); + } + finally { + catalog.close(); + } + + assertUpdate("DROP TABLE test_format_base"); + } + + private Map createValidMvProperties(String storageTableName) + { + Map properties = new HashMap<>(); + properties.put(PRESTO_MATERIALIZED_VIEW_ORIGINAL_SQL, "SELECT id FROM test_format_base"); + properties.put(PRESTO_MATERIALIZED_VIEW_STORAGE_SCHEMA, "test_schema"); + properties.put(PRESTO_MATERIALIZED_VIEW_STORAGE_TABLE_NAME, storageTableName); + properties.put(PRESTO_MATERIALIZED_VIEW_COLUMN_MAPPINGS, "[]"); + properties.put(PRESTO_MATERIALIZED_VIEW_OWNER, "test_user"); + properties.put(PRESTO_MATERIALIZED_VIEW_SECURITY_MODE, "DEFINER"); + return properties; + } +} diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergMaterializedViewOptimizer.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergMaterializedViewOptimizer.java new file mode 100644 index 0000000000000..8480b987817a1 --- /dev/null +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergMaterializedViewOptimizer.java @@ -0,0 +1,152 @@ +/* + * 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 com.facebook.presto.iceberg; + +import com.facebook.airlift.http.server.testing.TestingHttpServer; +import com.facebook.presto.testing.QueryRunner; +import com.facebook.presto.tests.AbstractTestQueryFramework; +import com.google.common.collect.ImmutableMap; +import org.assertj.core.util.Files; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import java.io.File; +import java.util.Optional; + +import static com.facebook.presto.iceberg.CatalogType.REST; +import static com.facebook.presto.iceberg.rest.IcebergRestTestUtil.getRestServer; +import static com.facebook.presto.iceberg.rest.IcebergRestTestUtil.restConnectorProperties; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.anyTree; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.join; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.tableScan; +import static com.google.common.io.MoreFiles.deleteRecursively; +import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; + +/** + * Plan-level tests for MaterializedView optimizer rule. + * Verifies that the optimizer correctly decides when to use UNION stitching vs full recompute. + */ +@Test(singleThreaded = true) +public class TestIcebergMaterializedViewOptimizer + extends AbstractTestQueryFramework +{ + private File warehouseLocation; + private TestingHttpServer restServer; + + @BeforeClass + @Override + public void init() + throws Exception + { + warehouseLocation = Files.newTemporaryFolder(); + restServer = getRestServer(warehouseLocation.getAbsolutePath()); + restServer.start(); + super.init(); + } + + @AfterClass(alwaysRun = true) + public void tearDown() + throws Exception + { + if (restServer != null) { + restServer.stop(); + } + if (warehouseLocation != null) { + deleteRecursively(warehouseLocation.toPath(), ALLOW_INSECURE); + } + } + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + return IcebergQueryRunner.builder() + .setCatalogType(REST) + .setExtraConnectorProperties(restConnectorProperties(restServer.getBaseUrl().toString())) + .setDataDirectory(Optional.of(warehouseLocation.toPath())) + .setSchemaName("test_schema") + .setCreateTpchTables(false) + .setExtraProperties(ImmutableMap.of("experimental.legacy-materialized-views", "false")) + .build().getQueryRunner(); + } + + @Test + public void testBasicOptimization() + { + assertUpdate("CREATE TABLE base_no_parts (id BIGINT, value BIGINT)"); + assertUpdate("INSERT INTO base_no_parts VALUES (1, 100), (2, 200)", 2); + + assertUpdate("CREATE MATERIALIZED VIEW mv_no_parts AS SELECT id, value FROM base_no_parts"); + getQueryRunner().execute("REFRESH MATERIALIZED VIEW mv_no_parts"); + + assertUpdate("INSERT INTO base_no_parts VALUES (3, 300)", 1); + + assertPlan("SELECT * FROM mv_no_parts", + anyTree(tableScan("base_no_parts"))); + + getQueryRunner().execute("REFRESH MATERIALIZED VIEW mv_no_parts"); + + assertPlan("SELECT * FROM mv_no_parts", + anyTree(tableScan("__mv_storage__mv_no_parts"))); + + assertUpdate("DROP MATERIALIZED VIEW mv_no_parts"); + assertUpdate("DROP TABLE base_no_parts"); + } + + @Test + public void testMultiTableStaleness() + { + // Create two partitioned base tables + assertUpdate("CREATE TABLE orders (order_id BIGINT, customer_id BIGINT, ds VARCHAR) " + + "WITH (partitioning = ARRAY['ds'])"); + assertUpdate("CREATE TABLE customers (customer_id BIGINT, name VARCHAR, reg_date VARCHAR) " + + "WITH (partitioning = ARRAY['reg_date'])"); + + assertUpdate("INSERT INTO orders VALUES (1, 100, '2024-01-01')", 1); + assertUpdate("INSERT INTO customers VALUES (100, 'Alice', '2024-01-01')", 1); + + // Create JOIN MV with partition columns in output + assertUpdate("CREATE MATERIALIZED VIEW mv_join AS " + + "SELECT o.order_id, c.name, o.ds, c.reg_date " + + "FROM orders o JOIN customers c ON o.customer_id = c.customer_id"); + getQueryRunner().execute("REFRESH MATERIALIZED VIEW mv_join"); + + // Make one table stale + assertUpdate("INSERT INTO orders VALUES (2, 200, '2024-01-02')", 1); + + assertPlan("SELECT * FROM mv_join", + anyTree( + anyTree( + join( + anyTree(tableScan("orders")), + anyTree(tableScan("customers")))))); + + getQueryRunner().execute("REFRESH MATERIALIZED VIEW mv_join"); + + // Make both tables stale + assertUpdate("INSERT INTO orders VALUES (2, 200, '2024-01-02')", 1); + assertUpdate("INSERT INTO customers VALUES (200, 'Bob', '2024-01-02')", 1); + + assertPlan("SELECT * FROM mv_join", + anyTree( + anyTree( + join( + anyTree(tableScan("orders")), + anyTree(tableScan("customers")))))); + assertUpdate("DROP MATERIALIZED VIEW mv_join"); + assertUpdate("DROP TABLE customers"); + assertUpdate("DROP TABLE orders"); + } +} diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergMaterializedViews.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergMaterializedViews.java new file mode 100644 index 0000000000000..ee4d4d51eb16e --- /dev/null +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergMaterializedViews.java @@ -0,0 +1,1514 @@ +/* + * 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 com.facebook.presto.iceberg; + +import com.facebook.airlift.http.server.testing.TestingHttpServer; +import com.facebook.presto.Session; +import com.facebook.presto.testing.QueryRunner; +import com.facebook.presto.tests.AbstractTestQueryFramework; +import com.google.common.collect.ImmutableMap; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.rest.RESTCatalog; +import org.apache.iceberg.types.Types; +import org.assertj.core.util.Files; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import java.io.File; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +import static com.facebook.presto.iceberg.CatalogType.REST; +import static com.facebook.presto.iceberg.rest.IcebergRestTestUtil.getRestServer; +import static com.facebook.presto.iceberg.rest.IcebergRestTestUtil.restConnectorProperties; +import static com.google.common.io.MoreFiles.deleteRecursively; +import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; + +@Test(singleThreaded = true) +public class TestIcebergMaterializedViews + extends AbstractTestQueryFramework +{ + private File warehouseLocation; + private TestingHttpServer restServer; + private String serverUri; + + @BeforeClass + @Override + public void init() + throws Exception + { + warehouseLocation = Files.newTemporaryFolder(); + + restServer = getRestServer(warehouseLocation.getAbsolutePath()); + restServer.start(); + + serverUri = restServer.getBaseUrl().toString(); + super.init(); + } + + @AfterClass(alwaysRun = true) + public void tearDown() + throws Exception + { + if (restServer != null) { + restServer.stop(); + } + deleteRecursively(warehouseLocation.toPath(), ALLOW_INSECURE); + } + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + return IcebergQueryRunner.builder() + .setCatalogType(REST) + .setExtraConnectorProperties(restConnectorProperties(serverUri)) + .setDataDirectory(Optional.of(warehouseLocation.toPath())) + .setSchemaName("test_schema") + .setCreateTpchTables(false) + .setExtraProperties(ImmutableMap.of("experimental.legacy-materialized-views", "false")) + .build().getQueryRunner(); + } + + @Test + public void testCreateMaterializedView() + { + assertUpdate("CREATE TABLE test_mv_base (id BIGINT, name VARCHAR, value BIGINT)"); + assertUpdate("INSERT INTO test_mv_base VALUES (1, 'Alice', 100), (2, 'Bob', 200), (3, 'Charlie', 300)", 3); + + assertUpdate("CREATE MATERIALIZED VIEW test_mv_simple AS SELECT id, name, value FROM test_mv_base"); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_mv_simple\"", "SELECT 0"); + + assertQuery("SELECT COUNT(*) FROM test_mv_simple", "SELECT 3"); + assertQuery("SELECT * FROM test_mv_simple ORDER BY id", + "VALUES (1, 'Alice', 100), (2, 'Bob', 200), (3, 'Charlie', 300)"); + + assertUpdate("DROP MATERIALIZED VIEW test_mv_simple"); + assertUpdate("DROP TABLE test_mv_base"); + } + + @Test + public void testCreateMaterializedViewWithFilter() + { + assertUpdate("CREATE TABLE test_mv_filtered_base (id BIGINT, status VARCHAR, amount BIGINT)"); + assertUpdate("INSERT INTO test_mv_filtered_base VALUES (1, 'active', 100), (2, 'inactive', 200), (3, 'active', 300)", 3); + + assertUpdate("CREATE MATERIALIZED VIEW test_mv_filtered AS SELECT id, amount FROM test_mv_filtered_base WHERE status = 'active'"); + + assertQuery("SELECT COUNT(*) FROM test_mv_filtered", "SELECT 2"); + assertQuery("SELECT * FROM test_mv_filtered ORDER BY id", + "VALUES (1, 100), (3, 300)"); + + assertUpdate("DROP MATERIALIZED VIEW test_mv_filtered"); + assertUpdate("DROP TABLE test_mv_filtered_base"); + } + + @Test + public void testCreateMaterializedViewWithAggregation() + { + assertUpdate("CREATE TABLE test_mv_sales (product_id BIGINT, category VARCHAR, revenue BIGINT)"); + assertUpdate("INSERT INTO test_mv_sales VALUES (1, 'Electronics', 1000), (2, 'Electronics', 1500), (3, 'Books', 500), (4, 'Books', 300)", 4); + + assertUpdate("CREATE MATERIALIZED VIEW test_mv_category_sales AS " + + "SELECT category, COUNT(*) as product_count, SUM(revenue) as total_revenue " + + "FROM test_mv_sales GROUP BY category"); + + assertQuery("SELECT COUNT(*) FROM test_mv_category_sales", "SELECT 2"); + assertQuery("SELECT * FROM test_mv_category_sales ORDER BY category", + "VALUES ('Books', 2, 800), ('Electronics', 2, 2500)"); + + assertUpdate("DROP MATERIALIZED VIEW test_mv_category_sales"); + assertUpdate("DROP TABLE test_mv_sales"); + } + + @Test + public void testMaterializedViewStaleness() + { + assertUpdate("CREATE TABLE test_mv_stale_base (id BIGINT, value BIGINT)"); + assertUpdate("INSERT INTO test_mv_stale_base VALUES (1, 100), (2, 200)", 2); + + assertUpdate("CREATE MATERIALIZED VIEW test_mv_stale AS SELECT id, value FROM test_mv_stale_base"); + + assertQuery("SELECT COUNT(*) FROM test_mv_stale", "SELECT 2"); + assertQuery("SELECT * FROM test_mv_stale ORDER BY id", "VALUES (1, 100), (2, 200)"); + + assertUpdate("INSERT INTO test_mv_stale_base VALUES (3, 300)", 1); + + assertQuery("SELECT COUNT(*) FROM test_mv_stale", "SELECT 3"); + assertQuery("SELECT * FROM test_mv_stale ORDER BY id", + "VALUES (1, 100), (2, 200), (3, 300)"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_mv_stale", 3); + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_mv_stale\"", "SELECT 3"); + + assertUpdate("TRUNCATE TABLE test_mv_stale_base"); + assertQuery("SELECT COUNT(*) FROM test_mv_stale_base", "SELECT 0"); + assertQuery("SELECT COUNT(*) FROM test_mv_stale", "SELECT 0"); + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_mv_stale\"", "SELECT 3"); + + assertUpdate("DROP MATERIALIZED VIEW test_mv_stale"); + assertUpdate("DROP TABLE test_mv_stale_base"); + } + + @Test + public void testDropMaterializedView() + { + assertUpdate("CREATE TABLE test_mv_drop_base (id BIGINT, value VARCHAR)"); + assertUpdate("INSERT INTO test_mv_drop_base VALUES (1, 'test')", 1); + + assertUpdate("CREATE MATERIALIZED VIEW test_mv_drop AS SELECT id, value FROM test_mv_drop_base"); + + assertQuery("SELECT COUNT(*) FROM test_mv_drop", "SELECT 1"); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_mv_drop\"", "SELECT 0"); + + assertUpdate("DROP MATERIALIZED VIEW test_mv_drop"); + + assertQueryFails("SELECT * FROM \"__mv_storage__test_mv_drop\"", ".*does not exist.*"); + + assertQuery("SELECT COUNT(*) FROM test_mv_drop_base", "SELECT 1"); + + assertUpdate("DROP TABLE test_mv_drop_base"); + } + + @Test + public void testMaterializedViewMetadata() + { + assertUpdate("CREATE TABLE test_mv_metadata_base (id BIGINT, name VARCHAR)"); + assertUpdate("INSERT INTO test_mv_metadata_base VALUES (1, 'test')", 1); + + assertUpdate("CREATE MATERIALIZED VIEW test_mv_metadata AS SELECT id, name FROM test_mv_metadata_base WHERE id > 0"); + + assertQueryReturnsEmptyResult("SELECT table_name FROM information_schema.tables " + + "WHERE table_schema = 'test_schema' AND table_name = 'test_mv_metadata' AND table_type = 'MATERIALIZED VIEW'"); + + assertUpdate("DROP MATERIALIZED VIEW test_mv_metadata"); + assertUpdate("DROP TABLE test_mv_metadata_base"); + } + + @Test + public void testRefreshMaterializedView() + { + assertUpdate("CREATE TABLE test_mv_refresh_base (id BIGINT, value BIGINT)"); + assertUpdate("INSERT INTO test_mv_refresh_base VALUES (1, 100), (2, 200)", 2); + + assertUpdate("CREATE MATERIALIZED VIEW test_mv_refresh AS SELECT id, value FROM test_mv_refresh_base"); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_mv_refresh\"", "SELECT 0"); + + assertQuery("SELECT COUNT(*) FROM test_mv_refresh", "SELECT 2"); + assertQuery("SELECT * FROM test_mv_refresh ORDER BY id", "VALUES (1, 100), (2, 200)"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_mv_refresh", 2); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_mv_refresh\"", "SELECT 2"); + assertQuery("SELECT * FROM \"__mv_storage__test_mv_refresh\" ORDER BY id", + "VALUES (1, 100), (2, 200)"); + + assertQuery("SELECT COUNT(*) FROM test_mv_refresh", "SELECT 2"); + assertQuery("SELECT * FROM test_mv_refresh ORDER BY id", "VALUES (1, 100), (2, 200)"); + + assertUpdate("INSERT INTO test_mv_refresh_base VALUES (3, 300)", 1); + + assertQuery("SELECT COUNT(*) FROM test_mv_refresh", "SELECT 3"); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_mv_refresh\"", "SELECT 2"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_mv_refresh", 3); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_mv_refresh\"", "SELECT 3"); + assertQuery("SELECT * FROM \"__mv_storage__test_mv_refresh\" ORDER BY id", + "VALUES (1, 100), (2, 200), (3, 300)"); + + assertUpdate("DROP MATERIALIZED VIEW test_mv_refresh"); + assertUpdate("DROP TABLE test_mv_refresh_base"); + } + + @Test + public void testRefreshMaterializedViewWithAggregation() + { + assertUpdate("CREATE TABLE test_mv_agg_refresh_base (category VARCHAR, value BIGINT)"); + assertUpdate("INSERT INTO test_mv_agg_refresh_base VALUES ('A', 10), ('B', 20), ('A', 15)", 3); + + assertUpdate("CREATE MATERIALIZED VIEW test_mv_agg_refresh AS " + + "SELECT category, SUM(value) as total FROM test_mv_agg_refresh_base GROUP BY category"); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_mv_agg_refresh\"", "SELECT 0"); + + assertQuery("SELECT * FROM test_mv_agg_refresh ORDER BY category", + "VALUES ('A', 25), ('B', 20)"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_mv_agg_refresh", 2); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_mv_agg_refresh\"", "SELECT 2"); + + assertUpdate("INSERT INTO test_mv_agg_refresh_base VALUES ('A', 5), ('C', 30)", 2); + + assertQuery("SELECT * FROM test_mv_agg_refresh ORDER BY category", + "VALUES ('A', 30), ('B', 20), ('C', 30)"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_mv_agg_refresh", 3); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_mv_agg_refresh\"", "SELECT 3"); + assertQuery("SELECT * FROM \"__mv_storage__test_mv_agg_refresh\" ORDER BY category", + "VALUES ('A', 30), ('B', 20), ('C', 30)"); + + assertUpdate("DROP MATERIALIZED VIEW test_mv_agg_refresh"); + assertUpdate("DROP TABLE test_mv_agg_refresh_base"); + } + + @Test + public void testPartitionedMaterializedViewWithStaleDataConstraints() + { + assertUpdate("CREATE TABLE test_mv_partitioned_base (" + + "id BIGINT, " + + "event_date DATE, " + + "value BIGINT) " + + "WITH (partitioning = ARRAY['event_date'])"); + + assertUpdate("INSERT INTO test_mv_partitioned_base VALUES " + + "(1, DATE '2024-01-01', 100), " + + "(2, DATE '2024-01-01', 200), " + + "(3, DATE '2024-01-02', 300)", 3); + + assertUpdate("CREATE MATERIALIZED VIEW test_mv_partitioned AS " + + "SELECT id, event_date, value FROM test_mv_partitioned_base"); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_mv_partitioned\"", "SELECT 0"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_mv_partitioned", 3); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_mv_partitioned\"", "SELECT 3"); + assertQuery("SELECT * FROM \"__mv_storage__test_mv_partitioned\" ORDER BY id", + "VALUES (1, DATE '2024-01-01', 100), (2, DATE '2024-01-01', 200), (3, DATE '2024-01-02', 300)"); + + assertQuery("SELECT COUNT(*) FROM test_mv_partitioned", "SELECT 3"); + assertQuery("SELECT * FROM test_mv_partitioned ORDER BY id", + "VALUES (1, DATE '2024-01-01', 100), (2, DATE '2024-01-01', 200), (3, DATE '2024-01-02', 300)"); + + assertUpdate("INSERT INTO test_mv_partitioned_base VALUES " + + "(4, DATE '2024-01-03', 400), " + + "(5, DATE '2024-01-03', 500)", 2); + + assertQuery("SELECT COUNT(*) FROM test_mv_partitioned", "SELECT 5"); + assertQuery("SELECT * FROM test_mv_partitioned ORDER BY id", + "VALUES (1, DATE '2024-01-01', 100), " + + "(2, DATE '2024-01-01', 200), " + + "(3, DATE '2024-01-02', 300), " + + "(4, DATE '2024-01-03', 400), " + + "(5, DATE '2024-01-03', 500)"); + + assertUpdate("INSERT INTO test_mv_partitioned_base VALUES " + + "(6, DATE '2024-01-04', 600)", 1); + + assertQuery("SELECT COUNT(*) FROM test_mv_partitioned", "SELECT 6"); + + assertUpdate("DROP MATERIALIZED VIEW test_mv_partitioned"); + assertUpdate("DROP TABLE test_mv_partitioned_base"); + } + + @Test + public void testMinimalRefresh() + { + assertUpdate("CREATE TABLE minimal_table (id BIGINT)"); + assertUpdate("INSERT INTO minimal_table VALUES (1)", 1); + assertUpdate("CREATE MATERIALIZED VIEW minimal_mv AS SELECT id FROM minimal_table"); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__minimal_mv\"", "SELECT 0"); + + try { + assertUpdate("REFRESH MATERIALIZED VIEW minimal_mv", 1); + } + catch (Exception e) { + System.err.println("REFRESH failed with: " + e.getMessage()); + } + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__minimal_mv\"", "SELECT 1"); + assertQuery("SELECT * FROM \"__mv_storage__minimal_mv\"", "SELECT 1"); + + assertUpdate("DROP MATERIALIZED VIEW minimal_mv"); + assertUpdate("DROP TABLE minimal_table"); + } + + @Test + public void testJoinMaterializedViewLifecycle() + { + assertUpdate("CREATE TABLE test_mv_orders (order_id BIGINT, customer_id BIGINT, amount BIGINT)"); + assertUpdate("CREATE TABLE test_mv_customers (customer_id BIGINT, customer_name VARCHAR)"); + + assertUpdate("INSERT INTO test_mv_orders VALUES (1, 100, 50), (2, 200, 75), (3, 100, 25)", 3); + assertUpdate("INSERT INTO test_mv_customers VALUES (100, 'Alice'), (200, 'Bob')", 2); + + assertUpdate("CREATE MATERIALIZED VIEW test_mv_order_details AS " + + "SELECT o.order_id, c.customer_name, o.amount " + + "FROM test_mv_orders o JOIN test_mv_customers c ON o.customer_id = c.customer_id"); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_mv_order_details\"", "SELECT 0"); + + assertQuery("SELECT COUNT(*) FROM test_mv_order_details", "SELECT 3"); + assertQuery("SELECT * FROM test_mv_order_details ORDER BY order_id", + "VALUES (1, 'Alice', 50), (2, 'Bob', 75), (3, 'Alice', 25)"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_mv_order_details", 3); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_mv_order_details\"", "SELECT 3"); + assertQuery("SELECT * FROM \"__mv_storage__test_mv_order_details\" ORDER BY order_id", + "VALUES (1, 'Alice', 50), (2, 'Bob', 75), (3, 'Alice', 25)"); + + assertQuery("SELECT COUNT(*) FROM test_mv_order_details", "SELECT 3"); + + assertUpdate("INSERT INTO test_mv_orders VALUES (4, 200, 100)", 1); + + assertQuery("SELECT COUNT(*) FROM test_mv_order_details", "SELECT 4"); + assertQuery("SELECT * FROM test_mv_order_details ORDER BY order_id", + "VALUES (1, 'Alice', 50), (2, 'Bob', 75), (3, 'Alice', 25), (4, 'Bob', 100)"); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_mv_order_details\"", "SELECT 3"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_mv_order_details", 4); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_mv_order_details\"", "SELECT 4"); + assertQuery("SELECT * FROM \"__mv_storage__test_mv_order_details\" ORDER BY order_id", + "VALUES (1, 'Alice', 50), (2, 'Bob', 75), (3, 'Alice', 25), (4, 'Bob', 100)"); + + assertUpdate("DROP MATERIALIZED VIEW test_mv_order_details"); + assertUpdate("DROP TABLE test_mv_customers"); + assertUpdate("DROP TABLE test_mv_orders"); + } + + @Test + public void testPartitionedJoinMaterializedView() + { + assertUpdate("CREATE TABLE test_mv_part_orders (" + + "order_id BIGINT, " + + "customer_id BIGINT, " + + "order_date DATE, " + + "amount BIGINT) " + + "WITH (partitioning = ARRAY['order_date'])"); + + assertUpdate("CREATE TABLE test_mv_part_customers (customer_id BIGINT, customer_name VARCHAR)"); + + assertUpdate("INSERT INTO test_mv_part_orders VALUES " + + "(1, 100, DATE '2024-01-01', 50), " + + "(2, 200, DATE '2024-01-01', 75), " + + "(3, 100, DATE '2024-01-02', 25)", 3); + assertUpdate("INSERT INTO test_mv_part_customers VALUES (100, 'Alice'), (200, 'Bob')", 2); + + assertUpdate("CREATE MATERIALIZED VIEW test_mv_part_join AS " + + "SELECT o.order_id, c.customer_name, o.order_date, o.amount " + + "FROM test_mv_part_orders o JOIN test_mv_part_customers c ON o.customer_id = c.customer_id"); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_mv_part_join\"", "SELECT 0"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_mv_part_join", 3); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_mv_part_join\"", "SELECT 3"); + assertQuery("SELECT * FROM \"__mv_storage__test_mv_part_join\" ORDER BY order_id", + "VALUES (1, 'Alice', DATE '2024-01-01', 50), " + + "(2, 'Bob', DATE '2024-01-01', 75), " + + "(3, 'Alice', DATE '2024-01-02', 25)"); + + assertQuery("SELECT COUNT(*) FROM test_mv_part_join", "SELECT 3"); + assertQuery("SELECT * FROM test_mv_part_join ORDER BY order_id", + "VALUES (1, 'Alice', DATE '2024-01-01', 50), " + + "(2, 'Bob', DATE '2024-01-01', 75), " + + "(3, 'Alice', DATE '2024-01-02', 25)"); + + assertUpdate("INSERT INTO test_mv_part_orders VALUES (4, 200, DATE '2024-01-03', 100)", 1); + + assertQuery("SELECT COUNT(*) FROM test_mv_part_join", "SELECT 4"); + assertQuery("SELECT * FROM test_mv_part_join ORDER BY order_id", + "VALUES (1, 'Alice', DATE '2024-01-01', 50), " + + "(2, 'Bob', DATE '2024-01-01', 75), " + + "(3, 'Alice', DATE '2024-01-02', 25), " + + "(4, 'Bob', DATE '2024-01-03', 100)"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_mv_part_join", 4); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_mv_part_join\"", "SELECT 4"); + + assertUpdate("DROP MATERIALIZED VIEW test_mv_part_join"); + assertUpdate("DROP TABLE test_mv_part_customers"); + assertUpdate("DROP TABLE test_mv_part_orders"); + } + + @Test + public void testMultiTableStaleness_TwoTablesBothStale() + { + assertUpdate("CREATE TABLE test_mv_orders (" + + "order_id BIGINT, " + + "order_date DATE, " + + "amount BIGINT) " + + "WITH (partitioning = ARRAY['order_date'])"); + + assertUpdate("CREATE TABLE test_mv_customers (" + + "customer_id BIGINT, " + + "reg_date DATE, " + + "name VARCHAR) " + + "WITH (partitioning = ARRAY['reg_date'])"); + + assertUpdate("INSERT INTO test_mv_orders VALUES " + + "(1, DATE '2024-01-01', 100), " + + "(2, DATE '2024-01-02', 200)", 2); + assertUpdate("INSERT INTO test_mv_customers VALUES " + + "(1, DATE '2024-01-01', 'Alice'), " + + "(2, DATE '2024-01-02', 'Bob')", 2); + + assertUpdate("CREATE MATERIALIZED VIEW test_mv_multi_stale AS " + + "SELECT o.order_id, c.name, o.order_date, c.reg_date, o.amount " + + "FROM test_mv_orders o JOIN test_mv_customers c ON o.order_id = c.customer_id"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_mv_multi_stale", 2); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_mv_multi_stale\"", "SELECT 2"); + + assertQuery("SELECT COUNT(*) FROM test_mv_multi_stale", "SELECT 2"); + + assertUpdate("INSERT INTO test_mv_orders VALUES (3, DATE '2024-01-03', 300)", 1); + assertUpdate("INSERT INTO test_mv_customers VALUES (3, DATE '2024-01-03', 'Charlie')", 1); + + assertQuery("SELECT COUNT(*) FROM test_mv_multi_stale", "SELECT 3"); + assertQuery("SELECT order_id, name, order_date, reg_date, amount FROM test_mv_multi_stale ORDER BY order_id", + "VALUES (1, 'Alice', DATE '2024-01-01', DATE '2024-01-01', 100), " + + "(2, 'Bob', DATE '2024-01-02', DATE '2024-01-02', 200), " + + "(3, 'Charlie', DATE '2024-01-03', DATE '2024-01-03', 300)"); + + assertUpdate("DROP MATERIALIZED VIEW test_mv_multi_stale"); + assertUpdate("DROP TABLE test_mv_customers"); + assertUpdate("DROP TABLE test_mv_orders"); + } + + @Test + public void testMultiTableStaleness_ThreeTablesWithTwoStale() + { + assertUpdate("CREATE TABLE test_mv_t1 (" + + "id BIGINT, " + + "date1 DATE, " + + "value1 BIGINT) " + + "WITH (partitioning = ARRAY['date1'])"); + + assertUpdate("CREATE TABLE test_mv_t2 (" + + "id BIGINT, " + + "date2 DATE, " + + "value2 BIGINT) " + + "WITH (partitioning = ARRAY['date2'])"); + + assertUpdate("CREATE TABLE test_mv_t3 (" + + "id BIGINT, " + + "date3 DATE, " + + "value3 BIGINT) " + + "WITH (partitioning = ARRAY['date3'])"); + + assertUpdate("INSERT INTO test_mv_t1 VALUES (1, DATE '2024-01-01', 100)", 1); + assertUpdate("INSERT INTO test_mv_t2 VALUES (1, DATE '2024-01-01', 200)", 1); + assertUpdate("INSERT INTO test_mv_t3 VALUES (1, DATE '2024-01-01', 300)", 1); + + assertUpdate("CREATE MATERIALIZED VIEW test_mv_three_tables AS " + + "SELECT t1.id, t1.date1, t2.date2, t3.date3, " + + " t1.value1, t2.value2, t3.value3 " + + "FROM test_mv_t1 t1 " + + "JOIN test_mv_t2 t2 ON t1.id = t2.id " + + "JOIN test_mv_t3 t3 ON t1.id = t3.id"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_mv_three_tables", 1); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_mv_three_tables\"", "SELECT 1"); + + assertUpdate("INSERT INTO test_mv_t1 VALUES (2, DATE '2024-01-02', 150)", 1); + assertUpdate("INSERT INTO test_mv_t2 VALUES (2, DATE '2024-01-01', 250)", 1); + assertUpdate("INSERT INTO test_mv_t3 VALUES (2, DATE '2024-01-02', 350)", 1); + + assertQuery("SELECT COUNT(*) FROM test_mv_three_tables", "SELECT 2"); + + assertUpdate("DROP MATERIALIZED VIEW test_mv_three_tables"); + assertUpdate("DROP TABLE test_mv_t3"); + assertUpdate("DROP TABLE test_mv_t2"); + assertUpdate("DROP TABLE test_mv_t1"); + } + + @Test + public void testMultiTableStaleness_DifferentPartitionCounts() + { + assertUpdate("CREATE TABLE test_mv_table_a (" + + "id BIGINT, " + + "date_a DATE, " + + "value BIGINT) " + + "WITH (partitioning = ARRAY['date_a'])"); + + assertUpdate("CREATE TABLE test_mv_table_b (" + + "id BIGINT, " + + "date_b DATE, " + + "status VARCHAR) " + + "WITH (partitioning = ARRAY['date_b'])"); + + assertUpdate("INSERT INTO test_mv_table_a VALUES " + + "(1, DATE '2024-01-01', 100), " + + "(2, DATE '2024-01-02', 200)", 2); + assertUpdate("INSERT INTO test_mv_table_b VALUES " + + "(1, DATE '2024-01-01', 'active'), " + + "(2, DATE '2024-01-02', 'inactive')", 2); + + assertUpdate("CREATE MATERIALIZED VIEW test_mv_diff_partitions AS " + + "SELECT a.id, a.date_a, b.date_b, a.value, b.status " + + "FROM test_mv_table_a a JOIN test_mv_table_b b ON a.id = b.id"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_mv_diff_partitions", 2); + + assertUpdate("INSERT INTO test_mv_table_a VALUES " + + "(3, DATE '2024-01-03', 300), " + + "(4, DATE '2024-01-04', 400), " + + "(5, DATE '2024-01-05', 500)", 3); + + assertUpdate("INSERT INTO test_mv_table_b VALUES " + + "(3, DATE '2024-01-03', 'active'), " + + "(4, DATE '2024-01-04', 'active'), " + + "(5, DATE '2024-01-05', 'pending')", 3); + + assertQuery("SELECT COUNT(*) FROM test_mv_diff_partitions", "SELECT 5"); + + assertUpdate("DROP MATERIALIZED VIEW test_mv_diff_partitions"); + assertUpdate("DROP TABLE test_mv_table_b"); + assertUpdate("DROP TABLE test_mv_table_a"); + } + + @Test + public void testMultiTableStaleness_NonPartitionedAndPartitionedBothStale() + { + assertUpdate("CREATE TABLE test_mv_non_part (id BIGINT, category VARCHAR)"); + + assertUpdate("CREATE TABLE test_mv_part_sales (" + + "id BIGINT, " + + "sale_date DATE, " + + "amount BIGINT) " + + "WITH (partitioning = ARRAY['sale_date'])"); + + assertUpdate("INSERT INTO test_mv_non_part VALUES (1, 'Electronics'), (2, 'Books')", 2); + assertUpdate("INSERT INTO test_mv_part_sales VALUES " + + "(1, DATE '2024-01-01', 500), " + + "(2, DATE '2024-01-02', 300)", 2); + + assertUpdate("CREATE MATERIALIZED VIEW test_mv_mixed_stale AS " + + "SELECT c.id, c.category, s.sale_date, s.amount " + + "FROM test_mv_non_part c JOIN test_mv_part_sales s ON c.id = s.id"); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_mv_mixed_stale\"", "SELECT 0"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_mv_mixed_stale", 2); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_mv_mixed_stale\"", "SELECT 2"); + assertQuery("SELECT id, category, sale_date, amount FROM \"__mv_storage__test_mv_mixed_stale\" ORDER BY id", + "VALUES (1, 'Electronics', DATE '2024-01-01', 500), (2, 'Books', DATE '2024-01-02', 300)"); + + assertUpdate("INSERT INTO test_mv_non_part VALUES (3, 'Toys')", 1); + assertUpdate("INSERT INTO test_mv_part_sales VALUES (3, DATE '2024-01-03', 700)", 1); + + assertQuery("SELECT COUNT(*) FROM test_mv_mixed_stale", "SELECT 3"); + assertQuery("SELECT id, category, sale_date, amount FROM test_mv_mixed_stale ORDER BY id", + "VALUES (1, 'Electronics', DATE '2024-01-01', 500), " + + "(2, 'Books', DATE '2024-01-02', 300), " + + "(3, 'Toys', DATE '2024-01-03', 700)"); + + assertUpdate("DROP MATERIALIZED VIEW test_mv_mixed_stale"); + assertUpdate("DROP TABLE test_mv_part_sales"); + assertUpdate("DROP TABLE test_mv_non_part"); + } + + @Test + public void testPartitionAlignment_MatchingColumns() + { + assertUpdate("CREATE TABLE test_pa_matching_base (" + + "id BIGINT, " + + "event_date DATE, " + + "amount BIGINT) " + + "WITH (partitioning = ARRAY['event_date'])"); + + assertUpdate("INSERT INTO test_pa_matching_base VALUES " + + "(1, DATE '2024-01-01', 100), " + + "(2, DATE '2024-01-02', 200), " + + "(3, DATE '2024-01-03', 300)", 3); + + assertUpdate("CREATE MATERIALIZED VIEW test_pa_matching_mv AS " + + "SELECT id, event_date, amount FROM test_pa_matching_base"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_pa_matching_mv", 3); + + assertUpdate("INSERT INTO test_pa_matching_base VALUES (4, DATE '2024-01-04', 400)", 1); + + assertQuery("SELECT COUNT(*) FROM test_pa_matching_mv", "SELECT 4"); + assertQuery("SELECT id, event_date, amount FROM test_pa_matching_mv ORDER BY id", + "VALUES " + + "(1, DATE '2024-01-01', 100), " + + "(2, DATE '2024-01-02', 200), " + + "(3, DATE '2024-01-03', 300), " + + "(4, DATE '2024-01-04', 400)"); + + assertUpdate("DROP MATERIALIZED VIEW test_pa_matching_mv"); + assertUpdate("DROP TABLE test_pa_matching_base"); + } + + @Test + public void testPartitionAlignment_MissingConstraintColumn() + { + assertUpdate("CREATE TABLE test_pa_missing_base (" + + "id BIGINT, " + + "event_date DATE, " + + "amount BIGINT) " + + "WITH (partitioning = ARRAY['event_date'])"); + + assertUpdate("INSERT INTO test_pa_missing_base VALUES " + + "(1, DATE '2024-01-01', 100), " + + "(2, DATE '2024-01-02', 200), " + + "(3, DATE '2024-01-03', 300)", 3); + + assertUpdate("CREATE MATERIALIZED VIEW test_pa_missing_mv AS " + + "SELECT id, amount FROM test_pa_missing_base"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_pa_missing_mv", 3); + + assertUpdate("INSERT INTO test_pa_missing_base VALUES (4, DATE '2024-01-04', 400)", 1); + + assertQuery("SELECT COUNT(*) FROM test_pa_missing_mv", "SELECT 4"); + assertQuery("SELECT id, amount FROM test_pa_missing_mv ORDER BY id", + "VALUES (1, 100), (2, 200), (3, 300), (4, 400)"); + + assertUpdate("DROP MATERIALIZED VIEW test_pa_missing_mv"); + assertUpdate("DROP TABLE test_pa_missing_base"); + } + + @Test + public void testPartitionAlignment_OverSpecifiedStorage() + { + assertUpdate("CREATE TABLE test_pa_over_table_a (" + + "id BIGINT, " + + "event_date DATE, " + + "amount BIGINT) " + + "WITH (partitioning = ARRAY['event_date'])"); + + assertUpdate("CREATE TABLE test_pa_over_table_b (" + + "customer_id BIGINT, " + + "region VARCHAR, " + + "name VARCHAR) " + + "WITH (partitioning = ARRAY['region'])"); + + assertUpdate("INSERT INTO test_pa_over_table_a VALUES " + + "(1, DATE '2024-01-01', 100), " + + "(2, DATE '2024-01-02', 200), " + + "(3, DATE '2024-01-03', 300)", 3); + + assertUpdate("INSERT INTO test_pa_over_table_b VALUES " + + "(1, 'US', 'Alice'), " + + "(2, 'US', 'Bob'), " + + "(3, 'UK', 'Charlie')", 3); + + assertUpdate("CREATE MATERIALIZED VIEW test_pa_over_mv AS " + + "SELECT a.id, a.event_date, a.amount, b.region, b.name " + + "FROM test_pa_over_table_a a " + + "JOIN test_pa_over_table_b b ON a.id = b.customer_id"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_pa_over_mv", 3); + + assertUpdate("INSERT INTO test_pa_over_table_a VALUES (1, DATE '2024-01-04', 150)", 1); + + assertQuery("SELECT COUNT(*) FROM test_pa_over_mv", "SELECT 4"); + assertQuery("SELECT id, event_date, amount, region, name FROM test_pa_over_mv ORDER BY id, event_date", + "VALUES " + + "(1, DATE '2024-01-01', 100, 'US', 'Alice'), " + + "(1, DATE '2024-01-04', 150, 'US', 'Alice'), " + + "(2, DATE '2024-01-02', 200, 'US', 'Bob'), " + + "(3, DATE '2024-01-03', 300, 'UK', 'Charlie')"); + + assertUpdate("DROP MATERIALIZED VIEW test_pa_over_mv"); + assertUpdate("DROP TABLE test_pa_over_table_b"); + assertUpdate("DROP TABLE test_pa_over_table_a"); + } + + @Test + public void testAggregationMV_MisalignedPartitioning() + { + // Bug: When GROUP BY column differs from partition column and multiple partitions + // are stale, the current implementation creates partial aggregates per partition + // and GROUP BY treats them as distinct rows instead of re-aggregating. + assertUpdate("CREATE TABLE test_agg_misaligned (" + + "id BIGINT, " + + "partition_col VARCHAR, " + + "region VARCHAR, " + + "sales BIGINT) " + + "WITH (partitioning = ARRAY['partition_col'])"); + + assertUpdate("INSERT INTO test_agg_misaligned VALUES " + + "(1, 'A', 'US', 100), " + + "(2, 'A', 'EU', 50), " + + "(3, 'B', 'US', 200), " + + "(4, 'B', 'EU', 75)", 4); + + assertUpdate("CREATE MATERIALIZED VIEW test_agg_mv AS " + + "SELECT region, SUM(sales) as total_sales " + + "FROM test_agg_misaligned " + + "GROUP BY region"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_agg_mv", 2); + + assertQuery("SELECT * FROM test_agg_mv ORDER BY region", + "VALUES ('EU', 125), ('US', 300)"); + + assertUpdate("INSERT INTO test_agg_misaligned VALUES " + + "(5, 'A', 'US', 10), " + + "(6, 'B', 'US', 20)", 2); + + assertQuery("SELECT * FROM test_agg_mv ORDER BY region", + "VALUES ('EU', 125), ('US', 330)"); + + assertUpdate("DROP MATERIALIZED VIEW test_agg_mv"); + assertUpdate("DROP TABLE test_agg_misaligned"); + } + + @Test + public void testAggregationMV_MultiTableJoin_BothStale() + { + // Bug: When both tables are stale, creates partial aggregates for each branch + // which are treated as distinct rows instead of being re-aggregated. + assertUpdate("CREATE TABLE test_multi_orders (" + + "order_id BIGINT, " + + "product_id BIGINT, " + + "order_date DATE, " + + "quantity BIGINT) " + + "WITH (partitioning = ARRAY['order_date'])"); + + assertUpdate("CREATE TABLE test_multi_products (" + + "product_id BIGINT, " + + "product_category VARCHAR, " + + "price BIGINT) " + + "WITH (partitioning = ARRAY['product_category'])"); + + assertUpdate("INSERT INTO test_multi_orders VALUES " + + "(1, 100, DATE '2024-01-01', 5), " + + "(2, 200, DATE '2024-01-01', 3)", 2); + assertUpdate("INSERT INTO test_multi_products VALUES " + + "(100, 'Electronics', 50), " + + "(200, 'Books', 20)", 2); + + assertUpdate("CREATE MATERIALIZED VIEW test_multi_agg_mv AS " + + "SELECT p.product_category, SUM(o.quantity * p.price) as total_revenue " + + "FROM test_multi_orders o " + + "JOIN test_multi_products p ON o.product_id = p.product_id " + + "GROUP BY p.product_category"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_multi_agg_mv", 2); + + assertQuery("SELECT * FROM test_multi_agg_mv ORDER BY product_category", + "VALUES ('Books', 60), ('Electronics', 250)"); + + assertUpdate("INSERT INTO test_multi_orders VALUES " + + "(3, 100, DATE '2024-01-02', 2), " + + "(4, 200, DATE '2024-01-02', 4)", 2); + + assertUpdate("INSERT INTO test_multi_products VALUES " + + "(300, 'Toys', 30)", 1); + + assertUpdate("INSERT INTO test_multi_orders VALUES " + + "(5, 300, DATE '2024-01-02', 1)", 1); + + String explainResult = (String) computeScalar("EXPLAIN SELECT * FROM test_multi_agg_mv ORDER BY product_category"); + System.out.println("=== EXPLAIN PLAN ==="); + System.out.println(explainResult); + System.out.println("==================="); + + assertQuery("SELECT * FROM test_multi_agg_mv ORDER BY product_category", + "VALUES ('Books', 140), ('Electronics', 350), ('Toys', 30)"); + + assertUpdate("DROP MATERIALIZED VIEW test_multi_agg_mv"); + assertUpdate("DROP TABLE test_multi_products"); + assertUpdate("DROP TABLE test_multi_orders"); + } + + @Test + public void testMaterializedViewWithCustomStorageTableName() + { + assertUpdate("CREATE TABLE test_custom_storage_base (id BIGINT, name VARCHAR, value BIGINT)"); + assertUpdate("INSERT INTO test_custom_storage_base VALUES (1, 'Alice', 100), (2, 'Bob', 200)", 2); + + assertUpdate("CREATE MATERIALIZED VIEW test_custom_storage_mv " + + "WITH (materialized_view_storage_table_name = 'my_custom_storage_table') " + + "AS SELECT id, name, value FROM test_custom_storage_base"); + + assertQuery("SELECT COUNT(*) FROM my_custom_storage_table", "SELECT 0"); + + assertQueryFails("SELECT * FROM \"__mv_storage__test_custom_storage_mv\"", ".*does not exist.*"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_custom_storage_mv", 2); + + assertQuery("SELECT COUNT(*) FROM my_custom_storage_table", "SELECT 2"); + assertQuery("SELECT * FROM my_custom_storage_table ORDER BY id", + "VALUES (1, 'Alice', 100), (2, 'Bob', 200)"); + + assertQuery("SELECT * FROM test_custom_storage_mv ORDER BY id", + "VALUES (1, 'Alice', 100), (2, 'Bob', 200)"); + + assertUpdate("INSERT INTO test_custom_storage_base VALUES (3, 'Charlie', 300)", 1); + assertUpdate("REFRESH MATERIALIZED VIEW test_custom_storage_mv", 3); + + assertQuery("SELECT COUNT(*) FROM my_custom_storage_table", "SELECT 3"); + assertQuery("SELECT * FROM my_custom_storage_table ORDER BY id", + "VALUES (1, 'Alice', 100), (2, 'Bob', 200), (3, 'Charlie', 300)"); + + assertUpdate("DROP MATERIALIZED VIEW test_custom_storage_mv"); + + assertQueryFails("SELECT * FROM my_custom_storage_table", ".*does not exist.*"); + + assertUpdate("DROP TABLE test_custom_storage_base"); + } + + @Test + public void testMaterializedViewWithCustomStorageSchema() + { + assertUpdate("CREATE SCHEMA IF NOT EXISTS test_storage_schema"); + + assertUpdate("CREATE TABLE test_custom_schema_base (id BIGINT, value BIGINT)"); + assertUpdate("INSERT INTO test_custom_schema_base VALUES (1, 100), (2, 200)", 2); + + assertUpdate("CREATE MATERIALIZED VIEW test_custom_schema_mv " + + "WITH (materialized_view_storage_schema = 'test_storage_schema', " + + "materialized_view_storage_table_name = 'storage_table') " + + "AS SELECT id, value FROM test_schema.test_custom_schema_base"); + + assertQuery("SELECT COUNT(*) FROM test_storage_schema.storage_table", "SELECT 0"); + + assertQueryFails("SELECT * FROM test_schema.storage_table", ".*does not exist.*"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_schema.test_custom_schema_mv", 2); + + assertQuery("SELECT COUNT(*) FROM test_storage_schema.storage_table", "SELECT 2"); + assertQuery("SELECT * FROM test_storage_schema.storage_table ORDER BY id", + "VALUES (1, 100), (2, 200)"); + + assertQuery("SELECT * FROM test_custom_schema_mv ORDER BY id", + "VALUES (1, 100), (2, 200)"); + + assertUpdate("DROP MATERIALIZED VIEW test_schema.test_custom_schema_mv"); + assertQueryFails("SELECT * FROM test_storage_schema.storage_table", ".*does not exist.*"); + + assertUpdate("DROP TABLE test_custom_schema_base"); + assertUpdate("DROP SCHEMA test_storage_schema"); + } + + @Test + public void testMaterializedViewWithCustomPrefix() + { + assertUpdate("CREATE TABLE test_custom_prefix_base (id BIGINT, name VARCHAR)"); + assertUpdate("INSERT INTO test_custom_prefix_base VALUES (1, 'test')", 1); + + Session sessionWithCustomPrefix = Session.builder(getSession()) + .setCatalogSessionProperty("iceberg", "materialized_view_storage_prefix", "custom_prefix_") + .build(); + + assertUpdate(sessionWithCustomPrefix, "CREATE MATERIALIZED VIEW test_custom_prefix_mv " + + "AS SELECT id, name FROM test_custom_prefix_base"); + + assertQuery("SELECT COUNT(*) FROM custom_prefix_test_custom_prefix_mv", "SELECT 0"); + + assertQueryFails("SELECT * FROM \"__mv_storage__test_custom_prefix_mv\"", ".*does not exist.*"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_custom_prefix_mv", 1); + + assertQuery("SELECT COUNT(*) FROM custom_prefix_test_custom_prefix_mv", "SELECT 1"); + assertQuery("SELECT * FROM custom_prefix_test_custom_prefix_mv", "VALUES (1, 'test')"); + + assertQuery("SELECT * FROM test_custom_prefix_mv", "VALUES (1, 'test')"); + + assertUpdate("DROP MATERIALIZED VIEW test_custom_prefix_mv"); + assertQueryFails("SELECT * FROM custom_prefix_test_custom_prefix_mv", ".*does not exist.*"); + + assertUpdate("DROP TABLE test_custom_prefix_base"); + } + + @Test + public void testMaterializedViewWithValuesOnly() + { + assertUpdate("CREATE MATERIALIZED VIEW test_values_mv AS SELECT * FROM (VALUES (1, 'Alice', 100), (2, 'Bob', 200), (3, 'Charlie', 300)) AS t(id, name, value)"); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_values_mv\"", "SELECT 0"); + + assertQuery("SELECT COUNT(*) FROM test_values_mv", "SELECT 3"); + assertQuery("SELECT * FROM test_values_mv ORDER BY id", + "VALUES (1, 'Alice', 100), (2, 'Bob', 200), (3, 'Charlie', 300)"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_values_mv", 3); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_values_mv\"", "SELECT 3"); + assertQuery("SELECT * FROM \"__mv_storage__test_values_mv\" ORDER BY id", + "VALUES (1, 'Alice', 100), (2, 'Bob', 200), (3, 'Charlie', 300)"); + + assertQuery("SELECT * FROM test_values_mv ORDER BY id", + "VALUES (1, 'Alice', 100), (2, 'Bob', 200), (3, 'Charlie', 300)"); + + assertUpdate("DROP MATERIALIZED VIEW test_values_mv"); + assertQueryFails("SELECT * FROM \"__mv_storage__test_values_mv\"", ".*does not exist.*"); + } + + @Test + public void testMaterializedViewWithBaseTableButNoColumnsSelected() + { + assertUpdate("CREATE TABLE test_no_cols_base (id BIGINT, name VARCHAR, value BIGINT)"); + assertUpdate("INSERT INTO test_no_cols_base VALUES (1, 'Alice', 100), (2, 'Bob', 200), (3, 'Charlie', 300)", 3); + + assertUpdate("CREATE MATERIALIZED VIEW test_no_cols_mv AS " + + "SELECT 'constant' as label, 42 as fixed_value FROM test_no_cols_base"); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_no_cols_mv\"", "SELECT 0"); + + assertQuery("SELECT COUNT(*) FROM test_no_cols_mv", "SELECT 3"); + assertQuery("SELECT * FROM test_no_cols_mv", + "VALUES ('constant', 42), ('constant', 42), ('constant', 42)"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_no_cols_mv", 3); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_no_cols_mv\"", "SELECT 3"); + assertQuery("SELECT * FROM \"__mv_storage__test_no_cols_mv\"", + "VALUES ('constant', 42), ('constant', 42), ('constant', 42)"); + + assertUpdate("INSERT INTO test_no_cols_base VALUES (4, 'Dave', 400)", 1); + + assertQuery("SELECT COUNT(*) FROM test_no_cols_mv", "SELECT 4"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_no_cols_mv", 4); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_no_cols_mv\"", "SELECT 4"); + + assertUpdate("DROP MATERIALIZED VIEW test_no_cols_mv"); + assertQueryFails("SELECT * FROM \"__mv_storage__test_no_cols_mv\"", ".*does not exist.*"); + + assertUpdate("DROP TABLE test_no_cols_base"); + } + + @Test + public void testMaterializedViewOnEmptyBaseTable() + { + assertUpdate("CREATE TABLE test_empty_base (id BIGINT, name VARCHAR, value BIGINT)"); + + assertUpdate("CREATE MATERIALIZED VIEW test_empty_mv AS SELECT id, name, value FROM test_empty_base"); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_empty_mv\"", "SELECT 0"); + + assertQuery("SELECT COUNT(*) FROM test_empty_mv", "SELECT 0"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_empty_mv", 0); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_empty_mv\"", "SELECT 0"); + + assertUpdate("INSERT INTO test_empty_base VALUES (1, 'Alice', 100), (2, 'Bob', 200)", 2); + + assertQuery("SELECT COUNT(*) FROM test_empty_mv", "SELECT 2"); + assertQuery("SELECT * FROM test_empty_mv ORDER BY id", + "VALUES (1, 'Alice', 100), (2, 'Bob', 200)"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_empty_mv", 2); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_empty_mv\"", "SELECT 2"); + assertQuery("SELECT * FROM \"__mv_storage__test_empty_mv\" ORDER BY id", + "VALUES (1, 'Alice', 100), (2, 'Bob', 200)"); + + assertUpdate("DROP MATERIALIZED VIEW test_empty_mv"); + assertQueryFails("SELECT * FROM \"__mv_storage__test_empty_mv\"", ".*does not exist.*"); + + assertUpdate("DROP TABLE test_empty_base"); + } + + @Test + public void testMaterializedViewMissingRequiredProperties() + throws Exception + { + assertUpdate("CREATE TABLE test_validation_base (id BIGINT, name VARCHAR)"); + assertUpdate("INSERT INTO test_validation_base VALUES (1, 'Alice')", 1); + + RESTCatalog catalog = new RESTCatalog(); + Map catalogProps = new HashMap<>(); + catalogProps.put("uri", serverUri); + catalogProps.put("warehouse", warehouseLocation.getAbsolutePath()); + catalog.initialize("test_catalog", catalogProps); + + try { + TableIdentifier viewId1 = TableIdentifier.of(Namespace.of("test_schema"), "test_mv_missing_base_tables"); + Map properties1 = new HashMap<>(); + properties1.put("presto.materialized_view.original_sql", "SELECT id, name FROM test_validation_base"); + properties1.put("presto.materialized_view.storage_schema", "test_schema"); + properties1.put("presto.materialized_view.storage_table_name", "storage1"); + properties1.put("presto.materialized_view.column_mappings", "[]"); + + catalog.buildView(viewId1) + .withSchema(new org.apache.iceberg.Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required(2, "name", Types.StringType.get()))) + .withQuery("spark", "SELECT id, name FROM test_validation_base") + .withDefaultNamespace(Namespace.of("test_schema")) + .withProperties(properties1) + .create(); + + assertQueryFails("SELECT * FROM test_mv_missing_base_tables", + ".*Materialized view missing required property: presto.materialized_view.base_tables.*"); + + catalog.dropView(viewId1); + + TableIdentifier viewId2 = TableIdentifier.of(Namespace.of("test_schema"), "test_mv_missing_column_mappings"); + Map properties2 = new HashMap<>(); + properties2.put("presto.materialized_view.original_sql", "SELECT id, name FROM test_validation_base"); + properties2.put("presto.materialized_view.storage_schema", "test_schema"); + properties2.put("presto.materialized_view.storage_table_name", "storage2"); + properties2.put("presto.materialized_view.base_tables", "test_schema.test_validation_base"); + + catalog.buildView(viewId2) + .withSchema(new org.apache.iceberg.Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required(2, "name", Types.StringType.get()))) + .withQuery("spark", "SELECT id, name FROM test_validation_base") + .withDefaultNamespace(Namespace.of("test_schema")) + .withProperties(properties2) + .create(); + + assertQueryFails("SELECT * FROM test_mv_missing_column_mappings", + ".*Materialized view missing required property: presto.materialized_view.column_mappings.*"); + + catalog.dropView(viewId2); + + TableIdentifier viewId3 = TableIdentifier.of(Namespace.of("test_schema"), "test_mv_missing_storage_schema"); + Map properties3 = new HashMap<>(); + properties3.put("presto.materialized_view.original_sql", "SELECT id, name FROM test_validation_base"); + properties3.put("presto.materialized_view.storage_table_name", "storage3"); + properties3.put("presto.materialized_view.base_tables", "test_schema.test_validation_base"); + properties3.put("presto.materialized_view.column_mappings", "[]"); + + catalog.buildView(viewId3) + .withSchema(new org.apache.iceberg.Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required(2, "name", Types.StringType.get()))) + .withQuery("spark", "SELECT id, name FROM test_validation_base") + .withDefaultNamespace(Namespace.of("test_schema")) + .withProperties(properties3) + .create(); + + assertQueryFails("SELECT * FROM test_mv_missing_storage_schema", + ".*Materialized view missing required property: presto.materialized_view.storage_schema.*"); + + catalog.dropView(viewId3); + + TableIdentifier viewId4 = TableIdentifier.of(Namespace.of("test_schema"), "test_mv_missing_storage_table_name"); + Map properties4 = new HashMap<>(); + properties4.put("presto.materialized_view.original_sql", "SELECT id, name FROM test_validation_base"); + properties4.put("presto.materialized_view.storage_schema", "test_schema"); + properties4.put("presto.materialized_view.base_tables", "test_schema.test_validation_base"); + properties4.put("presto.materialized_view.column_mappings", "[]"); + + catalog.buildView(viewId4) + .withSchema(new org.apache.iceberg.Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required(2, "name", Types.StringType.get()))) + .withQuery("spark", "SELECT id, name FROM test_validation_base") + .withDefaultNamespace(Namespace.of("test_schema")) + .withProperties(properties4) + .create(); + + assertQueryFails("SELECT * FROM test_mv_missing_storage_table_name", + ".*Materialized view missing required property: presto.materialized_view.storage_table_name.*"); + + catalog.dropView(viewId4); + } + finally { + catalog.close(); + } + + assertUpdate("DROP TABLE test_validation_base"); + } + + @Test + public void testRefreshFailurePreservesOldData() + { + assertUpdate("CREATE TABLE test_refresh_failure_base (id BIGINT, value BIGINT)"); + assertUpdate("INSERT INTO test_refresh_failure_base VALUES (1, 100), (2, 200)", 2); + + assertUpdate("CREATE MATERIALIZED VIEW test_refresh_failure_mv AS " + + "SELECT id, value FROM test_refresh_failure_base"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_refresh_failure_mv", 2); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_refresh_failure_mv\"", "SELECT 2"); + assertQuery("SELECT * FROM \"__mv_storage__test_refresh_failure_mv\" ORDER BY id", + "VALUES (1, 100), (2, 200)"); + + assertUpdate("DROP TABLE test_refresh_failure_base"); + + try { + getQueryRunner().execute("REFRESH MATERIALIZED VIEW test_refresh_failure_mv"); + throw new AssertionError("Expected REFRESH to fail when base table doesn't exist"); + } + catch (Exception e) { + if (!e.getMessage().contains("does not exist") && !e.getMessage().contains("not found")) { + throw new AssertionError("Expected 'does not exist' error, got: " + e.getMessage()); + } + } + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_refresh_failure_mv\"", "SELECT 2"); + assertQuery("SELECT * FROM \"__mv_storage__test_refresh_failure_mv\" ORDER BY id", + "VALUES (1, 100), (2, 200)"); + + assertUpdate("DROP MATERIALIZED VIEW test_refresh_failure_mv"); + } + + @Test + public void testBaseTableDroppedAndRecreated() + { + assertUpdate("CREATE TABLE test_recreate_base (id BIGINT, value BIGINT)"); + assertUpdate("INSERT INTO test_recreate_base VALUES (1, 100), (2, 200)", 2); + + assertUpdate("CREATE MATERIALIZED VIEW test_recreate_mv AS SELECT id, value FROM test_recreate_base"); + assertUpdate("REFRESH MATERIALIZED VIEW test_recreate_mv", 2); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_recreate_mv\"", "SELECT 2"); + assertQuery("SELECT * FROM \"__mv_storage__test_recreate_mv\" ORDER BY id", + "VALUES (1, 100), (2, 200)"); + + assertUpdate("DROP TABLE test_recreate_base"); + + assertUpdate("CREATE TABLE test_recreate_base (id BIGINT, value BIGINT)"); + assertUpdate("INSERT INTO test_recreate_base VALUES (3, 300), (4, 400), (5, 500)", 3); + + assertQuery("SELECT COUNT(*) FROM test_recreate_mv", "SELECT 3"); + assertQuery("SELECT * FROM test_recreate_mv ORDER BY id", + "VALUES (3, 300), (4, 400), (5, 500)"); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_recreate_mv\"", "SELECT 2"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_recreate_mv", 3); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_recreate_mv\"", "SELECT 3"); + assertQuery("SELECT * FROM \"__mv_storage__test_recreate_mv\" ORDER BY id", + "VALUES (3, 300), (4, 400), (5, 500)"); + + assertUpdate("DROP MATERIALIZED VIEW test_recreate_mv"); + assertUpdate("DROP TABLE test_recreate_base"); + } + + @Test + public void testStorageTableDroppedDirectly() + { + assertUpdate("CREATE TABLE test_storage_drop_base (id BIGINT, value BIGINT)"); + assertUpdate("INSERT INTO test_storage_drop_base VALUES (1, 100), (2, 200)", 2); + + assertUpdate("CREATE MATERIALIZED VIEW test_storage_drop_mv AS SELECT id, value FROM test_storage_drop_base"); + assertUpdate("REFRESH MATERIALIZED VIEW test_storage_drop_mv", 2); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_storage_drop_mv\"", "SELECT 2"); + + assertUpdate("DROP TABLE \"__mv_storage__test_storage_drop_mv\""); + + assertQueryFails("SELECT * FROM \"__mv_storage__test_storage_drop_mv\"", ".*does not exist.*"); + + assertQueryFails("SELECT * FROM test_storage_drop_mv", ".*does not exist.*"); + + assertUpdate("DROP MATERIALIZED VIEW test_storage_drop_mv"); + assertUpdate("DROP TABLE test_storage_drop_base"); + } + + @Test + public void testMaterializedViewWithRenamedColumns() + { + assertUpdate("CREATE TABLE test_renamed_base (id BIGINT, original_name VARCHAR, original_value BIGINT)"); + assertUpdate("INSERT INTO test_renamed_base VALUES (1, 'Alice', 100), (2, 'Bob', 200), (3, 'Charlie', 300)", 3); + + assertUpdate("CREATE MATERIALIZED VIEW test_renamed_mv AS " + + "SELECT id AS person_id, original_name AS full_name, original_value AS amount " + + "FROM test_renamed_base"); + + assertQuery("SELECT COUNT(*) FROM test_renamed_mv", "SELECT 3"); + assertQuery("SELECT * FROM test_renamed_mv ORDER BY person_id", + "VALUES (1, 'Alice', 100), (2, 'Bob', 200), (3, 'Charlie', 300)"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_renamed_mv", 3); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_renamed_mv\"", "SELECT 3"); + assertQuery("SELECT * FROM \"__mv_storage__test_renamed_mv\" ORDER BY person_id", + "VALUES (1, 'Alice', 100), (2, 'Bob', 200), (3, 'Charlie', 300)"); + + assertQuery("SELECT * FROM test_renamed_mv ORDER BY person_id", + "VALUES (1, 'Alice', 100), (2, 'Bob', 200), (3, 'Charlie', 300)"); + + assertQuery("SELECT person_id, full_name FROM test_renamed_mv WHERE amount > 150 ORDER BY person_id", + "VALUES (2, 'Bob'), (3, 'Charlie')"); + + assertUpdate("INSERT INTO test_renamed_base VALUES (4, 'Dave', 400)", 1); + + assertQuery("SELECT COUNT(*) FROM test_renamed_mv", "SELECT 4"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_renamed_mv", 4); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_renamed_mv\"", "SELECT 4"); + assertQuery("SELECT * FROM \"__mv_storage__test_renamed_mv\" ORDER BY person_id", + "VALUES (1, 'Alice', 100), (2, 'Bob', 200), (3, 'Charlie', 300), (4, 'Dave', 400)"); + + assertUpdate("DROP MATERIALIZED VIEW test_renamed_mv"); + assertUpdate("DROP TABLE test_renamed_base"); + } + + @Test + public void testMaterializedViewWithComputedColumns() + { + assertUpdate("CREATE TABLE test_computed_base (id BIGINT, quantity BIGINT, unit_price BIGINT)"); + assertUpdate("INSERT INTO test_computed_base VALUES (1, 5, 100), (2, 10, 50), (3, 3, 200)", 3); + + assertUpdate("CREATE MATERIALIZED VIEW test_computed_mv AS " + + "SELECT id, " + + "quantity, " + + "unit_price, " + + "quantity * unit_price AS total_price, " + + "quantity * 2 AS double_quantity, " + + "'Order_' || CAST(id AS VARCHAR) AS order_label " + + "FROM test_computed_base"); + + assertQuery("SELECT COUNT(*) FROM test_computed_mv", "SELECT 3"); + assertQuery("SELECT id, quantity, unit_price, total_price, double_quantity, order_label FROM test_computed_mv ORDER BY id", + "VALUES (1, 5, 100, 500, 10, 'Order_1'), " + + "(2, 10, 50, 500, 20, 'Order_2'), " + + "(3, 3, 200, 600, 6, 'Order_3')"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_computed_mv", 3); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_computed_mv\"", "SELECT 3"); + assertQuery("SELECT id, quantity, unit_price, total_price, double_quantity, order_label FROM \"__mv_storage__test_computed_mv\" ORDER BY id", + "VALUES (1, 5, 100, 500, 10, 'Order_1'), " + + "(2, 10, 50, 500, 20, 'Order_2'), " + + "(3, 3, 200, 600, 6, 'Order_3')"); + + assertQuery("SELECT * FROM test_computed_mv WHERE total_price > 550 ORDER BY id", + "VALUES (3, 3, 200, 600, 6, 'Order_3')"); + + assertQuery("SELECT id, order_label FROM test_computed_mv WHERE double_quantity >= 10 ORDER BY id", + "VALUES (1, 'Order_1'), (2, 'Order_2')"); + + assertUpdate("INSERT INTO test_computed_base VALUES (4, 8, 75)", 1); + + assertQuery("SELECT COUNT(*) FROM test_computed_mv", "SELECT 4"); + assertQuery("SELECT id, total_price, order_label FROM test_computed_mv WHERE id = 4", + "VALUES (4, 600, 'Order_4')"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_computed_mv", 4); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_computed_mv\"", "SELECT 4"); + assertQuery("SELECT id, quantity, unit_price, total_price, order_label FROM \"__mv_storage__test_computed_mv\" WHERE id = 4", + "VALUES (4, 8, 75, 600, 'Order_4')"); + + assertUpdate("DROP MATERIALIZED VIEW test_computed_mv"); + assertUpdate("DROP TABLE test_computed_base"); + } + + @Test + public void testMaterializedViewWithCustomTableProperties() + { + assertUpdate("CREATE TABLE test_custom_props_base (id BIGINT, name VARCHAR, region VARCHAR)"); + assertUpdate("INSERT INTO test_custom_props_base VALUES (1, 'Alice', 'US'), (2, 'Bob', 'EU'), (3, 'Charlie', 'APAC')", 3); + + assertUpdate("CREATE MATERIALIZED VIEW test_custom_props_mv " + + "WITH (" + + " partitioning = ARRAY['region'], " + + " sorted_by = ARRAY['id'], " + + " \"write.format.default\" = 'ORC'" + + ") AS " + + "SELECT id, name, region FROM test_custom_props_base"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_custom_props_mv", 3); + + assertQuery("SELECT COUNT(*) FROM test_custom_props_mv", "SELECT 3"); + assertQuery("SELECT name FROM test_custom_props_mv WHERE region = 'US'", "VALUES ('Alice')"); + assertQuery("SELECT name FROM test_custom_props_mv WHERE region = 'EU'", "VALUES ('Bob')"); + + String storageTableName = "__mv_storage__test_custom_props_mv"; + assertQuery("SELECT COUNT(*) FROM \"" + storageTableName + "\"", "SELECT 3"); + + assertQuery("SELECT COUNT(*) FROM \"" + storageTableName + "\" WHERE region = 'APAC'", "SELECT 1"); + + assertUpdate("INSERT INTO test_custom_props_base VALUES (4, 'David', 'US')", 1); + assertUpdate("REFRESH MATERIALIZED VIEW test_custom_props_mv", 4); + + assertQuery("SELECT COUNT(*) FROM test_custom_props_mv WHERE region = 'US'", "SELECT 2"); + assertQuery("SELECT name FROM test_custom_props_mv WHERE region = 'US' ORDER BY id", + "VALUES ('Alice'), ('David')"); + + assertUpdate("DROP MATERIALIZED VIEW test_custom_props_mv"); + assertUpdate("DROP TABLE test_custom_props_base"); + } + + @Test + public void testMaterializedViewWithNestedTypes() + { + assertUpdate("CREATE TABLE test_nested_base (" + + "id BIGINT, " + + "tags ARRAY(VARCHAR), " + + "properties MAP(VARCHAR, VARCHAR), " + + "address ROW(street VARCHAR, city VARCHAR, zipcode VARCHAR))"); + + assertUpdate("INSERT INTO test_nested_base VALUES " + + "(1, ARRAY['tag1', 'tag2'], MAP(ARRAY['key1', 'key2'], ARRAY['value1', 'value2']), ROW('123 Main St', 'NYC', '10001')), " + + "(2, ARRAY['tag3'], MAP(ARRAY['key3'], ARRAY['value3']), ROW('456 Oak Ave', 'LA', '90001'))", 2); + + assertUpdate("CREATE MATERIALIZED VIEW test_nested_mv AS " + + "SELECT id, tags, properties, address FROM test_nested_base"); + + assertQuery("SELECT COUNT(*) FROM test_nested_mv", "SELECT 2"); + assertQuery("SELECT id, cardinality(tags) FROM test_nested_mv ORDER BY id", + "VALUES (1, 2), (2, 1)"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_nested_mv", 2); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_nested_mv\"", "SELECT 2"); + + assertQuery("SELECT id, cardinality(tags), address.city FROM test_nested_mv ORDER BY id", + "VALUES (1, 2, 'NYC'), (2, 1, 'LA')"); + + assertQuery("SELECT id FROM test_nested_mv WHERE element_at(properties, 'key1') = 'value1'", + "VALUES (1)"); + + assertUpdate("INSERT INTO test_nested_base VALUES " + + "(3, ARRAY['tag4', 'tag5', 'tag6'], MAP(ARRAY['key4'], ARRAY['value4']), ROW('789 Elm St', 'Chicago', '60601'))", 1); + + assertQuery("SELECT COUNT(*) FROM test_nested_mv", "SELECT 3"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_nested_mv", 3); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_nested_mv\"", "SELECT 3"); + assertQuery("SELECT id, address.zipcode FROM test_nested_mv WHERE id = 3", + "VALUES (3, '60601')"); + + assertUpdate("DROP MATERIALIZED VIEW test_nested_mv"); + assertUpdate("DROP TABLE test_nested_base"); + } + + @Test + public void testMaterializedViewAfterColumnAdded() + { + assertUpdate("CREATE TABLE test_evolve_add_base (id BIGINT, name VARCHAR, value BIGINT)"); + assertUpdate("INSERT INTO test_evolve_add_base VALUES (1, 'Alice', 100), (2, 'Bob', 200)", 2); + + assertUpdate("CREATE MATERIALIZED VIEW test_evolve_add_mv AS " + + "SELECT id, name, value FROM test_evolve_add_base"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_evolve_add_mv", 2); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_evolve_add_mv\"", "SELECT 2"); + assertQuery("SELECT * FROM test_evolve_add_mv ORDER BY id", + "VALUES (1, 'Alice', 100), (2, 'Bob', 200)"); + + assertUpdate("ALTER TABLE test_evolve_add_base ADD COLUMN region VARCHAR"); + + assertUpdate("INSERT INTO test_evolve_add_base VALUES (3, 'Charlie', 300, 'US')", 1); + + assertQuery("SELECT COUNT(*) FROM test_evolve_add_mv", "SELECT 3"); + assertQuery("SELECT * FROM test_evolve_add_mv ORDER BY id", + "VALUES (1, 'Alice', 100), (2, 'Bob', 200), (3, 'Charlie', 300)"); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_evolve_add_mv\"", "SELECT 2"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_evolve_add_mv", 3); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_evolve_add_mv\"", "SELECT 3"); + assertQuery("SELECT * FROM \"__mv_storage__test_evolve_add_mv\" ORDER BY id", + "VALUES (1, 'Alice', 100), (2, 'Bob', 200), (3, 'Charlie', 300)"); + + assertUpdate("CREATE MATERIALIZED VIEW test_evolve_add_mv2 AS " + + "SELECT id, name, value, region FROM test_evolve_add_base"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_evolve_add_mv2", 3); + + assertQuery("SELECT * FROM test_evolve_add_mv2 WHERE id = 3", + "VALUES (3, 'Charlie', 300, 'US')"); + assertQuery("SELECT id, region FROM test_evolve_add_mv2 WHERE id IN (1, 2) ORDER BY id", + "VALUES (1, NULL), (2, NULL)"); + + assertUpdate("DROP MATERIALIZED VIEW test_evolve_add_mv"); + assertUpdate("DROP MATERIALIZED VIEW test_evolve_add_mv2"); + assertUpdate("DROP TABLE test_evolve_add_base"); + } + + @Test + public void testMaterializedViewAfterColumnDropped() + { + assertUpdate("CREATE TABLE test_evolve_drop_base (id BIGINT, name VARCHAR, value BIGINT, status VARCHAR)"); + assertUpdate("INSERT INTO test_evolve_drop_base VALUES (1, 'Alice', 100, 'active'), (2, 'Bob', 200, 'inactive')", 2); + + assertUpdate("CREATE MATERIALIZED VIEW test_evolve_drop_mv_all AS " + + "SELECT id, name, value, status FROM test_evolve_drop_base"); + + assertUpdate("CREATE MATERIALIZED VIEW test_evolve_drop_mv_subset AS " + + "SELECT id, name, value FROM test_evolve_drop_base"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_evolve_drop_mv_all", 2); + assertUpdate("REFRESH MATERIALIZED VIEW test_evolve_drop_mv_subset", 2); + + assertQuery("SELECT * FROM test_evolve_drop_mv_all ORDER BY id", + "VALUES (1, 'Alice', 100, 'active'), (2, 'Bob', 200, 'inactive')"); + assertQuery("SELECT * FROM test_evolve_drop_mv_subset ORDER BY id", + "VALUES (1, 'Alice', 100), (2, 'Bob', 200)"); + + assertUpdate("ALTER TABLE test_evolve_drop_base DROP COLUMN status"); + + assertUpdate("INSERT INTO test_evolve_drop_base VALUES (3, 'Charlie', 300)", 1); + + assertQuery("SELECT COUNT(*) FROM test_evolve_drop_mv_subset", "SELECT 3"); + assertQuery("SELECT * FROM test_evolve_drop_mv_subset ORDER BY id", + "VALUES (1, 'Alice', 100), (2, 'Bob', 200), (3, 'Charlie', 300)"); + + assertQueryFails("SELECT * FROM test_evolve_drop_mv_all", + ".*Column 'status' cannot be resolved.*"); + + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_evolve_drop_mv_all\"", "SELECT 2"); + assertQuery("SELECT * FROM \"__mv_storage__test_evolve_drop_mv_all\" ORDER BY id", + "VALUES (1, 'Alice', 100, 'active'), (2, 'Bob', 200, 'inactive')"); + + assertUpdate("REFRESH MATERIALIZED VIEW test_evolve_drop_mv_subset", 3); + assertQuery("SELECT COUNT(*) FROM \"__mv_storage__test_evolve_drop_mv_subset\"", "SELECT 3"); + + assertUpdate("DROP MATERIALIZED VIEW test_evolve_drop_mv_all"); + assertUpdate("DROP MATERIALIZED VIEW test_evolve_drop_mv_subset"); + assertUpdate("DROP TABLE test_evolve_drop_base"); + } + + @Test + public void testDropNonExistentMaterializedView() + { + assertQueryFails("DROP MATERIALIZED VIEW non_existent_mv", + ".*does not exist.*"); + } + + @Test + public void testCreateMaterializedViewWithSameNameAsExistingTable() + { + assertUpdate("CREATE TABLE existing_table_name (id BIGINT, value VARCHAR)"); + assertUpdate("INSERT INTO existing_table_name VALUES (1, 'test')", 1); + + assertQueryFails("CREATE MATERIALIZED VIEW existing_table_name AS SELECT id, value FROM existing_table_name", + ".*already exists.*"); + + assertQuery("SELECT COUNT(*) FROM existing_table_name", "SELECT 1"); + assertQuery("SELECT * FROM existing_table_name", "VALUES (1, 'test')"); + + assertUpdate("CREATE TABLE test_mv_base (id BIGINT, name VARCHAR)"); + assertUpdate("INSERT INTO test_mv_base VALUES (2, 'foo')", 1); + + assertQueryFails("CREATE MATERIALIZED VIEW existing_table_name AS SELECT id, name FROM test_mv_base", + ".*already exists.*"); + + assertUpdate("DROP TABLE existing_table_name"); + assertUpdate("DROP TABLE test_mv_base"); + } +} diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestStatisticsUtil.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestStatisticsUtil.java index 6d894ddb934c2..99d4e728e56ce 100644 --- a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestStatisticsUtil.java +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestStatisticsUtil.java @@ -287,7 +287,7 @@ public void testGenerateStatisticColumnSets() .setDataColumns(ImmutableList.of()) .setPredicateColumns(ImmutableMap.of()) .setRequestedColumns(Optional.empty()) - .setTable(new IcebergTableHandle("test", IcebergTableName.from("test"), false, Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), ImmutableList.of(), ImmutableList.of())) + .setTable(new IcebergTableHandle("test", IcebergTableName.from("test"), false, Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), Optional.empty(), ImmutableList.of(), ImmutableList.of(), Optional.empty())) .setDomainPredicate(TupleDomain.all()); // verify all selected columns are included List includedColumns = combineSelectedAndPredicateColumns( diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/hive/TestRenameTableOnFragileFileSystem.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/hive/TestRenameTableOnFragileFileSystem.java index 4e7f439cc50fc..46d4b7dc7478b 100644 --- a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/hive/TestRenameTableOnFragileFileSystem.java +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/hive/TestRenameTableOnFragileFileSystem.java @@ -76,6 +76,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import com.google.common.reflect.TypeToken; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -208,7 +209,8 @@ public String formatRowExpression(ConnectorSession session, RowExpression expres Optional.empty(), Optional.empty(), ImmutableList.of(), - ImmutableList.of()); + ImmutableList.of(), + Optional.empty()); @Test public void testRenameTableSucceed() @@ -413,6 +415,7 @@ private ConnectorMetadata getIcebergHiveMetadata(ExtendedHiveMetastore metastore FUNCTION_RESOLUTION, ROW_EXPRESSION_SERVICE, jsonCodec(CommitTaskData.class), + jsonCodec(new TypeToken<>() {}), new NodeVersion("test_node_v1"), FILTER_STATS_CALCULATOR_SERVICE, new IcebergHiveTableOperationsConfig(), diff --git a/presto-main-base/src/main/java/com/facebook/presto/sql/analyzer/StatementAnalyzer.java b/presto-main-base/src/main/java/com/facebook/presto/sql/analyzer/StatementAnalyzer.java index d4935c30f92fc..835f5c7fcdf0a 100644 --- a/presto-main-base/src/main/java/com/facebook/presto/sql/analyzer/StatementAnalyzer.java +++ b/presto-main-base/src/main/java/com/facebook/presto/sql/analyzer/StatementAnalyzer.java @@ -2362,7 +2362,7 @@ private Scope processMaterializedView( QualifiedName dataTableName = QualifiedName.of( materializedViewName.getCatalogName(), - materializedViewName.getSchemaName(), + materializedViewDefinition.getSchema(), materializedViewDefinition.getTable()); Table dataTable = new Table(dataTableName); diff --git a/presto-native-execution/presto_cpp/presto_protocol/connector/iceberg/presto_protocol_iceberg.cpp b/presto-native-execution/presto_cpp/presto_protocol/connector/iceberg/presto_protocol_iceberg.cpp index 302a8eb4a234b..228d31f569937 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/connector/iceberg/presto_protocol_iceberg.cpp +++ b/presto-native-execution/presto_cpp/presto_protocol/connector/iceberg/presto_protocol_iceberg.cpp @@ -845,6 +845,13 @@ void to_json(json& j, const IcebergInsertTableHandle& p) { "IcebergInsertTableHandle", "List", "sortOrder"); + to_json_key( + j, + "materializedViewName", + p.materializedViewName, + "IcebergInsertTableHandle", + "SchemaTableName", + "materializedViewName"); } void from_json(const json& j, IcebergInsertTableHandle& p) { @@ -919,6 +926,13 @@ void from_json(const json& j, IcebergInsertTableHandle& p) { "IcebergInsertTableHandle", "List", "sortOrder"); + from_json_key( + j, + "materializedViewName", + p.materializedViewName, + "IcebergInsertTableHandle", + "SchemaTableName", + "materializedViewName"); } } // namespace facebook::presto::protocol::iceberg namespace facebook::presto::protocol::iceberg { @@ -1317,6 +1331,13 @@ void to_json(json& j, const IcebergTableHandle& p) { "IcebergTableHandle", "List", "updatedColumns"); + to_json_key( + j, + "materializedViewName", + p.materializedViewName, + "IcebergTableHandle", + "SchemaTableName", + "materializedViewName"); } void from_json(const json& j, IcebergTableHandle& p) { @@ -1391,6 +1412,13 @@ void from_json(const json& j, IcebergTableHandle& p) { "IcebergTableHandle", "List", "updatedColumns"); + from_json_key( + j, + "materializedViewName", + p.materializedViewName, + "IcebergTableHandle", + "SchemaTableName", + "materializedViewName"); } } // namespace facebook::presto::protocol::iceberg namespace facebook::presto::protocol::iceberg { diff --git a/presto-native-execution/presto_cpp/presto_protocol/connector/iceberg/presto_protocol_iceberg.h b/presto-native-execution/presto_cpp/presto_protocol/connector/iceberg/presto_protocol_iceberg.h index d82e660ab42a8..a659dc24d103b 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/connector/iceberg/presto_protocol_iceberg.h +++ b/presto-native-execution/presto_cpp/presto_protocol/connector/iceberg/presto_protocol_iceberg.h @@ -206,6 +206,7 @@ struct IcebergInsertTableHandle : public ConnectorInsertTableHandle { hive::HiveCompressionCodec compressionCodec = {}; Map storageProperties = {}; List sortOrder = {}; + std::shared_ptr materializedViewName = {}; IcebergInsertTableHandle() noexcept; }; @@ -270,6 +271,7 @@ struct IcebergTableHandle : public ConnectorTableHandle { std::shared_ptr> equalityFieldIds = {}; List sortOrder = {}; List updatedColumns = {}; + std::shared_ptr materializedViewName = {}; IcebergTableHandle() noexcept; }; diff --git a/presto-native-execution/presto_cpp/presto_protocol/connector/iceberg/special/IcebergInsertTableHandle.hpp.inc b/presto-native-execution/presto_cpp/presto_protocol/connector/iceberg/special/IcebergInsertTableHandle.hpp.inc index 91b11fa183a53..cda054d046e5a 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/connector/iceberg/special/IcebergInsertTableHandle.hpp.inc +++ b/presto-native-execution/presto_cpp/presto_protocol/connector/iceberg/special/IcebergInsertTableHandle.hpp.inc @@ -27,6 +27,7 @@ struct IcebergInsertTableHandle : public ConnectorInsertTableHandle { hive::HiveCompressionCodec compressionCodec = {}; Map storageProperties = {}; List sortOrder = {}; + std::shared_ptr materializedViewName = {}; IcebergInsertTableHandle() noexcept; }; From 6c1a426ef4e5461097c2ec325171bc16d939775c Mon Sep 17 00:00:00 2001 From: Tim Meehan Date: Mon, 24 Nov 2025 12:33:32 -0500 Subject: [PATCH 2/2] Address review feedback, add format version to materialized view properties --- .../iceberg/IcebergAbstractMetadata.java | 63 ++++++---- .../IcebergEqualityDeleteAsJoin.java | 7 +- .../TestIcebergMaterializedViewMetadata.java | 115 ++++++++++++++++++ .../iceberg/TestIcebergMaterializedViews.java | 111 ----------------- 4 files changed, 160 insertions(+), 136 deletions(-) diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergAbstractMetadata.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergAbstractMetadata.java index 8c7736205548d..eb6ea59912b76 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergAbstractMetadata.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergAbstractMetadata.java @@ -237,6 +237,7 @@ public abstract class IcebergAbstractMetadata protected static final String INFORMATION_SCHEMA = "information_schema"; // Materialized view metadata property keys + protected static final String PRESTO_MATERIALIZED_VIEW_FORMAT_VERSION = "presto.materialized_view.format_version"; protected static final String PRESTO_MATERIALIZED_VIEW_ORIGINAL_SQL = "presto.materialized_view.original_sql"; protected static final String PRESTO_MATERIALIZED_VIEW_BASE_TABLES = "presto.materialized_view.base_tables"; protected static final String PRESTO_MATERIALIZED_VIEW_BASE_SNAPSHOT_PREFIX = "presto.materialized_view.base_snapshot."; @@ -247,6 +248,8 @@ public abstract class IcebergAbstractMetadata protected static final String PRESTO_MATERIALIZED_VIEW_OWNER = "presto.materialized_view.owner"; protected static final String PRESTO_MATERIALIZED_VIEW_SECURITY_MODE = "presto.materialized_view.security_mode"; + protected static final int CURRENT_MATERIALIZED_VIEW_FORMAT_VERSION = 1; + protected final TypeManager typeManager; protected final JsonCodec commitTaskCodec; protected final JsonCodec> columnMappingsCodec; @@ -1032,17 +1035,14 @@ public IcebergTableHandle getTableHandle(ConnectorSession session, SchemaTableNa SchemaTableName storageTableName = new SchemaTableName(definition.getSchema(), definition.getTable()); Table storageTable = getIcebergTable(session, storageTableName); - // TODO: time travel for materialized views doesn't work properly - Optional tableSnapshotId = tableVersion - .map(version -> { - long tableVersionSnapshotId = getSnapshotIdForTableVersion(storageTable, version); - return Optional.of(tableVersionSnapshotId); - }) - .orElseGet(() -> resolveSnapshotIdByName(storageTable, name)); + // Time travel on the materialized view itself is not supported + if (tableVersion.isPresent() || name.getSnapshotId().isPresent()) { + throw new PrestoException(NOT_SUPPORTED, "Time travel queries on materialized views are not supported"); + } return new IcebergTableHandle( storageTableName.getSchemaName(), - new IcebergTableName(storageTableName.getTableName(), name.getTableType(), tableSnapshotId, name.getChangelogEndSnapshot()), + new IcebergTableName(storageTableName.getTableName(), name.getTableType(), Optional.empty(), Optional.empty()), name.getSnapshotId().isPresent(), tryGetLocation(storageTable), tryGetProperties(storageTable), @@ -1465,6 +1465,7 @@ public void createMaterializedView( createTable(session, storageTableMetadata, false); Map properties = new HashMap<>(); + properties.put(PRESTO_MATERIALIZED_VIEW_FORMAT_VERSION, CURRENT_MATERIALIZED_VIEW_FORMAT_VERSION + ""); properties.put(PRESTO_MATERIALIZED_VIEW_ORIGINAL_SQL, viewDefinition.getOriginalSql()); properties.put(PRESTO_MATERIALIZED_VIEW_STORAGE_SCHEMA, storageTableName.getSchemaName()); properties.put(PRESTO_MATERIALIZED_VIEW_STORAGE_TABLE_NAME, storageTableName.getTableName()); @@ -1480,11 +1481,7 @@ public void createMaterializedView( properties.put(PRESTO_MATERIALIZED_VIEW_SECURITY_MODE, viewDefinition.getSecurityMode().get().name()); for (SchemaTableName baseTable : viewDefinition.getBaseTables()) { - Table baseIcebergTable = getIcebergTable(session, baseTable); - long baseSnapshotId = baseIcebergTable.currentSnapshot() != null - ? baseIcebergTable.currentSnapshot().snapshotId() - : 0L; - properties.put(getBaseTableViewPropertyName(baseTable), baseSnapshotId + ""); + properties.put(getBaseTableViewPropertyName(baseTable), "0"); } createIcebergView(session, viewName, viewMetadata.getColumns(), viewDefinition.getOriginalSql(), properties); @@ -1510,6 +1507,23 @@ public Optional getMaterializedView(ConnectorSession return Optional.empty(); } + // Validate format version + String formatVersion = getRequiredMaterializedViewProperty(viewProperties, PRESTO_MATERIALIZED_VIEW_FORMAT_VERSION); + int version; + try { + version = Integer.parseInt(formatVersion); + } + catch (NumberFormatException e) { + throw new PrestoException(ICEBERG_INVALID_MATERIALIZED_VIEW, + format("Invalid materialized view format version: %s", formatVersion)); + } + + if (version != CURRENT_MATERIALIZED_VIEW_FORMAT_VERSION) { + throw new PrestoException(ICEBERG_INVALID_MATERIALIZED_VIEW, + format("Materialized view format version %d is not supported by this version of Presto (current version: %d). Please upgrade Presto.", + version, CURRENT_MATERIALIZED_VIEW_FORMAT_VERSION)); + } + String baseTablesStr = getRequiredMaterializedViewProperty(viewProperties, PRESTO_MATERIALIZED_VIEW_BASE_TABLES); List baseTables; if (baseTablesStr.isEmpty()) { @@ -1570,9 +1584,8 @@ public void dropMaterializedView(ConnectorSession session, SchemaTableName viewN { Optional definition = getMaterializedView(session, viewName); - dropIcebergView(session, viewName); - if (definition.isPresent()) { + dropIcebergView(session, viewName); SchemaTableName storageTableName = new SchemaTableName( definition.get().getSchema(), definition.get().getTable()); @@ -1609,7 +1622,11 @@ public MaterializedViewStatus getMaterializedViewStatus( String key = getBaseTableViewPropertyName(baseTable); String recordedSnapshotStr = props.get(key); - long recordedSnapshotId = recordedSnapshotStr != null ? parseLong(recordedSnapshotStr) : -1L; + if (recordedSnapshotStr == null) { + throw new PrestoException(ICEBERG_INVALID_MATERIALIZED_VIEW, + format("Missing base table snapshot property for %s in materialized view %s", baseTable, materializedViewName)); + } + long recordedSnapshotId = parseLong(recordedSnapshotStr); if (currentSnapshotId != recordedSnapshotId) { return new MaterializedViewStatus(PARTIALLY_MATERIALIZED, ImmutableMap.of()); @@ -1624,15 +1641,15 @@ public ConnectorInsertTableHandle beginRefreshMaterializedView( ConnectorSession session, ConnectorTableHandle tableHandle) { - IcebergTableHandle mvHandle = (IcebergTableHandle) tableHandle; + IcebergTableHandle icebergTableHandle = (IcebergTableHandle) tableHandle; - if (mvHandle.getMaterializedViewName().isEmpty()) { + if (icebergTableHandle.getMaterializedViewName().isEmpty()) { throw new IllegalStateException(format( "beginRefreshMaterializedView called on non-materialized view table: %s", - mvHandle.getSchemaTableName())); + icebergTableHandle.getSchemaTableName())); } - SchemaTableName storageTableName = mvHandle.getSchemaTableName(); + SchemaTableName storageTableName = icebergTableHandle.getSchemaTableName(); IcebergTableHandle storageTableHandle = getTableHandle(session, storageTableName); Table storageTable = getIcebergTable(session, storageTableName); @@ -1640,8 +1657,7 @@ public ConnectorInsertTableHandle beginRefreshMaterializedView( transaction.newDelete().deleteFromRowFilter(alwaysTrue()).commit(); - SchemaTableName mvName = mvHandle.getMaterializedViewName() - .orElseThrow(() -> new PrestoException(ICEBERG_INVALID_MATERIALIZED_VIEW, "materializedViewName not set in handle for MV refresh")); + SchemaTableName materializedViewName = icebergTableHandle.getMaterializedViewName().get(); return new IcebergInsertTableHandle( storageTableHandle.getSchemaName(), @@ -1654,7 +1670,7 @@ public ConnectorInsertTableHandle beginRefreshMaterializedView( getCompressionCodec(session), storageTable.properties(), getSupportedSortFields(storageTable.schema(), storageTable.sortOrder()), - Optional.of(mvName)); + Optional.of(materializedViewName)); } @Override @@ -1685,7 +1701,6 @@ public Optional finishRefreshMaterializedView( if (definition.isPresent()) { for (SchemaTableName baseTable : definition.get().getBaseTables()) { try { - // TODO: this is not transactional - base table may change between refresh start and end Table baseIcebergTable = getIcebergTable(session, baseTable); long baseSnapshotId = baseIcebergTable.currentSnapshot() != null ? baseIcebergTable.currentSnapshot().snapshotId() diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/optimizer/IcebergEqualityDeleteAsJoin.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/optimizer/IcebergEqualityDeleteAsJoin.java index 36b41a3e3c5ff..ebef2d465de58 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/optimizer/IcebergEqualityDeleteAsJoin.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/optimizer/IcebergEqualityDeleteAsJoin.java @@ -187,6 +187,11 @@ public PlanNode visitTableScan(TableScanNode node, RewriteContext context) return node; } + if (icebergTableHandle.getMaterializedViewName().isPresent()) { + // Materialized views should not have delete files + return node; + } + IcebergAbstractMetadata metadata = (IcebergAbstractMetadata) transactionManager.get(table.getTransaction()); Table icebergTable = getIcebergTable(metadata, session, icebergTableHandle.getSchemaTableName()); @@ -355,7 +360,7 @@ private TableScanNode createDeletesTableScan(ImmutableMap properties = new HashMap<>(); + properties.put(PRESTO_MATERIALIZED_VIEW_FORMAT_VERSION, CURRENT_MATERIALIZED_VIEW_FORMAT_VERSION + ""); properties.put(PRESTO_MATERIALIZED_VIEW_ORIGINAL_SQL, "SELECT 1 as id"); properties.put(PRESTO_MATERIALIZED_VIEW_STORAGE_SCHEMA, "test_schema"); properties.put(PRESTO_MATERIALIZED_VIEW_STORAGE_TABLE_NAME, "storage_empty_base"); @@ -322,6 +325,7 @@ public void testMaterializedViewOtherValidationErrors() TableIdentifier viewId2 = TableIdentifier.of(Namespace.of("test_schema"), "test_mv_invalid_json"); Map properties2 = new HashMap<>(); + properties2.put(PRESTO_MATERIALIZED_VIEW_FORMAT_VERSION, CURRENT_MATERIALIZED_VIEW_FORMAT_VERSION + ""); properties2.put(PRESTO_MATERIALIZED_VIEW_ORIGINAL_SQL, "SELECT id FROM test_other_validation_base"); properties2.put(PRESTO_MATERIALIZED_VIEW_STORAGE_SCHEMA, "test_schema"); properties2.put(PRESTO_MATERIALIZED_VIEW_STORAGE_TABLE_NAME, "storage_invalid_json"); @@ -348,6 +352,7 @@ public void testMaterializedViewOtherValidationErrors() TableIdentifier viewId3 = TableIdentifier.of(Namespace.of("test_schema"), "test_mv_nonexistent_base"); Map properties3 = new HashMap<>(); + properties3.put(PRESTO_MATERIALIZED_VIEW_FORMAT_VERSION, CURRENT_MATERIALIZED_VIEW_FORMAT_VERSION + ""); properties3.put(PRESTO_MATERIALIZED_VIEW_ORIGINAL_SQL, "SELECT id FROM nonexistent_table"); properties3.put(PRESTO_MATERIALIZED_VIEW_STORAGE_SCHEMA, "test_schema"); properties3.put(PRESTO_MATERIALIZED_VIEW_STORAGE_TABLE_NAME, "storage_nonexistent_base"); @@ -518,9 +523,119 @@ public void testMaterializedViewInvalidBaseTableNameFormat() assertUpdate("DROP TABLE test_format_base"); } + @Test + public void testMaterializedViewMissingRequiredProperties() + throws Exception + { + assertUpdate("CREATE TABLE test_validation_base (id BIGINT, name VARCHAR)"); + assertUpdate("INSERT INTO test_validation_base VALUES (1, 'Alice')", 1); + + RESTCatalog catalog = new RESTCatalog(); + Map catalogProps = new HashMap<>(); + catalogProps.put("uri", serverUri); + catalogProps.put("warehouse", warehouseLocation.getAbsolutePath()); + catalog.initialize("test_catalog", catalogProps); + + try { + TableIdentifier viewId1 = TableIdentifier.of(Namespace.of("test_schema"), "test_mv_missing_base_tables"); + Map properties1 = new HashMap<>(); + properties1.put("presto.materialized_view.format_version", CURRENT_MATERIALIZED_VIEW_FORMAT_VERSION + ""); + properties1.put("presto.materialized_view.original_sql", "SELECT id, name FROM test_validation_base"); + properties1.put("presto.materialized_view.storage_schema", "test_schema"); + properties1.put("presto.materialized_view.storage_table_name", "storage1"); + properties1.put("presto.materialized_view.column_mappings", "[]"); + + catalog.buildView(viewId1) + .withSchema(new org.apache.iceberg.Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required(2, "name", Types.StringType.get()))) + .withQuery("spark", "SELECT id, name FROM test_validation_base") + .withDefaultNamespace(Namespace.of("test_schema")) + .withProperties(properties1) + .create(); + + assertQueryFails("SELECT * FROM test_mv_missing_base_tables", + ".*Materialized view missing required property: presto.materialized_view.base_tables.*"); + + catalog.dropView(viewId1); + + TableIdentifier viewId2 = TableIdentifier.of(Namespace.of("test_schema"), "test_mv_missing_column_mappings"); + Map properties2 = new HashMap<>(); + properties2.put("presto.materialized_view.format_version", CURRENT_MATERIALIZED_VIEW_FORMAT_VERSION + ""); + properties2.put("presto.materialized_view.original_sql", "SELECT id, name FROM test_validation_base"); + properties2.put("presto.materialized_view.storage_schema", "test_schema"); + properties2.put("presto.materialized_view.storage_table_name", "storage2"); + properties2.put("presto.materialized_view.base_tables", "test_schema.test_validation_base"); + + catalog.buildView(viewId2) + .withSchema(new org.apache.iceberg.Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required(2, "name", Types.StringType.get()))) + .withQuery("spark", "SELECT id, name FROM test_validation_base") + .withDefaultNamespace(Namespace.of("test_schema")) + .withProperties(properties2) + .create(); + + assertQueryFails("SELECT * FROM test_mv_missing_column_mappings", + ".*Materialized view missing required property: presto.materialized_view.column_mappings.*"); + + catalog.dropView(viewId2); + + TableIdentifier viewId3 = TableIdentifier.of(Namespace.of("test_schema"), "test_mv_missing_storage_schema"); + Map properties3 = new HashMap<>(); + properties3.put("presto.materialized_view.format_version", CURRENT_MATERIALIZED_VIEW_FORMAT_VERSION + ""); + properties3.put("presto.materialized_view.original_sql", "SELECT id, name FROM test_validation_base"); + properties3.put("presto.materialized_view.storage_table_name", "storage3"); + properties3.put("presto.materialized_view.base_tables", "test_schema.test_validation_base"); + properties3.put("presto.materialized_view.column_mappings", "[]"); + + catalog.buildView(viewId3) + .withSchema(new org.apache.iceberg.Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required(2, "name", Types.StringType.get()))) + .withQuery("spark", "SELECT id, name FROM test_validation_base") + .withDefaultNamespace(Namespace.of("test_schema")) + .withProperties(properties3) + .create(); + + assertQueryFails("SELECT * FROM test_mv_missing_storage_schema", + ".*Materialized view missing required property: presto.materialized_view.storage_schema.*"); + + catalog.dropView(viewId3); + + TableIdentifier viewId4 = TableIdentifier.of(Namespace.of("test_schema"), "test_mv_missing_storage_table_name"); + Map properties4 = new HashMap<>(); + properties4.put("presto.materialized_view.format_version", CURRENT_MATERIALIZED_VIEW_FORMAT_VERSION + ""); + properties4.put("presto.materialized_view.original_sql", "SELECT id, name FROM test_validation_base"); + properties4.put("presto.materialized_view.storage_schema", "test_schema"); + properties4.put("presto.materialized_view.base_tables", "test_schema.test_validation_base"); + properties4.put("presto.materialized_view.column_mappings", "[]"); + + catalog.buildView(viewId4) + .withSchema(new org.apache.iceberg.Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required(2, "name", Types.StringType.get()))) + .withQuery("spark", "SELECT id, name FROM test_validation_base") + .withDefaultNamespace(Namespace.of("test_schema")) + .withProperties(properties4) + .create(); + + assertQueryFails("SELECT * FROM test_mv_missing_storage_table_name", + ".*Materialized view missing required property: presto.materialized_view.storage_table_name.*"); + + catalog.dropView(viewId4); + } + finally { + catalog.close(); + } + + assertUpdate("DROP TABLE test_validation_base"); + } + private Map createValidMvProperties(String storageTableName) { Map properties = new HashMap<>(); + properties.put(PRESTO_MATERIALIZED_VIEW_FORMAT_VERSION, CURRENT_MATERIALIZED_VIEW_FORMAT_VERSION + ""); properties.put(PRESTO_MATERIALIZED_VIEW_ORIGINAL_SQL, "SELECT id FROM test_format_base"); properties.put(PRESTO_MATERIALIZED_VIEW_STORAGE_SCHEMA, "test_schema"); properties.put(PRESTO_MATERIALIZED_VIEW_STORAGE_TABLE_NAME, storageTableName); diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergMaterializedViews.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergMaterializedViews.java index ee4d4d51eb16e..638732f6b2bda 100644 --- a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergMaterializedViews.java +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergMaterializedViews.java @@ -18,18 +18,12 @@ import com.facebook.presto.testing.QueryRunner; import com.facebook.presto.tests.AbstractTestQueryFramework; import com.google.common.collect.ImmutableMap; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.rest.RESTCatalog; -import org.apache.iceberg.types.Types; import org.assertj.core.util.Files; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; import java.io.File; -import java.util.HashMap; -import java.util.Map; import java.util.Optional; import static com.facebook.presto.iceberg.CatalogType.REST; @@ -1028,111 +1022,6 @@ public void testMaterializedViewOnEmptyBaseTable() assertUpdate("DROP TABLE test_empty_base"); } - @Test - public void testMaterializedViewMissingRequiredProperties() - throws Exception - { - assertUpdate("CREATE TABLE test_validation_base (id BIGINT, name VARCHAR)"); - assertUpdate("INSERT INTO test_validation_base VALUES (1, 'Alice')", 1); - - RESTCatalog catalog = new RESTCatalog(); - Map catalogProps = new HashMap<>(); - catalogProps.put("uri", serverUri); - catalogProps.put("warehouse", warehouseLocation.getAbsolutePath()); - catalog.initialize("test_catalog", catalogProps); - - try { - TableIdentifier viewId1 = TableIdentifier.of(Namespace.of("test_schema"), "test_mv_missing_base_tables"); - Map properties1 = new HashMap<>(); - properties1.put("presto.materialized_view.original_sql", "SELECT id, name FROM test_validation_base"); - properties1.put("presto.materialized_view.storage_schema", "test_schema"); - properties1.put("presto.materialized_view.storage_table_name", "storage1"); - properties1.put("presto.materialized_view.column_mappings", "[]"); - - catalog.buildView(viewId1) - .withSchema(new org.apache.iceberg.Schema( - Types.NestedField.required(1, "id", Types.LongType.get()), - Types.NestedField.required(2, "name", Types.StringType.get()))) - .withQuery("spark", "SELECT id, name FROM test_validation_base") - .withDefaultNamespace(Namespace.of("test_schema")) - .withProperties(properties1) - .create(); - - assertQueryFails("SELECT * FROM test_mv_missing_base_tables", - ".*Materialized view missing required property: presto.materialized_view.base_tables.*"); - - catalog.dropView(viewId1); - - TableIdentifier viewId2 = TableIdentifier.of(Namespace.of("test_schema"), "test_mv_missing_column_mappings"); - Map properties2 = new HashMap<>(); - properties2.put("presto.materialized_view.original_sql", "SELECT id, name FROM test_validation_base"); - properties2.put("presto.materialized_view.storage_schema", "test_schema"); - properties2.put("presto.materialized_view.storage_table_name", "storage2"); - properties2.put("presto.materialized_view.base_tables", "test_schema.test_validation_base"); - - catalog.buildView(viewId2) - .withSchema(new org.apache.iceberg.Schema( - Types.NestedField.required(1, "id", Types.LongType.get()), - Types.NestedField.required(2, "name", Types.StringType.get()))) - .withQuery("spark", "SELECT id, name FROM test_validation_base") - .withDefaultNamespace(Namespace.of("test_schema")) - .withProperties(properties2) - .create(); - - assertQueryFails("SELECT * FROM test_mv_missing_column_mappings", - ".*Materialized view missing required property: presto.materialized_view.column_mappings.*"); - - catalog.dropView(viewId2); - - TableIdentifier viewId3 = TableIdentifier.of(Namespace.of("test_schema"), "test_mv_missing_storage_schema"); - Map properties3 = new HashMap<>(); - properties3.put("presto.materialized_view.original_sql", "SELECT id, name FROM test_validation_base"); - properties3.put("presto.materialized_view.storage_table_name", "storage3"); - properties3.put("presto.materialized_view.base_tables", "test_schema.test_validation_base"); - properties3.put("presto.materialized_view.column_mappings", "[]"); - - catalog.buildView(viewId3) - .withSchema(new org.apache.iceberg.Schema( - Types.NestedField.required(1, "id", Types.LongType.get()), - Types.NestedField.required(2, "name", Types.StringType.get()))) - .withQuery("spark", "SELECT id, name FROM test_validation_base") - .withDefaultNamespace(Namespace.of("test_schema")) - .withProperties(properties3) - .create(); - - assertQueryFails("SELECT * FROM test_mv_missing_storage_schema", - ".*Materialized view missing required property: presto.materialized_view.storage_schema.*"); - - catalog.dropView(viewId3); - - TableIdentifier viewId4 = TableIdentifier.of(Namespace.of("test_schema"), "test_mv_missing_storage_table_name"); - Map properties4 = new HashMap<>(); - properties4.put("presto.materialized_view.original_sql", "SELECT id, name FROM test_validation_base"); - properties4.put("presto.materialized_view.storage_schema", "test_schema"); - properties4.put("presto.materialized_view.base_tables", "test_schema.test_validation_base"); - properties4.put("presto.materialized_view.column_mappings", "[]"); - - catalog.buildView(viewId4) - .withSchema(new org.apache.iceberg.Schema( - Types.NestedField.required(1, "id", Types.LongType.get()), - Types.NestedField.required(2, "name", Types.StringType.get()))) - .withQuery("spark", "SELECT id, name FROM test_validation_base") - .withDefaultNamespace(Namespace.of("test_schema")) - .withProperties(properties4) - .create(); - - assertQueryFails("SELECT * FROM test_mv_missing_storage_table_name", - ".*Materialized view missing required property: presto.materialized_view.storage_table_name.*"); - - catalog.dropView(viewId4); - } - finally { - catalog.close(); - } - - assertUpdate("DROP TABLE test_validation_base"); - } - @Test public void testRefreshFailurePreservesOldData() {