Fix continuous_aggs info

This commit switches the remaining JOIN in the continuous_aggs_stats
view to LEFT JOIN. This way we'll still see info from the other columns
even when the background worker has not run yet.
This commit also switches the time fields to output text in the correct
format for the underlying time type.
This commit is contained in:
Joshua Lockerman 2019-04-19 09:43:04 -04:00 committed by Matvey Arye
parent 445895bfd7
commit ae3480c2cb
10 changed files with 263 additions and 24 deletions

View File

@ -11,3 +11,6 @@ CREATE OR REPLACE FUNCTION _timescaledb_internal.get_partition_hash(val anyeleme
RETURNS int RETURNS int
AS '@MODULE_PATHNAME@', 'ts_get_partition_hash' LANGUAGE C IMMUTABLE STRICT PARALLEL SAFE; AS '@MODULE_PATHNAME@', 'ts_get_partition_hash' LANGUAGE C IMMUTABLE STRICT PARALLEL SAFE;
CREATE OR REPLACE FUNCTION _timescaledb_internal.get_time_type(hypertable_id INTEGER)
RETURNS OID
AS '@MODULE_PATHNAME@', 'ts_hypertable_get_time_type' LANGUAGE C STABLE STRICT;

View File

@ -14,6 +14,7 @@ CREATE TABLE IF NOT EXISTS _timescaledb_catalog.continuous_agg (
refresh_lag BIGINT NOT NULL, refresh_lag BIGINT NOT NULL,
direct_view_schema NAME NOT NULL, direct_view_schema NAME NOT NULL,
direct_view_name NAME NOT NULL, direct_view_name NAME NOT NULL,
max_interval_per_job BIGINT NOT NULL,
UNIQUE(user_view_schema, user_view_name), UNIQUE(user_view_schema, user_view_name),
UNIQUE(partial_view_schema, partial_view_name) UNIQUE(partial_view_schema, partial_view_name)
); );

View File

