# Copyright Materialize, Inc. and contributors. All rights reserved. # # Use of this software is governed by the Business Source License # included in the LICENSE file at the root of this repository. # # As of the Change Date specified in that file, in accordance with # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. mode cockroach # Setup statement ok CREATE TABLE t (a int, b int) statement ok INSERT INTO t VALUES (1, 2), (3, 4), (5, 6) statement ok CREATE CLUSTER other REPLICAS (r1 (SIZE '1'), r2 (SIZE '2-2')) # Test: Materialized view can be created. statement ok CREATE MATERIALIZED VIEW mv AS SELECT 1 # Test: Materialized view can be replaced. statement ok CREATE OR REPLACE MATERIALIZED VIEW mv AS SELECT 2 query I SELECT * FROM mv ---- 2 # Test: Materialized view creation can be skipped if a materialized view already exists. statement error materialized view "materialize.public.mv" already exists CREATE MATERIALIZED VIEW mv AS SELECT 1 statement ok CREATE MATERIALIZED VIEW IF NOT EXISTS mv AS SELECT 1 query I SELECT * FROM mv ---- 2 # Test: Materialized view can have explicit column names. statement ok CREATE OR REPLACE MATERIALIZED VIEW mv (name, age) AS SELECT 'jon', 12 query TI colnames SELECT * FROM mv ---- name age jon 12 # Test: Explicit column names must have the right cardinality. statement error materialized view .+ definition names 2 columns, but materialized view .+ has 1 column CREATE MATERIALIZED VIEW error (name, age) AS SELECT 'jon' # Test: Materialized view can be created in another cluster. statement ok CREATE MATERIALIZED VIEW other_mv IN CLUSTER other AS SELECT 1 query TTT colnames,rowsort SHOW MATERIALIZED VIEWS ---- name cluster comment mv quickstart (empty) other_mv other (empty) statement ok DROP MATERIALIZED VIEW other_mv # Test: Materialized view can not be created in a non-existing cluster. statement error unknown cluster 'doesnotexist' CREATE MATERIALIZED VIEW error IN CLUSTER doesnotexist AS SELECT 1 # Test: Materialized view data is accessible from the same cluster. statement ok CREATE OR REPLACE MATERIALIZED VIEW mv AS SELECT a + b FROM t query I rowsort SELECT * FROM mv ---- 3 7 11 # Test: Materialized view data is accessible from other clusters. statement ok SET cluster = other query I rowsort SELECT * FROM mv ---- 3 7 11 statement ok RESET cluster # Test: Materialized view reflects input data changes. statement ok INSERT INTO t VALUES (7, 8) query I rowsort SELECT * FROM mv ---- 3 7 11 15 statement ok DELETE FROM t WHERE a = 1 query I rowsort SELECT * FROM mv ---- 7 11 15 # Test: Query errors are propagated through materialized views. statement ok CREATE OR REPLACE MATERIALIZED VIEW mv AS SELECT 100 / a FROM t query I rowsort SELECT * FROM mv ---- 14 20 33 statement ok INSERT INTO t VALUES (0, 0) query error Evaluation error: division by zero SELECT * FROM mv statement ok DELETE FROM t WHERE a = 0 query I rowsort SELECT * FROM mv ---- 14 20 33 # Test: Materialized views can be nested. statement ok CREATE MATERIALIZED VIEW mv2 AS SELECT count(*) FROM mv query I SELECT * FROM mv2 ---- 3 statement ok DROP MATERIALIZED VIEW mv2 # Test: Materialized views can have indexes on top. statement ok CREATE DEFAULT INDEX ON mv; # Test: Materialized views can be dropped. statement ok CREATE OR REPLACE MATERIALIZED VIEW mv AS SELECT 1 statement ok DROP MATERIALIZED VIEW mv # Test: Materialized views can not be dropped if they have dependants. statement ok CREATE MATERIALIZED VIEW mv AS SELECT 1 statement ok CREATE VIEW v AS SELECT * FROM mv statement error cannot drop materialized view "mv": still depended upon by view "v" DROP MATERIALIZED VIEW mv # Test: Materialized views with dependants can be dropped with CASCADE. statement ok DROP MATERIALIZED VIEW mv CASCADE query error unknown catalog item 'v' SELECT * FROM v # Test: a view on a materialized view that optimizes to the empty set # still prevents the underlying view from being dropped. # See: https://github.com/MaterializeInc/database-issues/issues/6101 statement ok CREATE VIEW v AS SELECT 1 AS c statement ok CREATE MATERIALIZED VIEW mv AS SELECT * FROM v WHERE c IS NULL statement error cannot drop view "v": still depended upon by materialized view "mv" DROP VIEW v statement ok DROP VIEW v CASCADE query I SELECT count(*) FROM mz_materialized_views WHERE name = 'mv' ---- 0 # mz_scheduling_elapsed_raw, a log source, is optimized away, but should still count as a dependency query error db error: ERROR: materialized view objects cannot depend on log sources CREATE MATERIALIZED VIEW mv AS SELECT (SELECT 1 FROM mz_introspection.mz_scheduling_elapsed_raw WHERE FALSE); simple conn=mz_system,user=mz_system ALTER SYSTEM SET unsafe_enable_table_keys = true ---- COMPLETE 0 statement ok CREATE TABLE t1 (f1 INTEGER NOT NULL PRIMARY KEY); statement ok CREATE TABLE t2 (f1 INTEGER NOT NULL PRIMARY KEY); # Folds to Constant, t1 is optimized away but must still be counted as a dependency statement ok CREATE MATERIALIZED VIEW mv AS SELECT * FROM t1 WHERE FALSE; statement error db error: ERROR: cannot drop table "t1": still depended upon by materialized view "mv" DROP TABLE t1 statement ok DROP MATERIALIZED VIEW mv # In the cases below, t2 is optimized away but should still be present as a dependency statement ok CREATE MATERIALIZED VIEW mv AS SELECT t1.* FROM t1 LEFT JOIN t2 ON (t1.f1 = t2.f1); statement error db error: ERROR: cannot drop table "t2": still depended upon by materialized view "mv" DROP TABLE t2 statement ok DROP MATERIALIZED VIEW mv statement ok CREATE MATERIALIZED VIEW mv AS SELECT * FROM t1 WHERE FALSE AND EXISTS (SELECT * FROM t2); statement error db error: ERROR: cannot drop table "t2": still depended upon by materialized view "mv" DROP TABLE t2 statement ok DROP MATERIALIZED VIEW mv statement ok CREATE MATERIALIZED VIEW mv AS SELECT * FROM t1 WHERE TRUE OR EXISTS (SELECT * FROM t2); statement error db error: ERROR: cannot drop table "t2": still depended upon by materialized view "mv" DROP TABLE t2 statement ok DROP MATERIALIZED VIEW mv statement ok CREATE MATERIALIZED VIEW mv AS SELECT (SELECT f1 FROM t2 WHERE FALSE) FROM t1; statement error db error: ERROR: cannot drop table "t2": still depended upon by materialized view "mv" DROP TABLE t2 statement ok DROP MATERIALIZED VIEW mv # No need to evaluate second argument of COALESCE if first is non-null statement ok CREATE MATERIALIZED VIEW mv AS SELECT COALESCE(1, (SELECT * FROM t2 LIMIT 1)) FROM t1; statement error db error: ERROR: cannot drop table "t2": still depended upon by materialized view "mv" DROP TABLE t2 statement ok DROP MATERIALIZED VIEW mv statement ok CREATE TYPE int4_list AS LIST (ELEMENT TYPE = int4); # Mention of int4_list is optimized away statement ok CREATE MATERIALIZED VIEW mv AS SELECT * FROM t1 WHERE NULL::int4_list IS NOT NULL; statement error db error: ERROR: cannot drop type "int4_list": still depended upon by materialized view "mv" DROP TYPE int4_list statement ok DROP MATERIALIZED VIEW mv statement ok DROP TYPE int4_list statement ok DROP TABLE t1 statement ok DROP TABLE t2 # Test: Materialized view prevents dropping its cluster. statement ok CREATE CLUSTER to_drop REPLICAS () statement ok CREATE MATERIALIZED VIEW to_drop_mv IN CLUSTER to_drop AS SELECT 1 statement error cannot drop cluster "to_drop" because other objects depend on it DROP CLUSTER to_drop # Test: Cluster with dependent materialized view can be dropped with CASCADE. statement ok DROP CLUSTER to_drop CASCADE query error unknown catalog item 'to_drop_mv' SELECT * FROM to_drop_mv # Test: SHOW CREATE MATERIALIZED VIEW statement ok CREATE MATERIALIZED VIEW mv AS SELECT 1 query TT colnames SHOW CREATE MATERIALIZED VIEW mv ---- name create_sql materialize.public.mv CREATE␠MATERIALIZED␠VIEW␠materialize.public.mv⏎␠␠␠␠IN␠CLUSTER␠quickstart⏎␠␠␠␠WITH␠(REFRESH␠=␠ON␠COMMIT)⏎␠␠␠␠AS␠SELECT␠1; # Test: SHOW CREATE MATERIALIZED VIEW as mz_support simple multiline,conn=mz_catalog_server,user=mz_support SELECT create_sql FROM (SHOW CREATE MATERIALIZED VIEW mv); ---- CREATE MATERIALIZED VIEW materialize.public.mv IN CLUSTER quickstart WITH (REFRESH = ON COMMIT) AS SELECT 1; EOF COMPLETE 1 # Test: SHOW MATERIALIZED VIEWS statement ok CREATE MATERIALIZED VIEW other_mv IN CLUSTER other AS SELECT 1 query TTT colnames,rowsort SHOW MATERIALIZED VIEWS ---- name cluster comment mv quickstart (empty) other_mv other (empty) query TTT colnames,rowsort SHOW MATERIALIZED VIEWS IN CLUSTER other ---- name cluster comment other_mv other (empty) statement ok DROP MATERIALIZED VIEW other_mv # Test: Materialized view can be renamed. statement ok ALTER MATERIALIZED VIEW mv RENAME TO mv2 query I SELECT * FROM mv2 ---- 1 statement ok DROP MATERIALIZED VIEW mv2 # Test: Materialized views show up in mz_materialized_views. statement ok CREATE MATERIALIZED VIEW mv AS SELECT 1 query TT colnames SELECT name, definition FROM mz_materialized_views ---- name definition mv SELECT␠1; statement ok DROP MATERIALIZED VIEW mv query I SELECT count(*) FROM mz_materialized_views ---- 0 # Test: Materialized views show in `SHOW OBJECTS`. statement ok CREATE MATERIALIZED VIEW mv AS SELECT 1 mode standard query TTT colnames,rowsort SHOW OBJECTS ---- name type comment mv materialized-view (empty) t table (empty) mode cockroach # Test: Indexes on materialized views show in `SHOW INDEXES`. statement ok CREATE DEFAULT INDEX ON mv query TTTTT colnames SHOW INDEXES ON mv ---- name on cluster key comment mv_primary_idx mv quickstart {?column?} (empty) # Test: Creating materialized views that depend on log sources is forbidden. statement error materialized view objects cannot depend on log sources CREATE OR REPLACE MATERIALIZED VIEW mv AS SELECT id, name FROM mz_introspection.mz_dataflow_operators; # Test: Attempting to use view commands on materialized views gives helpful errors. statement error mv is not a view\nHINT: Use DROP MATERIALIZED VIEW to remove a materialized view\. DROP VIEW mv statement error mv is not a view\nHINT: Use SHOW CREATE MATERIALIZED VIEW to show a materialized view\. SHOW CREATE VIEW mv statement error mv is not a view\nHINT: Use ALTER MATERIALIZED VIEW to rename a materialized view\. ALTER VIEW mv RENAME TO mv2 # We should not be able to create materialized views on top of 'SHOW' commands. statement error SHOW commands are not allowed in views CREATE MATERIALIZED VIEW mat_clusters AS SELECT name FROM (SHOW CLUSTERS); statement error SHOW commands are not allowed in views CREATE MATERIALIZED VIEW mat_cluster_replicas AS SELECT cluster, replica, size, ready FROM (SHOW CLUSTER REPLICAS); statement error SHOW commands are not allowed in views CREATE MATERIALIZED VIEW mat_columns AS SELECT name, nullable, type FROM (SHOW COLUMNS FROM mz_tables); statement error SHOW commands are not allowed in views CREATE MATERIALIZED VIEW mat_connections AS SELECT name, type FROM (SHOW CONNECTIONS); statement error SHOW commands are not allowed in views CREATE MATERIALIZED VIEW mat_databases AS SELECT name FROM (SHOW DATABASES); statement error SHOW commands are not allowed in views CREATE MATERIALIZED VIEW mat_indexes AS SELECT name, on, cluster, key FROM (SHOW INDEXES); statement error SHOW commands are not allowed in views CREATE MATERIALIZED VIEW mat_mat_views AS SELECT name, cluster FROM (SHOW MATERIALIZED VIEWS); statement error SHOW commands are not allowed in views CREATE MATERIALIZED VIEW mat_objects AS SELECT name FROM (SHOW OBJECTS); statement error SHOW commands are not allowed in views CREATE MATERIALIZED VIEW mat_schemas AS SELECT name FROM (SHOW SCHEMAS); statement error SHOW commands are not allowed in views CREATE MATERIALIZED VIEW mat_secrets AS SELECT name FROM (SHOW SECRETS); statement error SHOW commands are not allowed in views CREATE MATERIALIZED VIEW mat_sinks AS SELECT name, type FROM (SHOW SINKS); statement error SHOW commands are not allowed in views CREATE MATERIALIZED VIEW mat_sources AS SELECT name, type FROM (SHOW SOURCES); statement error SHOW commands are not allowed in views CREATE MATERIALIZED VIEW mat_tables AS SELECT name FROM (SHOW TABLES); statement error SHOW commands are not allowed in views CREATE MATERIALIZED VIEW mat_views AS SELECT name FROM (SHOW VIEWS); # LIMIT in materialized view statement ok CREATE MATERIALIZED VIEW mv_limited AS SELECT a FROM t ORDER BY a DESC, a+b LIMIT 3; query I SELECT * FROM mv_limited; ---- 3 5 7 statement ok DELETE FROM t WHERE a = 5; query I SELECT * FROM mv_limited; ---- 3 7 query I SELECT * FROM mv_limited ORDER BY a LIMIT 1; ---- 3 # Cleanup statement ok DROP TABLE t CASCADE statement ok DROP CLUSTER other CASCADE statement ok CREATE TABLE t2 (x int, y int, z int); statement ok INSERT INTO t2 VALUES (NULL, 2, 3), (4, NULL, 6), (7, 8, NULL); statement ok CREATE MATERIALIZED VIEW mv_no_assertions AS SELECT * FROM t2; query III SELECT * FROM mv_no_assertions ORDER BY x; ---- 4 NULL 6 7 8 NULL NULL 2 3 statement ok CREATE MATERIALIZED VIEW mv_assertion_at_begin WITH (ASSERT NOT NULL x) AS SELECT * FROM t2; statement error column "x" must not be null SELECT * FROM mv_assertion_at_begin; statement ok CREATE MATERIALIZED VIEW mv_assertion_at_end WITH (ASSERT NOT NULL z) AS SELECT * FROM t2; statement error column "z" must not be null SELECT * FROM mv_assertion_at_end; statement ok CREATE MATERIALIZED VIEW mv_two_assertions WITH (ASSERT NOT NULL x, ASSERT NOT NULL z) AS SELECT * FROM t2; statement error column "x" must not be null SELECT * FROM mv_two_assertions; statement ok CREATE MATERIALIZED VIEW mv_misordered_assertions WITH (ASSERT NOT NULL z, ASSERT NOT NULL y) AS SELECT * FROM t2; statement error must not be null SELECT * FROM mv_misordered_assertions statement error duplicate column "y" in non-null assertions CREATE MATERIALIZED VIEW mv_duplicate_assertions WITH (ASSERT NOT NULL y, ASSERT NOT NULL y) AS SELECT * FROM t2; statement error column "x" in ASSERT NOT NULL option not found CREATE MATERIALIZED VIEW mv_bad_assertion_on_renamed_column (a, b, c) WITH (ASSERT NOT NULL x) AS SELECT * FROM t2; statement ok CREATE MATERIALIZED VIEW mv_good_assertion_on_renamed_column (a, b, c) WITH (ASSERT NOT NULL b) AS SELECT * FROM t2; statement error column "b" must not be null SELECT * FROM mv_good_assertion_on_renamed_column; statement ok UPDATE t2 SET x=1 WHERE x IS NULL; query III SELECT * FROM mv_assertion_at_begin ORDER BY x; ---- 1 2 3 4 NULL 6 7 8 NULL # ------------------------------------------------------------------ # REFRESH options (see also in materialized-view-refresh-options.td) # ------------------------------------------------------------------ # Planning/parsing errors simple conn=mz_system,user=mz_system ALTER SYSTEM SET enable_refresh_every_mvs = false ---- COMPLETE 0 # Should be disabled query error db error: ERROR: REFRESH EVERY and REFRESH AT materialized views is not available CREATE MATERIALIZED VIEW mv_bad WITH (ASSERT NOT NULL x, REFRESH EVERY '8 seconds') AS SELECT * FROM t2; simple conn=mz_system,user=mz_system ALTER SYSTEM SET enable_refresh_every_mvs = true ---- COMPLETE 0 query error Expected one of ON or AT or EVERY, found number "5" CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH 5) AS SELECT 1; query error db error: ERROR: REFRESH ON COMMIT cannot be specified multiple times CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH ON COMMIT, REFRESH ON COMMIT) AS SELECT 1; query error db error: ERROR: REFRESH ON COMMIT is not compatible with any of the other REFRESH options CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH ON COMMIT, REFRESH EVERY '1 day') AS SELECT 1; query error db error: ERROR: REFRESH AT does not support casting from record\(f1: integer,f2: integer\) to mz_timestamp CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH AT row(1,2)) AS SELECT 1; query error db error: ERROR: REFRESH AT argument must be an expression that can be simplified and/or cast to a constant whose type is mz_timestamp CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH AT 'aaaa') AS SELECT 1; query error db error: ERROR: column "ccc" does not exist CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH AT ccc) AS SELECT 1 as ccc; query error db error: ERROR: REFRESH AT argument must be an expression that can be simplified and/or cast to a constant whose type is mz_timestamp CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH AT now()) AS SELECT 1; query error db error: ERROR: REFRESH AT argument must be an expression that can be simplified and/or cast to a constant whose type is mz_timestamp CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH AT now()::mz_timestamp) AS SELECT 1; query error db error: ERROR: greatest types mz_timestamp and timestamp with time zone cannot be matched CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH AT greatest(mz_now(), now())) AS SELECT 1; query error db error: ERROR: REFRESH AT argument must be an expression that can be simplified and/or cast to a constant whose type is mz_timestamp CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH AT greatest(mz_now(), now()::mz_timestamp)) AS SELECT 1; query error db error: ERROR: aggregate functions are not allowed in REFRESH AT \(function pg_catalog\.sum\) CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH AT sum(5)) AS SELECT 1; query error db error: ERROR: REFRESH AT does not allow subqueries CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH AT (SELECT 1)) AS SELECT 1; query error db error: ERROR: window functions are not allowed in REFRESH AT \(function pg_catalog\.lag\) CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH AT lag(7) OVER ()) AS SELECT 1; query error Expected literal string, found number "42" CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH EVERY 42) AS SELECT 1; query error db error: ERROR: invalid input syntax for type interval: unknown units dayy: "1 dayy" CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH EVERY '1 dayy') AS SELECT 1; query error Expected literal string, found INTERVAL CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH EVERY INTERVAL '1 day') AS SELECT 1; query error db error: ERROR: REFRESH interval must be positive; got: \-00:01:00 CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH EVERY '-1 minutes') AS SELECT 1; query error db error: ERROR: REFRESH interval must not involve units larger than days CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH EVERY '1 month') AS SELECT 1; query error db error: ERROR: REFRESH interval must not involve units larger than days CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH EVERY '1 year') AS SELECT 1; query error db error: ERROR: REFRESH EVERY \.\.\. ALIGNED TO argument must be an expression that can be simplified and/or cast to a constant whose type is mz_timestamp CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH EVERY '1 day' ALIGNED TO now()) AS SELECT 1; query error db error: ERROR: REFRESH EVERY \.\.\. ALIGNED TO argument must be an expression that can be simplified and/or cast to a constant whose type is mz_timestamp CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH EVERY '1 day' ALIGNED TO now()::mz_timestamp) AS SELECT 1; query error db error: ERROR: greatest types mz_timestamp and timestamp with time zone cannot be matched CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH EVERY '1 day' ALIGNED TO greatest(mz_now(), now())) AS SELECT 1; query error db error: ERROR: REFRESH EVERY \.\.\. ALIGNED TO argument must be an expression that can be simplified and/or cast to a constant whose type is mz_timestamp CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH EVERY '1 day' ALIGNED TO greatest(mz_now(), now()::mz_timestamp)) AS SELECT 1; query error db error: ERROR: aggregate functions are not allowed in REFRESH EVERY \.\.\. ALIGNED TO \(function pg_catalog\.sum\) CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH EVERY '1 day' ALIGNED TO sum(5)) AS SELECT 1; query error db error: ERROR: REFRESH EVERY \.\.\. ALIGNED TO does not allow subqueries CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH EVERY '1 day' ALIGNED TO (SELECT 1)) AS SELECT 1; query error db error: ERROR: window functions are not allowed in REFRESH EVERY \.\.\. ALIGNED TO \(function pg_catalog\.lag\) CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH EVERY '1 day' ALIGNED TO lag(7) OVER ()) AS SELECT 1; query error Expected literal string, found right parenthesis CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH EVERY) AS SELECT * FROM t2; query error Expected literal string, found comma CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH EVERY, ASSERT NOT NULL x) AS SELECT * FROM t2; query error Expected right parenthesis, found REFRESH CREATE MATERIALIZED VIEW mv_bad WITH (ASSERT NOT NULL x REFRESH EVERY '8 seconds') AS SELECT * FROM t2; query error Expected literal string, found ASSERT CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH EVERY ASSERT NOT NULL x) AS SELECT * FROM t2; query error db error: ERROR: invalid input syntax for type interval: Overflows maximum days; cannot exceed 2147483647/\-2147483648 days: "213503982001" CREATE MATERIALIZED VIEW mv_bad WITH (REFRESH EVERY '213503982001' days) AS SELECT * FROM t2; # This tests that we don't forget to purify EXPLAIN CREATE MATERIALIZED VIEW statement ok EXPLAIN OPTIMIZED PLAN WITH (humanized expressions) AS VERBOSE TEXT FOR CREATE MATERIALIZED VIEW mv_explain WITH (REFRESH EVERY '2 seconds') AS SELECT * FROM t2; statement ok CREATE MATERIALIZED VIEW mv_on_commit WITH (REFRESH ON COMMIT) AS SELECT * FROM t2; query III rowsort SELECT 1000*x, 1000*y, 1000*z FROM mv_on_commit; ---- 7000 8000 NULL 4000 NULL 6000 1000 2000 3000 # Test that we call `transform_ast::transform`. (This has an `Expr::Nested`, which needs to be desugared, or we panic.) statement ok CREATE MATERIALIZED VIEW mv_desugar1 WITH (REFRESH AT (mz_now())) AS SELECT * FROM t2; # Same with ALIGNED TO statement ok CREATE MATERIALIZED VIEW mv_desugar2 WITH (REFRESH EVERY '1 day' ALIGNED TO (mz_now())) AS SELECT * FROM t2; ## REFRESH options together with ASSERT NOT NULL options statement ok INSERT INTO t2 VALUES (10, 11, 12); statement ok CREATE MATERIALIZED VIEW mv_assertion_plus_refresh_every WITH (ASSERT NOT NULL x, REFRESH EVERY '8 seconds') AS SELECT * FROM t2; # There should be a refresh immediately when creating the MV. This refresh should already see what we just inserted. query III SELECT * FROM mv_assertion_plus_refresh_every ORDER BY x; ---- 1 2 3 4 NULL 6 7 8 NULL 10 11 12 statement ok INSERT INTO t2 VALUES (NULL, -1, -2); # This insert shouldn't be visible yet. query III SELECT * FROM mv_assertion_plus_refresh_every ORDER BY x; ---- 1 2 3 4 NULL 6 7 8 NULL 10 11 12 # Sleep for the refresh interval, so that we get a refresh. # Actually, we sleep a bit more than the refresh interval, because we don't have real-time recency guarantees: # When we query the MV at a particular wall clock time `t`, there is no guarantee that we see a refresh that # happened at, say, `t - 1ms`. Note that the test was actually failing sometimes when it was sleeping for only 8s or 9s. # A proper solution might be to add `AS OF now()` to the following SELECT, but calling `now()` seems to not be currently # allowed in `AS OF`. statement ok SELECT mz_unsafe.mz_sleep(8+2); # Now we should see the NULL that should error out the MV. query error db error: ERROR: Evaluation error: column "x" must not be null SELECT * FROM mv_assertion_plus_refresh_every ORDER BY x; ## Check `REFRESH AT greatest(, mz_now())`, because this is an idiom that we are recommending to users. # Insert something into the underlying table. statement ok INSERT INTO t2 VALUES (30, 30, 30); statement ok CREATE MATERIALIZED VIEW mv_greatest WITH (REFRESH AT greatest('1990-01-04 11:00', mz_now())) AS SELECT * FROM t2; query III rowsort SELECT * FROM mv_greatest; ---- NULL -1 -2 4 NULL 6 7 8 NULL 1 2 3 10 11 12 30 30 30 ## If there is no creation refresh, then a query should block until the first refresh. # Save the current time. statement ok CREATE TABLE start_time(t timestamp); statement ok INSERT INTO start_time VALUES (now()); # Create an MV whose first refresh is 5 sec after its creation. statement ok CREATE MATERIALIZED VIEW mv_no_creation_refresh WITH (REFRESH EVERY '100000 sec' ALIGNED TO mz_now()::string::int8 + 5000) AS SELECT * FROM t2; # Insert something into the underlying table. statement ok INSERT INTO t2 VALUES (100, 100, 100); # Query it. # - The query should block until the first refresh. # - The newly inserted stuff should be visible. query III rowsort SELECT * FROM mv_no_creation_refresh; ---- NULL -1 -2 4 NULL 6 7 8 NULL 1 2 3 10 11 12 30 30 30 100 100 100 # Verify that at least 5 seconds have passed. query B SELECT now() - (SELECT * from start_time) >= INTERVAL '5 sec'; ---- true ## Check ALIGNED TO in the far past # Save the current time. statement ok DELETE FROM start_time; statement ok INSERT INTO start_time VALUES (now()); statement ok CREATE MATERIALIZED VIEW mv_aligned_to_past WITH (REFRESH EVERY '10000 ms' ALIGNED TO mz_now()::text::int8 - 10*10000 + 3000) AS SELECT * FROM t2; query III rowsort SELECT * FROM mv_aligned_to_past; ---- NULL -1 -2 4 NULL 6 7 8 NULL 1 2 3 10 11 12 30 30 30 100 100 100 # Verify that at least 3 seconds have passed. query B SELECT now() - (SELECT * from start_time) >= INTERVAL '3 sec'; ---- true ## Check ALIGNED TO in the far future # Save the current time. statement ok DELETE FROM start_time; statement ok INSERT INTO start_time VALUES (now()); statement ok CREATE MATERIALIZED VIEW mv_aligned_to_future WITH (REFRESH EVERY '10000 ms' ALIGNED TO mz_now()::text::int8 + 10*10000 + 3000) AS SELECT * FROM t2; query III rowsort SELECT * FROM mv_aligned_to_future; ---- NULL -1 -2 4 NULL 6 7 8 NULL 1 2 3 10 11 12 30 30 30 100 100 100 # Verify that at least 3 seconds have passed. query B SELECT now() - (SELECT * from start_time) >= INTERVAL '3 sec'; ---- true ## Constant query in an MV with REFRESH options statement ok CREATE MATERIALIZED VIEW const_mv WITH (REFRESH EVERY '1 day') AS SELECT 1; query I SELECT * FROM const_mv ---- 1 ## We should be able to immediately query a constant MV under serializable isolation even if the ## first refresh is in the future. The since will be advanced to [3000-01-01 23:59] and the upper ## will be advanced to []. statement ok CREATE MATERIALIZED VIEW const_mv2 WITH (REFRESH AT '3000-01-01 23:59') AS SELECT 2; statement ok SET transaction_isolation = 'serializable' query I SELECT * FROM const_mv2 ---- 2 statement ok SET transaction_isolation = 'strict serializable' ## MV that has refreshes only in the past query error db error: ERROR: REFRESH AT requested for a time where not all the inputs are readable CREATE MATERIALIZED VIEW mv_no_refresh WITH (REFRESH AT '2000-01-01 10:00') AS SELECT * FROM t2; ## Query MV after the last refresh statement ok CREATE MATERIALIZED VIEW mv3 WITH (REFRESH AT mz_now()::text::int8 + 2000, REFRESH AT mz_now()::text::int8 + 4000) AS SELECT * FROM t2; # Wait until the first refresh query III rowsort SELECT * FROM mv3; ---- NULL -1 -2 4 NULL 6 7 8 NULL 1 2 3 10 11 12 30 30 30 100 100 100 # Wait until we are past the second refresh, which is the last one. # (See the explanation for the `+2` above at a similar `mz_sleep`.) statement ok SELECT mz_unsafe.mz_sleep(2+2); # This insert will happen after the last refresh. statement ok INSERT INTO t2 VALUES (70, 70, 70); # We should be able to query the MV after the last refresh, and the newly inserted data shouldn't be visible. query III rowsort SELECT * FROM mv3; ---- NULL -1 -2 4 NULL 6 7 8 NULL 1 2 3 10 11 12 30 30 30 100 100 100 # Regression test for https://github.com/MaterializeInc/database-issues/issues/7265 # The sleep makes _optimization_ take a few seconds, so we need to grab read holds in purification right away after # choosing a timestamp for mz_now. simple conn=mz_system,user=mz_system ALTER SYSTEM SET unsafe_enable_unstable_dependencies = true ---- COMPLETE 0 statement ok create materialized view mv4 with (refresh at creation) as select * from (select mz_unsafe.mz_sleep(3)), (select * from t2); # EXPLAIN and EXPLAIN TIMESTAMP on an MV that hasn't had its first refresh yet shouldn't block # See also `test_explain_timestamp_blocking` statement ok CREATE MATERIALIZED VIEW mv5 WITH (REFRESH AT mz_now()::text::int8 + 1000000) AS SELECT x+y+z from t2; statement ok EXPLAIN OPTIMIZED PLAN WITH (humanized expressions) AS VERBOSE TEXT FOR SELECT * FROM mv5; statement ok EXPLAIN TIMESTAMP FOR SELECT * FROM mv5; ## Stacked REFRESH MVs -- both have their first and only refresh in the future, at similar times statement ok CREATE MATERIALIZED VIEW mv6 WITH (REFRESH AT mz_now()::text::int8 + 3000) AS SELECT x-y+z from t2; statement ok CREATE MATERIALIZED VIEW mv7 WITH (REFRESH AT mz_now()::text::int8 + 3000) AS SELECT * from mv6; query I rowsort SELECT * FROM mv7 ---- NULL NULL NULL 2 11 30 70 100 # We now insert something into the underlying table. This shouldn't be visible in mv6, because mv6's refresh shouldn't # be later than mv7's refresh. statement ok INSERT INTO t2 VALUES (0, 0, 0); query I rowsort SELECT * FROM mv6 ---- NULL NULL NULL 2 11 30 70 100 ## REFRESH AT CREATION + REFRESH EVERY ALIGNED TO ## We create an MV that would be first refreshed only much later, if it were not for the REFRESH AT CREATION. statement ok CREATE MATERIALIZED VIEW mv_multiple_refresh_options WITH ( REFRESH AT CREATION, REFRESH EVERY '1 day' ALIGNED TO mz_now()::text::int8 + 1000000 ) AS SELECT DISTINCT 5*x FROM t2; # Should return quickly due to the creation refresh. query I rowsort SELECT * FROM mv_multiple_refresh_options; ---- NULL 0 5 20 35 50 150 350 500 ## EXPLAIN shouldn't try to grab read holds in `create_materialized_view_validate` statement ok CREATE TABLE t3(x int); statement ok INSERT INTO t3 VALUES (5), (6); statement ok CREATE MATERIALIZED VIEW mv8 WITH (REFRESH AT CREATION) AS SELECT DISTINCT x-x FROM t3; # Sleep until we could no longer grab read holds at the original creation time. statement ok SELECT mz_unsafe.mz_sleep(2); # This would fail to get read holds if it attempted to do so. statement ok EXPLAIN REPLAN MATERIALIZED VIEW mv8; ## Indexes on REFRESH MVs statement ok CREATE MATERIALIZED VIEW mvi1 WITH (REFRESH EVERY '8s' ALIGNED TO mz_now()::string::int8 + 2000) AS SELECT 5*x FROM t3; statement ok CREATE DEFAULT INDEX on mvi1; query I SELECT * FROM mvi1; ---- 25 30 query I (SELECT * FROM mvi1) UNION ALL (SELECT * FROM t3); ---- 5 6 25 30 statement ok INSERT INTO t3 values (7); # Not visible yet. query I SELECT * FROM mvi1; ---- 25 30 query I (SELECT * FROM mvi1) UNION ALL (SELECT * FROM t3); ---- 5 6 7 25 30 statement ok SELECT mz_unsafe.mz_sleep(8+2); # Visible now. query I rowsort SELECT * FROM mvi1; ---- 25 30 35 query I (SELECT * FROM mvi1) UNION ALL (SELECT * FROM t3); ---- 5 6 7 25 30 35 # First refresh immediately, next one much later. statement ok CREATE MATERIALIZED VIEW mvi2 WITH (REFRESH EVERY '10 hours') AS SELECT DISTINCT x+x+x FROM t3; statement ok CREATE DEFAULT INDEX on mvi2; query I rowsort SELECT * FROM mvi2; ---- 15 18 21 query I (SELECT * FROM mvi2) UNION ALL (SELECT * FROM t3); ---- 5 6 7 15 18 21 # There is a last refresh. statement ok CREATE MATERIALIZED VIEW mvi3 WITH (REFRESH AT CREATION, REFRESH AT mz_now()::string::int8 + 2000) AS SELECT DISTINCT 5*x FROM t3; statement ok CREATE DEFAULT INDEX ON mvi3; query I rowsort SELECT * FROM mvi3 ---- 25 30 35 query I (SELECT * FROM mvi3) UNION ALL (SELECT * FROM t3); ---- 5 6 7 25 30 35 # Check that it's still queryable after the last refresh, but new input changes are not taken into account. statement ok SELECT mz_unsafe.mz_sleep(2+2); statement ok INSERT INTO t3 VALUES (-1); query I rowsort SELECT * FROM mvi3 ---- 25 30 35 query I (SELECT * FROM mvi3) UNION ALL (SELECT * FROM t3); ---- -1 5 6 7 25 30 35 # ---------------------------------------- # Automated cluster scheduling for REFRESH # ---------------------------------------- simple conn=mz_system,user=mz_system ALTER SYSTEM SET enable_cluster_schedule_refresh = false ---- COMPLETE 0 # Should be disabled query error db error: ERROR: `SCHEDULE = ON REFRESH` cluster option is not available CREATE CLUSTER c_schedule_0 (SIZE = '1', SCHEDULE = ON REFRESH); simple conn=mz_system,user=mz_system ALTER SYSTEM SET enable_cluster_schedule_refresh = true ---- COMPLETE 0 # Let's not complicate things with `cluster_refresh_mv_compaction_estimate` at first. simple conn=mz_system,user=mz_system ALTER SYSTEM SET cluster_refresh_mv_compaction_estimate = 0 ---- COMPLETE 0 statement error db error: ERROR: Expected one of MANUAL or ON, found identifier "aaaaaaaa" CREATE CLUSTER c_schedule_0 (SIZE = '1', SCHEDULE = AAAAAAAA); statement error db error: ERROR: Expected one of MANUAL or ON, found number "42" CREATE CLUSTER c_schedule_0 (SIZE = '1', SCHEDULE = 42); statement error db error: ERROR: Expected one of MANUAL or ON, found REFRESH CREATE CLUSTER c_schedule_0 (SIZE = '1', SCHEDULE = REFRESH); statement error db error: ERROR: REPLICATION FACTOR cannot be given together with any SCHEDULE other than MANUAL CREATE CLUSTER c_schedule_0 (SIZE = '1', SCHEDULE = ON REFRESH, REPLICATION FACTOR = 1); statement ok CREATE CLUSTER c_schedule_1 (SIZE = '1', SCHEDULE = MANUAL); statement ok CREATE CLUSTER c_schedule_2 (SIZE = '1', SCHEDULE MANUAL); statement ok CREATE CLUSTER c_schedule_3 (SIZE = '1', SCHEDULE = ON REFRESH); statement error db error: ERROR: REPLICATION FACTOR cannot be set if the cluster SCHEDULE is anything other than MANUAL ALTER CLUSTER c_schedule_3 SET (REPLICATION FACTOR = 1); statement ok ALTER CLUSTER c_schedule_1 RESET (SCHEDULE); statement error db error: ERROR: REPLICATION FACTOR cannot be given together with any SCHEDULE other than MANUAL ALTER CLUSTER c_schedule_1 SET (REPLICATION FACTOR = 1, SCHEDULE = ON REFRESH); statement ok ALTER CLUSTER c_schedule_1 SET (SCHEDULE = MANUAL); statement ok ALTER CLUSTER c_schedule_1 SET (SCHEDULE = ON REFRESH); statement ok SELECT mz_unsafe.mz_sleep(3+2); # Should turn off. query I SELECT replication_factor FROM mz_catalog.mz_clusters WHERE name = 'c_schedule_1'; ---- 0 statement error db error: ERROR: cluster schedules other than MANUAL are not supported for unmanaged clusters ALTER CLUSTER c_schedule_1 SET (MANAGED = false, SCHEDULE = ON REFRESH); # The SCHEDULE shouldn't simply "fall off" when switching to unmanaged, statement error db error: ERROR: when switching a cluster to unmanaged, if the managed cluster's SCHEDULE is anything other than MANUAL, you have to explicitly set the SCHEDULE to MANUAL ALTER CLUSTER c_schedule_1 SET (MANAGED = false); # ... but can be explicitly set to MANUAL in the same command. statement ok ALTER CLUSTER c_schedule_1 SET (MANAGED = false, SCHEDULE = MANUAL); statement ok ALTER CLUSTER c_schedule_1 SET (MANAGED = true, SIZE = '1'); statement ok ALTER CLUSTER c_schedule_1 SET (SCHEDULE = ON REFRESH (HYDRATION TIME ESTIMATE = '0 seconds')); # Setting some other cluster option in ALTER CLUSTER shouldn't change the SCHEDULE. # (The sleep is needed so that if the following ALTER erroneously sets the SCHEDULE to MANUAL, then we should be in a # turned off state at that moment to trigger errors in later tests.) statement ok SELECT mz_unsafe.mz_sleep(3); statement ok ALTER CLUSTER c_schedule_1 SET (INTROSPECTION DEBUGGING = TRUE); statement ok ALTER CLUSTER c_schedule_1 RESET (INTROSPECTION DEBUGGING); statement ok CREATE CLUSTER unmanaged1 (SCHEDULE = MANUAL, REPLICAS (r1 (SIZE '1'))) statement ok ALTER cluster unmanaged1 SET (SCHEDULE = MANUAL); statement error db error: ERROR: cluster schedules other than MANUAL are not supported for unmanaged clusters ALTER cluster unmanaged1 SET (SCHEDULE = ON REFRESH); statement error db error: ERROR: REPLICATION FACTOR cannot be given together with any SCHEDULE other than MANUAL ALTER cluster unmanaged1 SET (managed = true, SCHEDULE = ON REFRESH, REPLICATION FACTOR = 1, SIZE = '1'); statement error db error: ERROR: cluster schedules other than MANUAL are not supported for unmanaged clusters CREATE CLUSTER unmanaged2 (SCHEDULE = ON REFRESH, REPLICAS (r1 (SIZE '1'))) statement ok CREATE CLUSTER c_schedule_5 (SIZE = '1'); statement error db error: ERROR: Expected one of OWNER or RENAME or RESET or SET or SWAP, found left parenthesis ALTER CLUSTER c_schedule_5 (MANAGED = false, SCHEDULE = REFRESH); statement ok CREATE MATERIALIZED VIEW mv9 IN CLUSTER c_schedule_1 WITH (REFRESH = EVERY '8 sec') AS SELECT sum(x*y*z) + count(*) FROM t2; query I SELECT * FROM mv9; ---- 1371335 statement ok INSERT INTO t2 VALUES (1, 0, 1); statement ok SELECT mz_unsafe.mz_sleep(8+2); query I SELECT * FROM mv9; ---- 1371336 statement ok CREATE MATERIALIZED VIEW mv10 IN CLUSTER c_schedule_1 WITH (REFRESH AT CREATION) AS SELECT count(*) FROM t2; query I SELECT * FROM mv10 ---- 10 # The other refresh cluster should be off, because there is no refresh MV on it yet. query I SELECT replication_factor FROM mz_catalog.mz_clusters WHERE name = 'c_schedule_3'; ---- 0 # A `REFRESH AT CREATION` MV alone on a cluster should make the cluster turn on. statement ok CREATE MATERIALIZED VIEW mv11 IN CLUSTER c_schedule_3 WITH (REFRESH AT CREATION) AS SELECT count(*) FROM t2; query I SELECT * FROM mv11 ---- 10 ## Very short refresh interval. statement ok CREATE CLUSTER c_schedule_4 (SIZE = '1', SCHEDULE = ON REFRESH); statement ok CREATE MATERIALIZED VIEW mv12 IN CLUSTER c_schedule_4 WITH (REFRESH EVERY '1 millisecond') AS SELECT count(*) FROM t2; query I SELECT * FROM mv12 ---- 10 statement ok INSERT INTO t2 VALUES (1, 1, 10); query I SELECT * FROM mv12 ---- 11 # This should set the schedule back to manual. statement ok ALTER CLUSTER c_schedule_4 RESET (SCHEDULE); statement ok ALTER CLUSTER c_schedule_4 SET (REPLICATION FACTOR = 0); statement ok SELECT mz_unsafe.mz_sleep(3); # Should stay off, because it was reset to manual. query I SELECT replication_factor FROM mz_catalog.mz_clusters WHERE name = 'c_schedule_4'; ---- 0 ## HYDRATION TIME ESTIMATE statement error db error: ERROR: Expected literal string, found number "0" CREATE CLUSTER c_bad (SIZE = '1', SCHEDULE = ON REFRESH (HYDRATION TIME ESTIMATE = 0)); statement error db error: ERROR: HYDRATION TIME ESTIMATE must be non\-negative; got: \-01:00:00 CREATE CLUSTER c_bad (SIZE = '1', SCHEDULE = ON REFRESH (HYDRATION TIME ESTIMATE = '-1 hour')); statement error db error: ERROR: invalid input syntax for type interval: unknown units aaaa: "1 aaaa" CREATE CLUSTER c_bad (SIZE = '1', SCHEDULE = ON REFRESH (HYDRATION TIME ESTIMATE = '1 aaaa')); statement error db error: ERROR: HYDRATION TIME ESTIMATE must not involve units larger than days CREATE CLUSTER c_bad (SIZE = '1', SCHEDULE = ON REFRESH (HYDRATION TIME ESTIMATE = '1 month')); statement error db error: ERROR: invalid input syntax for type interval: Overflows maximum days; cannot exceed 2147483647/\-2147483648 days: "1000000000000 days" CREATE CLUSTER c_bad (SIZE = '1', SCHEDULE = ON REFRESH (HYDRATION TIME ESTIMATE = '1000000000000 days')); # ---------------------------------------- # Introspection # ---------------------------------------- query TTTBB SELECT name, type, interval, now() - aligned_to < INTERVAL '30 minutes', now() - at < INTERVAL '30 minutes' FROM mz_internal.mz_materialized_view_refresh_strategies mvrs, mz_catalog.mz_materialized_views mv WHERE mv.id = mvrs.materialized_view_id ORDER BY name; ---- const_mv every 24:00:00 true NULL const_mv2 at NULL NULL true mv on-commit NULL NULL NULL mv10 at NULL NULL true mv11 at NULL NULL true mv12 every 00:00:00.001 true NULL mv3 at NULL NULL true mv3 at NULL NULL true mv4 at NULL NULL true mv5 at NULL NULL true mv6 at NULL NULL true mv7 at NULL NULL true mv8 at NULL NULL true mv9 every 00:00:08 true NULL mv_aligned_to_future every 00:00:10 true NULL mv_aligned_to_past every 00:00:10 true NULL mv_assertion_at_begin on-commit NULL NULL NULL mv_assertion_at_end on-commit NULL NULL NULL mv_assertion_plus_refresh_every every 00:00:08 true NULL mv_desugar1 at NULL NULL true mv_desugar2 every 24:00:00 true NULL mv_good_assertion_on_renamed_column on-commit NULL NULL NULL mv_greatest at NULL NULL true mv_misordered_assertions on-commit NULL NULL NULL mv_multiple_refresh_options at NULL NULL true mv_multiple_refresh_options every 24:00:00 true NULL mv_no_assertions on-commit NULL NULL NULL mv_no_creation_refresh every 27:46:40 true NULL mv_on_commit on-commit NULL NULL NULL mv_two_assertions on-commit NULL NULL NULL mvi1 every 00:00:08 true NULL mvi2 every 10:00:00 true NULL mvi3 at NULL NULL true mvi3 at NULL NULL true statement ok CREATE CLUSTER c_schedule_hydration_time_estimate (SIZE = '1', SCHEDULE = ON REFRESH (HYDRATION TIME ESTIMATE = '995 seconds')); # Make the above cluster turn on, so that we can test how the HYDRATION TIME ESTIMATE looks in `mz_audit_events`. statement ok CREATE MATERIALIZED VIEW mv_rte IN CLUSTER c_schedule_hydration_time_estimate WITH (REFRESH EVERY '1 sec') AS SELECT * FROM t2; query TTT SELECT name, cs.type, cs.refresh_hydration_time_estimate FROM mz_internal.mz_cluster_schedules cs, mz_catalog.mz_clusters c WHERE c.id = cs.cluster_id AND name LIKE 'c_schedule_%' ORDER BY name; ---- c_schedule_1 on-refresh 00:00:00 c_schedule_2 manual NULL c_schedule_3 on-refresh 00:00:00 c_schedule_4 manual NULL c_schedule_5 manual NULL c_schedule_hydration_time_estimate on-refresh 00:16:35 statement ok SELECT mz_unsafe.mz_sleep(4); query TTTTBT rowsort SELECT DISTINCT event_type, object_type, (details->'cluster_name')::text, (details->'reason')::text, (details->'scheduling_policies') IS NULL, regexp_replace((details->'scheduling_policies'->'on_refresh')::text, '\["u.*"\]', '["uXXX"]') FROM mz_audit_events WHERE event_type IN ('create', 'drop') AND object_type = 'cluster-replica' AND ((details->'cluster_name')::text LIKE '"c_schedule_%"' OR (details->'cluster_name')::text = '"other"'); ---- drop cluster-replica "other" "manual" true NULL create cluster-replica "other" "manual" true NULL drop cluster-replica "c_schedule_4" "manual" true NULL create cluster-replica "c_schedule_1" "manual" true NULL create cluster-replica "c_schedule_2" "manual" true NULL create cluster-replica "c_schedule_5" "manual" true NULL drop cluster-replica "c_schedule_1" "schedule" false {"decision":"off","hydration_time_estimate":"00:00:00","objects_needing_compaction":[],"objects_needing_refresh":[]} drop cluster-replica "c_schedule_3" "schedule" false {"decision":"off","hydration_time_estimate":"00:00:00","objects_needing_compaction":[],"objects_needing_refresh":[]} create cluster-replica "c_schedule_1" "schedule" false {"decision":"on","hydration_time_estimate":"00:00:00","objects_needing_compaction":[],"objects_needing_refresh":["uXXX"]} create cluster-replica "c_schedule_3" "schedule" false {"decision":"on","hydration_time_estimate":"00:00:00","objects_needing_compaction":[],"objects_needing_refresh":["uXXX"]} create cluster-replica "c_schedule_4" "schedule" false {"decision":"on","hydration_time_estimate":"00:00:00","objects_needing_compaction":[],"objects_needing_refresh":["uXXX"]} create cluster-replica "c_schedule_hydration_time_estimate" "schedule" false {"decision":"on","hydration_time_estimate":"00:16:35","objects_needing_compaction":[],"objects_needing_refresh":["uXXX"]} ## Now test `cluster_refresh_mv_compaction_estimate`. ## (This would make the above audit test flaky, so it should be after that.) simple conn=mz_system,user=mz_system ALTER SYSTEM SET cluster_refresh_mv_compaction_estimate = 1200000 ---- COMPLETE 0 statement ok CREATE CLUSTER c_schedule_6 (SIZE = '1', SCHEDULE = ON REFRESH); query I SELECT replication_factor FROM mz_catalog.mz_clusters WHERE name = 'c_schedule_6'; ---- 0 statement ok CREATE MATERIALIZED VIEW mv13 IN CLUSTER c_schedule_6 WITH (REFRESH AT CREATION) AS SELECT sum(x*y) - count(*) AS r FROM t2; # Wait until the first refresh is complete. query I SELECT r+r FROM mv13; ---- 31916 # We'd turn it off at the next scheduling decision if it were not for `cluster_refresh_mv_compaction_estimate` statement ok SELECT mz_unsafe.mz_sleep(3+1+1); query I SELECT replication_factor FROM mz_catalog.mz_clusters WHERE name = 'c_schedule_6'; ---- 1 simple conn=mz_system,user=mz_system ALTER SYSTEM SET cluster_refresh_mv_compaction_estimate = 0 ---- COMPLETE 0 # Should turn off at the next scheduling decision. statement ok SELECT mz_unsafe.mz_sleep(3+1+1); query I SELECT replication_factor FROM mz_catalog.mz_clusters WHERE name = 'c_schedule_6'; ---- 0 simple conn=mz_system,user=mz_system ALTER SYSTEM SET cluster_refresh_mv_compaction_estimate = 120000 ---- COMPLETE 0 # Should turn on at the next scheduling decision. statement ok SELECT mz_unsafe.mz_sleep(3+1+1); query I SELECT replication_factor FROM mz_catalog.mz_clusters WHERE name = 'c_schedule_6'; ---- 1 # The audit events should now have a row that has a non-empty `objects_needing_compaction`. query TTTTBT rowsort SELECT DISTINCT event_type, object_type, (details->'cluster_name')::text, (details->'reason')::text, (details->'scheduling_policies') IS NULL, regexp_replace((details->'scheduling_policies'->'on_refresh')::text, '\["u.*"\]', '["uXXX"]') FROM mz_audit_events WHERE event_type IN ('create', 'drop') AND object_type = 'cluster-replica' AND (details->'cluster_name')::text = '"c_schedule_6"'; ---- drop cluster-replica "c_schedule_6" "schedule" false {"decision":"off","hydration_time_estimate":"00:00:00","objects_needing_compaction":[],"objects_needing_refresh":[]} create cluster-replica "c_schedule_6" "schedule" false {"decision":"on","hydration_time_estimate":"00:00:00","objects_needing_compaction":["uXXX"],"objects_needing_refresh":[]} create cluster-replica "c_schedule_6" "schedule" false {"decision":"on","hydration_time_estimate":"00:00:00","objects_needing_compaction":[],"objects_needing_refresh":["uXXX"]} simple conn=mz_system,user=mz_system ALTER SYSTEM SET cluster_refresh_mv_compaction_estimate = 0 ---- COMPLETE 0 ## EXPLAIN FILTER PUSHDOWN can be run on materialized views in this file simple conn=mz_system,user=mz_system ALTER SYSTEM SET enable_explain_pushdown = true ---- COMPLETE 0 # Pulling stats for refresh-every and similar MVs can time out, # since data may not yet be available... statement ok SET statement_timeout = '1s' statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW const_mv; statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW const_mv2; statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mv; statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mv3; statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mv5; statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mv8; statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mv9; statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mv12; statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mv_aligned_to_future; statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mv_aligned_to_past; statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mv_assertion_at_begin; statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mv_assertion_at_end; statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mv_assertion_plus_refresh_every; statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mv_desugar1; statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mv_desugar2; statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mv_good_assertion_on_renamed_column; statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mv_greatest; statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mv_misordered_assertions; statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mv_multiple_refresh_options; statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mv_multiple_refresh_options; statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mv_no_assertions; statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mv_no_creation_refresh; statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mv_on_commit; statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mv_two_assertions; statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mvi1; statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mvi2; statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mvi3; statement ok EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mvi3; # Attempting to explain MVs which are not readable at the current time can block statement error db error: ERROR: canceling statement due to statement timeout EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mv4; statement error db error: ERROR: canceling statement due to statement timeout EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mv10; statement error db error: ERROR: canceling statement due to statement timeout EXPLAIN FILTER PUSHDOWN FOR MATERIALIZED VIEW mv11; simple conn=mz_system,user=mz_system ALTER SYSTEM SET enable_explain_pushdown = false ---- COMPLETE 0 query T multiline SELECT regexp_replace(create_sql, 'AT \d+', 'XXX', 'g') FROM (SHOW CREATE MATERIALIZED VIEW mvi3); ---- CREATE MATERIALIZED VIEW materialize.public.mvi3 IN CLUSTER quickstart WITH ( REFRESH = XXX::mz_catalog.mz_timestamp, REFRESH = XXX::mz_catalog.mz_timestamp::pg_catalog.text::pg_catalog.int8 + 2000 ) AS SELECT DISTINCT 5 * x FROM materialize.public.t3; EOF