@ -9,6 +9,16 @@ CREATE OR REPLACE FUNCTION _timescaledb_internal.to_unix_microseconds(ts TIMESTA
CREATE OR REPLACE FUNCTION _timescaledb_internal.to_timestamp(unixtime_us BIGINT) RETURNS TIMESTAMPTZ CREATE OR REPLACE FUNCTION _timescaledb_internal.to_timestamp(unixtime_us BIGINT) RETURNS TIMESTAMPTZ
AS '@MODULE_PATHNAME@', 'ts_pg_unix_microseconds_to_timestamp' LANGUAGE C IMMUTABLE STRICT PARALLEL SAFE; AS '@MODULE_PATHNAME@', 'ts_pg_unix_microseconds_to_timestamp' LANGUAGE C IMMUTABLE STRICT PARALLEL SAFE;
CREATE OR REPLACE FUNCTION _timescaledb_internal.to_timestamp_without_timezone(unixtime_us BIGINT)
RETURNS TIMESTAMP
AS '@MODULE_PATHNAME@', 'ts_pg_unix_microseconds_to_timestamp'
LANGUAGE C IMMUTABLE STRICT PARALLEL SAFE;
CREATE OR REPLACE FUNCTION _timescaledb_internal.to_date(unixtime_us BIGINT)
RETURNS DATE
AS '@MODULE_PATHNAME@', 'ts_pg_unix_microseconds_to_date'
LANGUAGE C IMMUTABLE STRICT PARALLEL SAFE;
CREATE OR REPLACE FUNCTION _timescaledb_internal.to_interval(unixtime_us BIGINT) RETURNS INTERVAL CREATE OR REPLACE FUNCTION _timescaledb_internal.to_interval(unixtime_us BIGINT) RETURNS INTERVAL
AS '@MODULE_PATHNAME@', 'ts_pg_unix_microseconds_to_interval' LANGUAGE C IMMUTABLE STRICT PARALLEL SAFE; AS '@MODULE_PATHNAME@', 'ts_pg_unix_microseconds_to_interval' LANGUAGE C IMMUTABLE STRICT PARALLEL SAFE;

View File

@ -29,25 +29,25 @@ CREATE OR REPLACE VIEW timescaledb_information.license AS
_timescaledb_internal.license_expiration_time() <= now() AS expired, _timescaledb_internal.license_expiration_time() <= now() AS expired,
_timescaledb_internal.license_expiration_time() AS expiration_time; _timescaledb_internal.license_expiration_time() AS expiration_time;
CREATE OR REPLACE VIEW timescaledb_information.drop_chunks_policies as CREATE OR REPLACE VIEW timescaledb_information.drop_chunks_policies as
SELECT format('%1$I.%2$I', ht.schema_name, ht.table_name)::regclass as hypertable, p.older_than, p.cascade, p.job_id, j.schedule_interval, SELECT format('%1$I.%2$I', ht.schema_name, ht.table_name)::regclass as hypertable, p.older_than, p.cascade, p.job_id, j.schedule_interval,
j.max_runtime, j.max_retries, j.retry_period j.max_runtime, j.max_retries, j.retry_period
FROM _timescaledb_config.bgw_policy_drop_chunks p FROM _timescaledb_config.bgw_policy_drop_chunks p
INNER JOIN _timescaledb_catalog.hypertable ht ON p.hypertable_id = ht.id INNER JOIN _timescaledb_catalog.hypertable ht ON p.hypertable_id = ht.id
INNER JOIN _timescaledb_config.bgw_job j ON p.job_id = j.id; INNER JOIN _timescaledb_config.bgw_job j ON p.job_id = j.id;
CREATE OR REPLACE VIEW timescaledb_information.reorder_policies as CREATE OR REPLACE VIEW timescaledb_information.reorder_policies as
SELECT format('%1$I.%2$I', ht.schema_name, ht.table_name)::regclass as hypertable, p.hypertable_index_name, p.job_id, j.schedule_interval, SELECT format('%1$I.%2$I', ht.schema_name, ht.table_name)::regclass as hypertable, p.hypertable_index_name, p.job_id, j.schedule_interval,
j.max_runtime, j.max_retries, j.retry_period j.max_runtime, j.max_retries, j.retry_period
FROM _timescaledb_config.bgw_policy_reorder p FROM _timescaledb_config.bgw_policy_reorder p
INNER JOIN _timescaledb_catalog.hypertable ht ON p.hypertable_id = ht.id INNER JOIN _timescaledb_catalog.hypertable ht ON p.hypertable_id = ht.id
INNER JOIN _timescaledb_config.bgw_job j ON p.job_id = j.id; INNER JOIN _timescaledb_config.bgw_job j ON p.job_id = j.id;
CREATE OR REPLACE VIEW timescaledb_information.policy_stats as CREATE OR REPLACE VIEW timescaledb_information.policy_stats as
SELECT format('%1$I.%2$I', ht.schema_name, ht.table_name)::regclass as hypertable, p.job_id, j.job_type, js.last_run_success, js.last_finish, js.last_start, js.next_start, SELECT format('%1$I.%2$I', ht.schema_name, ht.table_name)::regclass as hypertable, p.job_id, j.job_type, js.last_run_success, js.last_finish, js.last_start, js.next_start,
js.total_runs, js.total_failures js.total_runs, js.total_failures
FROM (SELECT job_id, hypertable_id FROM _timescaledb_config.bgw_policy_reorder FROM (SELECT job_id, hypertable_id FROM _timescaledb_config.bgw_policy_reorder
UNION SELECT job_id, hypertable_id FROM _timescaledb_config.bgw_policy_drop_chunks) p UNION SELECT job_id, hypertable_id FROM _timescaledb_config.bgw_policy_drop_chunks) p
INNER JOIN _timescaledb_catalog.hypertable ht ON p.hypertable_id = ht.id INNER JOIN _timescaledb_catalog.hypertable ht ON p.hypertable_id = ht.id
INNER JOIN _timescaledb_config.bgw_job j ON p.job_id = j.id INNER JOIN _timescaledb_config.bgw_job j ON p.job_id = j.id
INNER JOIN _timescaledb_internal.bgw_job_stat js on p.job_id = js.job_id INNER JOIN _timescaledb_internal.bgw_job_stat js on p.job_id = js.job_id
@ -57,11 +57,28 @@ CREATE OR REPLACE VIEW timescaledb_information.policy_stats as
CREATE OR REPLACE VIEW timescaledb_information.continuous_aggregates as CREATE OR REPLACE VIEW timescaledb_information.continuous_aggregates as
SELECT format('%1$I.%2$I', cagg.user_view_schema, cagg.user_view_name)::regclass as view_name, SELECT format('%1$I.%2$I', cagg.user_view_schema, cagg.user_view_name)::regclass as view_name,
viewinfo.viewowner as view_owner, viewinfo.viewowner as view_owner,
cagg.refresh_lag, CASE _timescaledb_internal.get_time_type(cagg.raw_hypertable_id)
WHEN 'TIMESTAMP'::regtype
THEN _timescaledb_internal.to_interval(cagg.refresh_lag)::TEXT
WHEN 'TIMESTAMPTZ'::regtype
THEN _timescaledb_internal.to_interval(cagg.refresh_lag)::TEXT
WHEN 'DATE'::regtype
THEN _timescaledb_internal.to_interval(cagg.refresh_lag)::TEXT
ELSE cagg.refresh_lag::TEXT
END AS refresh_lag,
bgwjob.schedule_interval as refresh_interval, bgwjob.schedule_interval as refresh_interval,
CASE _timescaledb_internal.get_time_type(cagg.raw_hypertable_id)
WHEN 'TIMESTAMP'::regtype
THEN _timescaledb_internal.to_interval(cagg.max_interval_per_job)::TEXT
WHEN 'TIMESTAMPTZ'::regtype
THEN _timescaledb_internal.to_interval(cagg.max_interval_per_job)::TEXT
WHEN 'DATE'::regtype
THEN _timescaledb_internal.to_interval(cagg.max_interval_per_job)::TEXT
ELSE cagg.max_interval_per_job::TEXT
END AS max_interval_per_job,
format('%1$I.%2$I', ht.schema_name, ht.table_name)::regclass as materialization_hypertable, format('%1$I.%2$I', ht.schema_name, ht.table_name)::regclass as materialization_hypertable,
directview.viewdefinition as view_definition directview.viewdefinition as view_definition
FROM _timescaledb_catalog.continuous_agg cagg, FROM _timescaledb_catalog.continuous_agg cagg,
_timescaledb_catalog.hypertable ht, LATERAL _timescaledb_catalog.hypertable ht, LATERAL
( select C.oid, pg_get_userbyid( C.relowner) as viewowner ( select C.oid, pg_get_userbyid( C.relowner) as viewowner
FROM pg_class C LEFT JOIN pg_namespace N on (N.oid = C.relnamespace) FROM pg_class C LEFT JOIN pg_namespace N on (N.oid = C.relnamespace)
@ -78,18 +95,35 @@ CREATE OR REPLACE VIEW timescaledb_information.continuous_aggregates as
CREATE OR REPLACE VIEW timescaledb_information.continuous_aggregate_stats as CREATE OR REPLACE VIEW timescaledb_information.continuous_aggregate_stats as
SELECT format('%1$I.%2$I', cagg.user_view_schema, cagg.user_view_name)::regclass as view_name, SELECT format('%1$I.%2$I', cagg.user_view_schema, cagg.user_view_name)::regclass as view_name,
ct.watermark as completed_threshold, CASE _timescaledb_internal.get_time_type(cagg.raw_hypertable_id)
it.watermark as invalidation_threshold, WHEN 'TIMESTAMP'::regtype
THEN _timescaledb_internal.to_timestamp_without_timezone(ct.watermark)::TEXT
WHEN 'TIMESTAMPTZ'::regtype
THEN _timescaledb_internal.to_timestamp(ct.watermark)::TEXT
WHEN 'DATE'::regtype
THEN _timescaledb_internal.to_date(ct.watermark)::TEXT
ELSE ct.watermark::TEXT
END AS completed_threshold,
CASE _timescaledb_internal.get_time_type(cagg.raw_hypertable_id)
WHEN 'TIMESTAMP'::regtype
THEN _timescaledb_internal.to_timestamp_without_timezone(it.watermark)::TEXT
WHEN 'TIMESTAMPTZ'::regtype
THEN _timescaledb_internal.to_timestamp(it.watermark)::TEXT
WHEN 'DATE'::regtype
THEN _timescaledb_internal.to_date(it.watermark)::TEXT
ELSE it.watermark::TEXT
END AS invalidation_threshold,
cagg.job_id as job_id,
bgw_job_stat.last_start as last_run_started_at, bgw_job_stat.last_start as last_run_started_at,
case when bgw_job_stat.last_finish < '4714-11-24 00:00:00+00 BC' then 'running' case when bgw_job_stat.last_finish < '4714-11-24 00:00:00+00 BC' then 'running'
when bgw_job_stat.next_start is not null then 'scheduled' when bgw_job_stat.next_start is not null then 'scheduled'
end as job_status, end as job_status,
case when bgw_job_stat.last_finish > bgw_job_stat.last_start then (bgw_job_stat.last_finish - bgw_job_stat.last_start) case when bgw_job_stat.last_finish > bgw_job_stat.last_start then (bgw_job_stat.last_finish - bgw_job_stat.last_start)
end as last_run_duration, end as last_run_duration,
bgw_job_stat.next_start as next_scheduled_run bgw_job_stat.next_start as next_scheduled_run
FROM FROM
_timescaledb_catalog.continuous_agg as cagg JOIN _timescaledb_catalog.continuous_agg as cagg
_timescaledb_internal.bgw_job_stat as bgw_job_stat LEFT JOIN _timescaledb_internal.bgw_job_stat as bgw_job_stat
ON ( cagg.job_id = bgw_job_stat.job_id ) ON ( cagg.job_id = bgw_job_stat.job_id )
LEFT JOIN _timescaledb_catalog.continuous_aggs_invalidation_threshold as it LEFT JOIN _timescaledb_catalog.continuous_aggs_invalidation_threshold as it
ON ( cagg.raw_hypertable_id = it.hypertable_id) ON ( cagg.raw_hypertable_id = it.hypertable_id)

View File

@ -200,6 +200,25 @@ ts_hypertable_relid_to_id(Oid relid)
return result; return result;
} }
TS_FUNCTION_INFO_V1(ts_hypertable_get_time_type);
Datum
ts_hypertable_get_time_type(PG_FUNCTION_ARGS)
{
int32 hypertable_id = PG_GETARG_INT32(0);
Cache *hcache = ts_hypertable_cache_pin();
Hypertable *ht = ts_hypertable_cache_get_entry_by_id(hcache, hypertable_id);
Dimension *time_dimension;
Oid time_type;
if (ht == NULL)
PG_RETURN_NULL();
time_dimension = hyperspace_get_open_dimension(ht->space, 0);
if (time_dimension == NULL)
PG_RETURN_NULL();
time_type = time_dimension->fd.column_type;
ts_cache_release(hcache);
PG_RETURN_OID(time_type);
}
typedef struct ChunkStoreEntry typedef struct ChunkStoreEntry
{ {
MemoryContext mcxt; MemoryContext mcxt;

View File

@ -67,6 +67,8 @@ ts_pg_timestamp_to_unix_microseconds(PG_FUNCTION_ARGS)
} }
TS_FUNCTION_INFO_V1(ts_pg_unix_microseconds_to_timestamp); TS_FUNCTION_INFO_V1(ts_pg_unix_microseconds_to_timestamp);
TS_FUNCTION_INFO_V1(ts_pg_unix_microseconds_to_timestamp_without_timezone);
TS_FUNCTION_INFO_V1(ts_pg_unix_microseconds_to_date);
/* /*
* Convert BIGINT microseconds relative the UNIX epoch to a Postgres TIMESTAMP. * Convert BIGINT microseconds relative the UNIX epoch to a Postgres TIMESTAMP.
@ -100,6 +102,16 @@ ts_pg_unix_microseconds_to_timestamp(PG_FUNCTION_ARGS)
PG_RETURN_TIMESTAMPTZ(timestamp); PG_RETURN_TIMESTAMPTZ(timestamp);
} }
Datum
ts_pg_unix_microseconds_to_date(PG_FUNCTION_ARGS)
{
int64 microseconds = PG_GETARG_INT64(0);
Datum res =
DirectFunctionCall1(ts_pg_unix_microseconds_to_timestamp, Int64GetDatum(microseconds));
res = DirectFunctionCall1(timestamp_date, res);
PG_RETURN_DATUM(res);
}
static int64 ts_integer_to_internal(Datum time_val, Oid type_oid); static int64 ts_integer_to_internal(Datum time_val, Oid type_oid);
/* Convert valid timescale time column type to internal representation */ /* Convert valid timescale time column type to internal representation */
@ -276,7 +288,6 @@ static Datum ts_integer_to_internal_value(int64 value, Oid type);
TSDLLEXPORT Datum TSDLLEXPORT Datum
ts_internal_to_time_value(int64 value, Oid type) ts_internal_to_time_value(int64 value, Oid type)
{ {
Datum res;
switch (type) switch (type)
{ {
case INT2OID: case INT2OID:
@ -287,12 +298,9 @@ ts_internal_to_time_value(int64 value, Oid type)
case TIMESTAMPTZOID: case TIMESTAMPTZOID:
/* we continue ts_time_value_to_internal's incorrect handling of TIMESTAMPs for /* we continue ts_time_value_to_internal's incorrect handling of TIMESTAMPs for
* compatibility */ * compatibility */
res = DirectFunctionCall1(ts_pg_unix_microseconds_to_timestamp, Int64GetDatum(value)); return DirectFunctionCall1(ts_pg_unix_microseconds_to_timestamp, Int64GetDatum(value));
return TimestampTzGetDatum(res);
case DATEOID: case DATEOID:
res = DirectFunctionCall1(ts_pg_unix_microseconds_to_timestamp, Int64GetDatum(value)); return DirectFunctionCall1(ts_pg_unix_microseconds_to_date, Int64GetDatum(value));
res = DirectFunctionCall1(timestamp_date, res);
return DateADTGetDatum(res);
default: default:
if (ts_type_is_int8_binary_compatible(type)) if (ts_type_is_int8_binary_compatible(type))
return Int64GetDatum(value); return Int64GetDatum(value);

View File

@ -86,6 +86,14 @@ CREATE VIEW test_continuous_agg_view
FROM test_continuous_agg_table FROM test_continuous_agg_table
GROUP BY 1; GROUP BY 1;
NOTICE: adding not-null constraint to column "time_partition_col" NOTICE: adding not-null constraint to column "time_partition_col"
-- even before running, stats shows something
SELECT view_name, completed_threshold, invalidation_threshold, job_status, last_run_duration
FROM timescaledb_information.continuous_aggregate_stats;
view_name | completed_threshold | invalidation_threshold | job_status | last_run_duration
--------------------------+---------------------+------------------------+------------+-------------------
test_continuous_agg_view | | | |
(1 row)
SELECT id as raw_table_id FROM _timescaledb_catalog.hypertable WHERE table_name='test_continuous_agg_table' \gset SELECT id as raw_table_id FROM _timescaledb_catalog.hypertable WHERE table_name='test_continuous_agg_table' \gset
-- min distance from end should be 1 -- min distance from end should be 1
SELECT mat_hypertable_id, user_view_schema, user_view_name, bucket_width, job_id, refresh_lag FROM _timescaledb_catalog.continuous_agg; SELECT mat_hypertable_id, user_view_schema, user_view_name, bucket_width, job_id, refresh_lag FROM _timescaledb_catalog.continuous_agg;
@ -286,7 +294,7 @@ SELECT * FROM test_continuous_agg_view ORDER BY 1;
\x \x
--check the information views -- --check the information views --
select view_name, view_owner, refresh_lag, refresh_interval, materialization_hypertable select view_name, view_owner, refresh_lag, refresh_interval, max_interval_per_job, materialization_hypertable
from timescaledb_information.continuous_aggregates from timescaledb_information.continuous_aggregates
where view_name::text like '%test_continuous_agg_view'; where view_name::text like '%test_continuous_agg_view';
-[ RECORD 1 ]--------------+------------------------------------------------- -[ RECORD 1 ]--------------+-------------------------------------------------
@ -294,6 +302,7 @@ view_name | test_continuous_agg_view
view_owner | default_perm_user view_owner | default_perm_user
refresh_lag | -2 refresh_lag | -2
refresh_interval | @ 12 hours refresh_interval | @ 12 hours
max_interval_per_job | 2
materialization_hypertable | _timescaledb_internal._materialized_hypertable_3 materialization_hypertable | _timescaledb_internal._materialized_hypertable_3
select view_name, view_definition from timescaledb_information.continuous_aggregates select view_name, view_definition from timescaledb_information.continuous_aggregates

View File

@ -1196,3 +1196,105 @@ SELECT * FROM max_mat_view_t ORDER BY 1;
2019-09-09 10:00:00+00 | 1 2019-09-09 10:00:00+00 | 1
(6 rows) (6 rows)
-- regular timestamp
CREATE TABLE continuous_agg_max_mat_timestamp(time TIMESTAMP);
SELECT create_hypertable('continuous_agg_max_mat_timestamp', 'time');
NOTICE: adding not-null constraint to column "time"
create_hypertable
------------------------------------------------
(15,public,continuous_agg_max_mat_timestamp,t)
(1 row)
CREATE VIEW max_mat_view_timestamp
WITH (timescaledb.continuous, timescaledb.refresh_lag='-2 hours')
AS SELECT time_bucket('2 hours', time)
FROM continuous_agg_max_mat_timestamp
GROUP BY 1;
NOTICE: adding not-null constraint to column "time_partition_col"
INSERT INTO continuous_agg_max_mat_timestamp
SELECT generate_series('2019-09-09 1:00'::TIMESTAMPTZ, '2019-09-09 10:00', '1 hour');
-- first materializes everything
REFRESH MATERIALIZED VIEW max_mat_view_timestamp;
INFO: new materialization range for public.continuous_agg_max_mat_timestamp (time column time) (1568030400000000)
INFO: materializing continuous aggregate public.max_mat_view_timestamp: new range up to 1568030400000000
SELECT * FROM max_mat_view_timestamp ORDER BY 1;
time_bucket
---------------------
2019-09-09 00:00:00
2019-09-09 02:00:00
2019-09-09 04:00:00
2019-09-09 06:00:00
2019-09-09 08:00:00
2019-09-09 10:00:00
(6 rows)
-- date
CREATE TABLE continuous_agg_max_mat_date(time DATE);
SELECT create_hypertable('continuous_agg_max_mat_date', 'time');
NOTICE: adding not-null constraint to column "time"
create_hypertable
-------------------------------------------
(17,public,continuous_agg_max_mat_date,t)
(1 row)
CREATE VIEW max_mat_view_date
WITH (timescaledb.continuous, timescaledb.refresh_lag='-7 days')
AS SELECT time_bucket('7 days', time)
FROM continuous_agg_max_mat_date
GROUP BY 1;
NOTICE: adding not-null constraint to column "time_partition_col"
INSERT INTO continuous_agg_max_mat_date
SELECT generate_series('2019-09-01'::DATE, '2019-09-010 10:00', '1 day');
-- first materializes everything
REFRESH MATERIALIZED VIEW max_mat_view_date;
INFO: new materialization range for public.continuous_agg_max_mat_date (time column time) (1568592000000000)
INFO: materializing continuous aggregate public.max_mat_view_date: new range up to 1568592000000000
SELECT * FROM max_mat_view_date ORDER BY 1;
time_bucket
-------------
2019-08-26
2019-09-02
2019-09-09
(3 rows)
SELECT view_name, completed_threshold, invalidation_threshold, job_id, job_status, last_run_duration
FROM timescaledb_information.continuous_aggregate_stats ORDER BY 1;
view_name | completed_threshold | invalidation_threshold | job_id | job_status | last_run_duration
------------------------+------------------------+------------------------+--------+------------+-------------------
max_mat_view | 12 | 12 | 1003 | |
max_mat_view_t | 2019-09-09 12:00:00+00 | 2019-09-09 12:00:00+00 | 1004 | |
max_mat_view_timestamp | 2019-09-09 12:00:00 | 2019-09-09 12:00:00 | 1005 | |
max_mat_view_date | 2019-09-16 | 2019-09-16 | 1006 | |
(4 rows)
SELECT view_name, refresh_lag, max_interval_per_job
FROM timescaledb_information.continuous_aggregates ORDER BY 1;
view_name | refresh_lag | max_interval_per_job
------------------------+---------------+-----------------------------------------
max_mat_view | -2 | 4
max_mat_view_t | @ 2 hours ago | @ 4 hours
max_mat_view_timestamp | @ 2 hours ago | @ 106751991 days 4 hours 54.775807 secs
max_mat_view_date | @ 7 days ago | @ 106751991 days 4 hours 54.775807 secs
(4 rows)
SET SESSION timezone TO 'EST';
SELECT view_name, completed_threshold, invalidation_threshold, job_id, job_status, last_run_duration
FROM timescaledb_information.continuous_aggregate_stats ORDER BY 1;
view_name | completed_threshold | invalidation_threshold | job_id | job_status | last_run_duration
------------------------+------------------------+------------------------+--------+------------+-------------------
max_mat_view | 12 | 12 | 1003 | |
max_mat_view_t | 2019-09-09 07:00:00-05 | 2019-09-09 07:00:00-05 | 1004 | |
max_mat_view_timestamp | 2019-09-09 12:00:00 | 2019-09-09 12:00:00 | 1005 | |
max_mat_view_date | 2019-09-16 | 2019-09-16 | 1006 | |
(4 rows)
SELECT view_name, refresh_lag, max_interval_per_job
FROM timescaledb_information.continuous_aggregates ORDER BY 1;
view_name | refresh_lag | max_interval_per_job
------------------------+---------------+-----------------------------------------
max_mat_view | -2 | 4
max_mat_view_t | @ 2 hours ago | @ 4 hours
max_mat_view_timestamp | @ 2 hours ago | @ 106751991 days 4 hours 54.775807 secs
max_mat_view_date | @ 7 days ago | @ 106751991 days 4 hours 54.775807 secs
(4 rows)

View File

@ -69,6 +69,11 @@ CREATE VIEW test_continuous_agg_view
AS SELECT time_bucket('2', time), SUM(data) as value AS SELECT time_bucket('2', time), SUM(data) as value
FROM test_continuous_agg_table FROM test_continuous_agg_table
GROUP BY 1; GROUP BY 1;
-- even before running, stats shows something
SELECT view_name, completed_threshold, invalidation_threshold, job_status, last_run_duration
FROM timescaledb_information.continuous_aggregate_stats;
SELECT id as raw_table_id FROM _timescaledb_catalog.hypertable WHERE table_name='test_continuous_agg_table' \gset SELECT id as raw_table_id FROM _timescaledb_catalog.hypertable WHERE table_name='test_continuous_agg_table' \gset
-- min distance from end should be 1 -- min distance from end should be 1
@ -161,7 +166,7 @@ SELECT * FROM test_continuous_agg_view ORDER BY 1;
\x \x
--check the information views -- --check the information views --
select view_name, view_owner, refresh_lag, refresh_interval, materialization_hypertable select view_name, view_owner, refresh_lag, refresh_interval, max_interval_per_job, materialization_hypertable
from timescaledb_information.continuous_aggregates from timescaledb_information.continuous_aggregates
where view_name::text like '%test_continuous_agg_view'; where view_name::text like '%test_continuous_agg_view';

View File

@ -524,3 +524,51 @@ SELECT * FROM max_mat_view_t ORDER BY 1;
REFRESH MATERIALIZED VIEW max_mat_view_t; REFRESH MATERIALIZED VIEW max_mat_view_t;
SELECT * FROM max_mat_view_t ORDER BY 1; SELECT * FROM max_mat_view_t ORDER BY 1;
-- regular timestamp
CREATE TABLE continuous_agg_max_mat_timestamp(time TIMESTAMP);
SELECT create_hypertable('continuous_agg_max_mat_timestamp', 'time');
CREATE VIEW max_mat_view_timestamp
WITH (timescaledb.continuous, timescaledb.refresh_lag='-2 hours')
AS SELECT time_bucket('2 hours', time)
FROM continuous_agg_max_mat_timestamp
GROUP BY 1;
INSERT INTO continuous_agg_max_mat_timestamp
SELECT generate_series('2019-09-09 1:00'::TIMESTAMPTZ, '2019-09-09 10:00', '1 hour');
-- first materializes everything
REFRESH MATERIALIZED VIEW max_mat_view_timestamp;
SELECT * FROM max_mat_view_timestamp ORDER BY 1;
-- date
CREATE TABLE continuous_agg_max_mat_date(time DATE);
SELECT create_hypertable('continuous_agg_max_mat_date', 'time');
CREATE VIEW max_mat_view_date
WITH (timescaledb.continuous, timescaledb.refresh_lag='-7 days')
AS SELECT time_bucket('7 days', time)
FROM continuous_agg_max_mat_date
GROUP BY 1;
INSERT INTO continuous_agg_max_mat_date
SELECT generate_series('2019-09-01'::DATE, '2019-09-010 10:00', '1 day');
-- first materializes everything
REFRESH MATERIALIZED VIEW max_mat_view_date;
SELECT * FROM max_mat_view_date ORDER BY 1;
SELECT view_name, completed_threshold, invalidation_threshold, job_id, job_status, last_run_duration
FROM timescaledb_information.continuous_aggregate_stats ORDER BY 1;
SELECT view_name, refresh_lag, max_interval_per_job
FROM timescaledb_information.continuous_aggregates ORDER BY 1;
SET SESSION timezone TO 'EST';
SELECT view_name, completed_threshold, invalidation_threshold, job_id, job_status, last_run_duration
FROM timescaledb_information.continuous_aggregate_stats ORDER BY 1;
SELECT view_name, refresh_lag, max_interval_per_job
FROM timescaledb_information.continuous_aggregates ORDER BY 1;