timescaledb/sql/size_utils.sql
Nikhil Sontakke 293104add2 Use numrows_pre_compression in approx row count
The approximate_row_count function was using the reltuples from
compressed chunks and multiplying that with 1000 which is the default
batch size. This was leading to a huge skew between the actual row
count and the approximate one. We now use the numrows_pre_compression
value from the timescaledb catalog which accurately represents the
number of rows before the actual compression.
2023-12-04 22:26:56 +05:30

864 lines
33 KiB
PL/PgSQL

-- This file and its contents are licensed under the Apache License 2.0.
-- Please see the included NOTICE for copyright information and
-- LICENSE-APACHE for a copy of the license.
-- This file contains utility functions to get the relation size
-- of hypertables, chunks, and indexes on hypertables.
CREATE OR REPLACE FUNCTION _timescaledb_functions.relation_size(relation REGCLASS)
RETURNS TABLE (total_size BIGINT, heap_size BIGINT, index_size BIGINT, toast_size BIGINT)
AS '@MODULE_PATHNAME@', 'ts_relation_size' LANGUAGE C VOLATILE;
CREATE OR REPLACE VIEW _timescaledb_internal.hypertable_chunk_local_size AS
SELECT
h.schema_name AS hypertable_schema,
h.table_name AS hypertable_name,
h.id AS hypertable_id,
c.id AS chunk_id,
c.schema_name AS chunk_schema,
c.table_name AS chunk_name,
COALESCE((relsize).total_size, 0) AS total_bytes,
COALESCE((relsize).heap_size, 0) AS heap_bytes,
COALESCE((relsize).index_size, 0) AS index_bytes,
COALESCE((relsize).toast_size, 0) AS toast_bytes,
COALESCE((relcompsize).total_size, 0) AS compressed_total_size,
COALESCE((relcompsize).heap_size, 0) AS compressed_heap_size,
COALESCE((relcompsize).index_size, 0) AS compressed_index_size,
COALESCE((relcompsize).toast_size, 0) AS compressed_toast_size
FROM
_timescaledb_catalog.hypertable h
JOIN _timescaledb_catalog.chunk c ON h.id = c.hypertable_id
AND c.dropped IS FALSE
JOIN pg_class cl ON cl.relname = c.table_name AND cl.relkind = 'r'
JOIN pg_namespace n ON n.oid = cl.relnamespace
AND n.nspname = c.schema_name
JOIN LATERAL _timescaledb_functions.relation_size(cl.oid) AS relsize ON TRUE
LEFT JOIN _timescaledb_catalog.chunk comp ON comp.id = c.compressed_chunk_id
LEFT JOIN LATERAL _timescaledb_functions.relation_size(
CASE WHEN comp.schema_name IS NOT NULL AND comp.table_name IS NOT NULL THEN
format('%I.%I', comp.schema_name, comp.table_name)::regclass
ELSE
NULL::regclass
END
) AS relcompsize ON TRUE;
GRANT SELECT ON _timescaledb_internal.hypertable_chunk_local_size TO PUBLIC;
CREATE OR REPLACE FUNCTION _timescaledb_functions.data_node_hypertable_info(
node_name NAME,
schema_name_in name,
table_name_in name
)
RETURNS TABLE (
table_bytes bigint,
index_bytes bigint,
toast_bytes bigint,
total_bytes bigint)
AS '@MODULE_PATHNAME@', 'ts_dist_remote_hypertable_info' LANGUAGE C VOLATILE STRICT;
CREATE OR REPLACE FUNCTION _timescaledb_functions.data_node_chunk_info(
node_name NAME,
schema_name_in name,
table_name_in name
)
RETURNS TABLE (
chunk_id integer,
chunk_schema name,
chunk_name name,
table_bytes bigint,
index_bytes bigint,
toast_bytes bigint,
total_bytes bigint)
AS '@MODULE_PATHNAME@', 'ts_dist_remote_chunk_info' LANGUAGE C VOLATILE STRICT;
CREATE OR REPLACE FUNCTION _timescaledb_functions.hypertable_local_size(
schema_name_in name,
table_name_in name)
RETURNS TABLE (
table_bytes BIGINT,
index_bytes BIGINT,
toast_bytes BIGINT,
total_bytes BIGINT)
LANGUAGE SQL VOLATILE STRICT AS
$BODY$
/* get the main hypertable id and sizes */
WITH _hypertable_sizes AS (
SELECT
id,
COALESCE((relsize).total_size, 0) AS total_bytes,
COALESCE((relsize).heap_size, 0) AS heap_bytes,
COALESCE((relsize).index_size, 0) AS index_bytes,
COALESCE((relsize).toast_size, 0) AS toast_bytes,
0::BIGINT AS compressed_total_size,
0::BIGINT AS compressed_index_size,
0::BIGINT AS compressed_toast_size,
0::BIGINT AS compressed_heap_size
FROM
_timescaledb_catalog.hypertable ht
JOIN pg_class c ON relname = ht.table_name AND c.relkind = 'r'
JOIN pg_namespace n ON n.oid = c.relnamespace
AND n.nspname = ht.schema_name
JOIN LATERAL _timescaledb_functions.relation_size(c.oid) AS relsize ON TRUE
WHERE
schema_name = schema_name_in
AND table_name = table_name_in
),
/* calculate the size of the hypertable chunks */
_chunk_sizes AS (
SELECT
chunk_id,
COALESCE(ch.total_bytes, 0) AS total_bytes,
COALESCE(ch.heap_bytes, 0) AS heap_bytes,
COALESCE(ch.index_bytes, 0) AS index_bytes,
COALESCE(ch.toast_bytes, 0) AS toast_bytes,
COALESCE(ch.compressed_total_size, 0) AS compressed_total_size,
COALESCE(ch.compressed_index_size, 0) AS compressed_index_size,
COALESCE(ch.compressed_toast_size, 0) AS compressed_toast_size,
COALESCE(ch.compressed_heap_size, 0) AS compressed_heap_size
FROM
_timescaledb_internal.hypertable_chunk_local_size ch
JOIN _hypertable_sizes ht ON ht.id = ch.hypertable_id
WHERE hypertable_schema = schema_name_in
AND hypertable_name = table_name_in
)
/* calculate the SUM of the hypertable and chunk sizes */
SELECT
(SUM(heap_bytes) + SUM(compressed_heap_size))::BIGINT AS heap_bytes,
(SUM(index_bytes) + SUM(compressed_index_size))::BIGINT AS index_bytes,
(SUM(toast_bytes) + SUM(compressed_toast_size))::BIGINT AS toast_bytes,
(SUM(total_bytes) + SUM(compressed_total_size))::BIGINT AS total_bytes
FROM
(SELECT * FROM _hypertable_sizes
UNION ALL
SELECT * FROM _chunk_sizes) AS sizes;
$BODY$ SET search_path TO pg_catalog, pg_temp;
CREATE OR REPLACE FUNCTION _timescaledb_functions.hypertable_remote_size(
schema_name_in name,
table_name_in name)
RETURNS TABLE (
table_bytes bigint,
index_bytes bigint,
toast_bytes bigint,
total_bytes bigint,
node_name NAME)
LANGUAGE SQL VOLATILE STRICT AS
$BODY$
SELECT
sum(entry.table_bytes)::bigint AS table_bytes,
sum(entry.index_bytes)::bigint AS index_bytes,
sum(entry.toast_bytes)::bigint AS toast_bytes,
sum(entry.total_bytes)::bigint AS total_bytes,
srv.node_name
FROM (
SELECT
s.node_name
FROM
_timescaledb_catalog.hypertable AS ht,
_timescaledb_catalog.hypertable_data_node AS s
WHERE
ht.schema_name = schema_name_in
AND ht.table_name = table_name_in
AND s.hypertable_id = ht.id
) AS srv
LEFT OUTER JOIN LATERAL _timescaledb_functions.data_node_hypertable_info(
srv.node_name, schema_name_in, table_name_in) entry ON TRUE
GROUP BY srv.node_name;
$BODY$ SET search_path TO pg_catalog, pg_temp;
-- Get relation size of hypertable
-- like pg_relation_size(hypertable)
--
-- hypertable - hypertable to get size of
--
-- Returns:
-- table_bytes - Disk space used by hypertable (like pg_relation_size(hypertable))
-- index_bytes - Disk space used by indexes
-- toast_bytes - Disk space of toast tables
-- total_bytes - Total disk space used by the specified table, including all indexes and TOAST data
CREATE OR REPLACE FUNCTION @extschema@.hypertable_detailed_size(
hypertable REGCLASS)
RETURNS TABLE (table_bytes BIGINT,
index_bytes BIGINT,
toast_bytes BIGINT,
total_bytes BIGINT,
node_name NAME)
LANGUAGE PLPGSQL VOLATILE STRICT AS
$BODY$
DECLARE
table_name NAME = NULL;
schema_name NAME = NULL;
is_distributed BOOL = FALSE;
BEGIN
SELECT relname, nspname, replication_factor > 0
INTO table_name, schema_name, is_distributed
FROM pg_class c
INNER JOIN pg_namespace n ON (n.OID = c.relnamespace)
INNER JOIN _timescaledb_catalog.hypertable ht ON (ht.schema_name = n.nspname AND ht.table_name = c.relname)
WHERE c.OID = hypertable;
IF table_name IS NULL THEN
SELECT h.schema_name, h.table_name, replication_factor > 0
INTO schema_name, table_name, is_distributed
FROM pg_class c
INNER JOIN pg_namespace n ON (n.OID = c.relnamespace)
INNER JOIN _timescaledb_catalog.continuous_agg a ON (a.user_view_schema = n.nspname AND a.user_view_name = c.relname)
INNER JOIN _timescaledb_catalog.hypertable h ON h.id = a.mat_hypertable_id
WHERE c.OID = hypertable;
IF table_name IS NULL THEN
RETURN;
END IF;
END IF;
CASE WHEN is_distributed THEN
RETURN QUERY
SELECT *, NULL::name
FROM _timescaledb_functions.hypertable_local_size(schema_name, table_name)
UNION
SELECT *
FROM _timescaledb_functions.hypertable_remote_size(schema_name, table_name);
ELSE
RETURN QUERY
SELECT *, NULL::name
FROM _timescaledb_functions.hypertable_local_size(schema_name, table_name);
END CASE;
END;
$BODY$ SET search_path TO pg_catalog, pg_temp;
--- returns total-bytes for a hypertable (includes table + index)
CREATE OR REPLACE FUNCTION @extschema@.hypertable_size(
hypertable REGCLASS)
RETURNS BIGINT
LANGUAGE SQL VOLATILE STRICT AS
$BODY$
-- One row per data node is returned (in case of a distributed
-- hypertable), so sum them up:
SELECT sum(total_bytes)::bigint
FROM @extschema@.hypertable_detailed_size(hypertable);
$BODY$ SET search_path TO pg_catalog, pg_temp;
CREATE OR REPLACE FUNCTION _timescaledb_functions.chunks_local_size(
schema_name_in name,
table_name_in name)
RETURNS TABLE (
chunk_id integer,
chunk_schema NAME,
chunk_name NAME,
table_bytes bigint,
index_bytes bigint,
toast_bytes bigint,
total_bytes bigint)
LANGUAGE SQL VOLATILE STRICT AS
$BODY$
SELECT
ch.chunk_id,
ch.chunk_schema,
ch.chunk_name,
(ch.total_bytes - COALESCE( ch.index_bytes , 0 ) - COALESCE( ch.toast_bytes, 0 ) + COALESCE( ch.compressed_heap_size , 0 ))::bigint as heap_bytes,
(COALESCE( ch.index_bytes, 0 ) + COALESCE( ch.compressed_index_size , 0) )::bigint as index_bytes,
(COALESCE( ch.toast_bytes, 0 ) + COALESCE( ch.compressed_toast_size, 0 ))::bigint as toast_bytes,
(ch.total_bytes + COALESCE( ch.compressed_total_size, 0 ))::bigint as total_bytes
FROM
_timescaledb_internal.hypertable_chunk_local_size ch
WHERE
ch.hypertable_schema = schema_name_in
AND ch.hypertable_name = table_name_in;
$BODY$ SET search_path TO pg_catalog, pg_temp;
---should return same information as chunks_local_size--
CREATE OR REPLACE FUNCTION _timescaledb_functions.chunks_remote_size(
schema_name_in name,
table_name_in name)
RETURNS TABLE (
chunk_id integer,
chunk_schema NAME,
chunk_name NAME,
table_bytes bigint,
index_bytes bigint,
toast_bytes bigint,
total_bytes bigint,
node_name NAME)
LANGUAGE SQL VOLATILE STRICT AS
$BODY$
SELECT
entry.chunk_id,
entry.chunk_schema,
entry.chunk_name,
entry.table_bytes AS table_bytes,
entry.index_bytes AS index_bytes,
entry.toast_bytes AS toast_bytes,
entry.total_bytes AS total_bytes,
srv.node_name
FROM (
SELECT
s.node_name
FROM
_timescaledb_catalog.hypertable AS ht,
_timescaledb_catalog.hypertable_data_node AS s
WHERE
ht.schema_name = schema_name_in
AND ht.table_name = table_name_in
AND s.hypertable_id = ht.id
) AS srv
LEFT OUTER JOIN LATERAL _timescaledb_functions.data_node_chunk_info(
srv.node_name, schema_name_in, table_name_in) entry ON TRUE
WHERE
entry.chunk_name IS NOT NULL;
$BODY$ SET search_path TO pg_catalog, pg_temp;
-- Get relation size of the chunks of an hypertable
-- hypertable - hypertable to get size of
--
-- Returns:
-- chunk_schema - schema name for chunk
-- chunk_name - chunk table name
-- table_bytes - Disk space used by chunk table
-- index_bytes - Disk space used by indexes
-- toast_bytes - Disk space of toast tables
-- total_bytes - Disk space used in total
-- node_name - node on which chunk lives if this is
-- a distributed hypertable.
CREATE OR REPLACE FUNCTION @extschema@.chunks_detailed_size(
hypertable REGCLASS
)
RETURNS TABLE (
chunk_schema NAME,
chunk_name NAME,
table_bytes BIGINT,
index_bytes BIGINT,
toast_bytes BIGINT,
total_bytes BIGINT,
node_name NAME)
LANGUAGE PLPGSQL VOLATILE STRICT AS
$BODY$
DECLARE
table_name NAME;
schema_name NAME;
is_distributed BOOL;
BEGIN
SELECT relname, nspname, replication_factor > 0
INTO table_name, schema_name, is_distributed
FROM pg_class c
INNER JOIN pg_namespace n ON (n.OID = c.relnamespace)
INNER JOIN _timescaledb_catalog.hypertable ht ON (ht.schema_name = n.nspname AND ht.table_name = c.relname)
WHERE c.OID = hypertable;
IF table_name IS NULL THEN
SELECT h.schema_name, h.table_name, replication_factor > 0
INTO schema_name, table_name, is_distributed
FROM pg_class c
INNER JOIN pg_namespace n ON (n.OID = c.relnamespace)
INNER JOIN _timescaledb_catalog.continuous_agg a ON (a.user_view_schema = n.nspname AND a.user_view_name = c.relname)
INNER JOIN _timescaledb_catalog.hypertable h ON h.id = a.mat_hypertable_id
WHERE c.OID = hypertable;
IF table_name IS NULL THEN
RETURN;
END IF;
END IF;
CASE WHEN is_distributed THEN
RETURN QUERY SELECT ch.chunk_schema, ch.chunk_name, ch.table_bytes, ch.index_bytes,
ch.toast_bytes, ch.total_bytes, ch.node_name
FROM _timescaledb_functions.chunks_remote_size(schema_name, table_name) ch;
ELSE
RETURN QUERY SELECT chl.chunk_schema, chl.chunk_name, chl.table_bytes, chl.index_bytes,
chl.toast_bytes, chl.total_bytes, NULL::NAME
FROM _timescaledb_functions.chunks_local_size(schema_name, table_name) chl;
END CASE;
END;
$BODY$ SET search_path TO pg_catalog, pg_temp;
---------- end of detailed size functions ------
CREATE OR REPLACE FUNCTION _timescaledb_functions.range_value_to_pretty(
time_value BIGINT,
column_type REGTYPE
)
RETURNS TEXT LANGUAGE PLPGSQL STABLE AS
$BODY$
DECLARE
BEGIN
IF NOT (time_value > (-9223372036854775808)::bigint AND
time_value < 9223372036854775807::bigint) THEN
RETURN '';
END IF;
IF time_value IS NULL THEN
RETURN format('%L', NULL);
END IF;
CASE column_type
WHEN 'BIGINT'::regtype, 'INTEGER'::regtype, 'SMALLINT'::regtype THEN
RETURN format('%L', time_value); -- scale determined by user.
WHEN 'TIMESTAMP'::regtype, 'TIMESTAMPTZ'::regtype THEN
-- assume time_value is in microsec
RETURN format('%1$L', _timescaledb_functions.to_timestamp(time_value)); -- microseconds
WHEN 'DATE'::regtype THEN
RETURN format('%L', timezone('UTC',_timescaledb_functions.to_timestamp(time_value))::date);
ELSE
RETURN time_value;
END CASE;
END
$BODY$ SET search_path TO pg_catalog, pg_temp;
-- Convenience function to return approximate row count
--
-- relation - table or hypertable to get approximate row count for
--
-- Returns:
-- Estimated number of rows according to catalog tables
CREATE OR REPLACE FUNCTION @extschema@.approximate_row_count(relation REGCLASS)
RETURNS BIGINT
LANGUAGE PLPGSQL VOLATILE STRICT AS
$BODY$
DECLARE
mat_ht REGCLASS = NULL;
local_table_name NAME = NULL;
local_schema_name NAME = NULL;
is_distributed BOOL = FALSE;
is_compressed BOOL = FALSE;
uncompressed_row_count BIGINT = 0;
compressed_row_count BIGINT = 0;
local_compressed_hypertable_id INTEGER = 0;
local_compressed_chunk_id INTEGER = 0;
compressed_hypertable_oid OID;
local_compressed_chunk_oid OID;
max_compressed_row_count BIGINT = 1000;
is_compressed_chunk INTEGER;
BEGIN
-- Check if input relation is continuous aggregate view then
-- get the corresponding materialized hypertable and schema name
SELECT format('%I.%I', ht.schema_name, ht.table_name)::regclass
INTO mat_ht
FROM pg_class c
JOIN pg_namespace n ON (n.OID = c.relnamespace)
JOIN _timescaledb_catalog.continuous_agg a ON (a.user_view_schema = n.nspname AND a.user_view_name = c.relname)
JOIN _timescaledb_catalog.hypertable ht ON (a.mat_hypertable_id = ht.id)
WHERE c.OID = relation;
IF mat_ht IS NOT NULL THEN
relation = mat_ht;
END IF;
SELECT relname, nspname FROM pg_class c
INNER JOIN pg_namespace n ON (n.OID = c.relnamespace)
INTO local_table_name, local_schema_name
WHERE c.OID = relation;
-- Check for input relation is Hypertable
IF EXISTS (SELECT 1
FROM _timescaledb_catalog.hypertable WHERE table_name = local_table_name AND schema_name = local_schema_name) THEN
SELECT compressed_hypertable_id FROM _timescaledb_catalog.hypertable INTO local_compressed_hypertable_id
WHERE table_name = local_table_name AND schema_name = local_schema_name;
IF local_compressed_hypertable_id IS NOT NULL THEN
uncompressed_row_count = _timescaledb_functions.get_approx_row_count(relation);
-- use the compression_chunk_size stats to fetch precompressed num rows
SELECT COALESCE(SUM(numrows_pre_compression), 0) FROM _timescaledb_catalog.chunk srcch,
_timescaledb_catalog.compression_chunk_size map, _timescaledb_catalog.hypertable srcht
INTO compressed_row_count
WHERE map.chunk_id = srcch.id
AND srcht.id = srcch.hypertable_id AND srcht.table_name = local_table_name
AND srcht.schema_name = local_schema_name;
RETURN (uncompressed_row_count + compressed_row_count);
ELSE
uncompressed_row_count = _timescaledb_functions.get_approx_row_count(relation);
RETURN uncompressed_row_count;
END IF;
END IF;
-- Check for input relation is CHUNK
IF EXISTS (SELECT 1 FROM _timescaledb_catalog.chunk WHERE table_name = local_table_name AND schema_name = local_schema_name) THEN
with compressed_chunk as (select 1 as is_compressed_chunk from _timescaledb_catalog.chunk c
inner join _timescaledb_catalog.hypertable h on (c.hypertable_id = h.compressed_hypertable_id)
where c.table_name = local_table_name and c.schema_name = local_schema_name ),
chunk_temp as (select compressed_chunk_id from _timescaledb_catalog.chunk c where c.table_name = local_table_name and c.schema_name = local_schema_name)
select ct.compressed_chunk_id, cc.is_compressed_chunk from chunk_temp ct LEFT OUTER JOIN compressed_chunk cc ON 1 = 1
INTO local_compressed_chunk_id, is_compressed_chunk;
-- 'input is chunk #1';
IF is_compressed_chunk IS NULL AND local_compressed_chunk_id IS NOT NULL THEN
-- 'Include both uncompressed and compressed chunk #2';
-- use the compression_chunk_size stats to fetch precompressed num rows
SELECT COALESCE(numrows_pre_compression, 0) FROM _timescaledb_catalog.compression_chunk_size
INTO compressed_row_count
WHERE compressed_chunk_id = local_compressed_chunk_id;
uncompressed_row_count = _timescaledb_functions.get_approx_row_count(relation);
RETURN (uncompressed_row_count + compressed_row_count);
ELSIF is_compressed_chunk IS NULL AND local_compressed_chunk_id IS NULL THEN
-- 'input relation is uncompressed chunk #3';
uncompressed_row_count = _timescaledb_functions.get_approx_row_count(relation);
RETURN uncompressed_row_count;
ELSE
-- 'compressed chunk only #4';
-- use the compression_chunk_size stats to fetch precompressed num rows
SELECT COALESCE(SUM(numrows_pre_compression), 0) FROM _timescaledb_catalog.chunk srcch,
_timescaledb_catalog.compression_chunk_size map INTO compressed_row_count
WHERE map.compressed_chunk_id = srcch.id
AND srcch.table_name = local_table_name AND srcch.schema_name = local_schema_name;
RETURN compressed_row_count;
END IF;
END IF;
-- Check for input relation is Plain RELATION
uncompressed_row_count = _timescaledb_functions.get_approx_row_count(relation);
RETURN uncompressed_row_count;
END;
$BODY$ SET search_path TO pg_catalog, pg_temp;
CREATE OR REPLACE FUNCTION _timescaledb_functions.get_approx_row_count(relation REGCLASS)
RETURNS BIGINT
LANGUAGE SQL VOLATILE STRICT AS
$BODY$
WITH RECURSIVE inherited_id(oid) AS
(
SELECT relation
UNION ALL
SELECT i.inhrelid
FROM pg_inherits i
JOIN inherited_id b ON i.inhparent = b.oid
)
-- reltuples for partitioned tables is the sum of it's children in pg14 so we need to filter those out
SELECT COALESCE((SUM(reltuples) FILTER (WHERE reltuples > 0 AND relkind <> 'p')), 0)::BIGINT
FROM inherited_id
JOIN pg_class USING (oid);
$BODY$ SET search_path TO pg_catalog, pg_temp;
-------- stats related to compression ------
CREATE OR REPLACE VIEW _timescaledb_internal.compressed_chunk_stats AS
SELECT
srcht.schema_name AS hypertable_schema,
srcht.table_name AS hypertable_name,
srcch.schema_name AS chunk_schema,
srcch.table_name AS chunk_name,
CASE WHEN srcch.compressed_chunk_id IS NULL THEN
'Uncompressed'::text
ELSE
'Compressed'::text
END AS compression_status,
map.uncompressed_heap_size,
map.uncompressed_index_size,
map.uncompressed_toast_size,
map.uncompressed_heap_size + map.uncompressed_toast_size + map.uncompressed_index_size AS uncompressed_total_size,
map.compressed_heap_size,
map.compressed_index_size,
map.compressed_toast_size,
map.compressed_heap_size + map.compressed_toast_size + map.compressed_index_size AS compressed_total_size
FROM
_timescaledb_catalog.hypertable AS srcht
JOIN _timescaledb_catalog.chunk AS srcch ON srcht.id = srcch.hypertable_id
AND srcht.compressed_hypertable_id IS NOT NULL
AND srcch.dropped = FALSE
LEFT JOIN _timescaledb_catalog.compression_chunk_size map ON srcch.id = map.chunk_id;
GRANT SELECT ON _timescaledb_internal.compressed_chunk_stats TO PUBLIC;
CREATE OR REPLACE FUNCTION _timescaledb_functions.data_node_compressed_chunk_stats(node_name name, schema_name_in name, table_name_in name)
RETURNS TABLE (
chunk_schema name,
chunk_name name,
compression_status text,
before_compression_table_bytes bigint,
before_compression_index_bytes bigint,
before_compression_toast_bytes bigint,
before_compression_total_bytes bigint,
after_compression_table_bytes bigint,
after_compression_index_bytes bigint,
after_compression_toast_bytes bigint,
after_compression_total_bytes bigint
)
AS '@MODULE_PATHNAME@' , 'ts_dist_remote_compressed_chunk_info' LANGUAGE C VOLATILE STRICT;
CREATE OR REPLACE FUNCTION _timescaledb_functions.compressed_chunk_local_stats(schema_name_in name, table_name_in name)
RETURNS TABLE (
chunk_schema name,
chunk_name name,
compression_status text,
before_compression_table_bytes bigint,
before_compression_index_bytes bigint,
before_compression_toast_bytes bigint,
before_compression_total_bytes bigint,
after_compression_table_bytes bigint,
after_compression_index_bytes bigint,
after_compression_toast_bytes bigint,
after_compression_total_bytes bigint)
LANGUAGE SQL
STABLE STRICT
AS
$BODY$
SELECT
ch.chunk_schema,
ch.chunk_name,
ch.compression_status,
ch.uncompressed_heap_size,
ch.uncompressed_index_size,
ch.uncompressed_toast_size,
ch.uncompressed_total_size,
ch.compressed_heap_size,
ch.compressed_index_size,
ch.compressed_toast_size,
ch.compressed_total_size
FROM
_timescaledb_internal.compressed_chunk_stats ch
WHERE
ch.hypertable_schema = schema_name_in
AND ch.hypertable_name = table_name_in;
$BODY$ SET search_path TO pg_catalog, pg_temp;
CREATE OR REPLACE FUNCTION _timescaledb_functions.compressed_chunk_remote_stats(schema_name_in name, table_name_in name)
RETURNS TABLE (
chunk_schema name,
chunk_name name,
compression_status text,
before_compression_table_bytes bigint,
before_compression_index_bytes bigint,
before_compression_toast_bytes bigint,
before_compression_total_bytes bigint,
after_compression_table_bytes bigint,
after_compression_index_bytes bigint,
after_compression_toast_bytes bigint,
after_compression_total_bytes bigint,
node_name name)
LANGUAGE SQL
STABLE STRICT
AS
$BODY$
SELECT
ch.*,
srv.node_name
FROM (
SELECT
s.node_name
FROM
_timescaledb_catalog.hypertable AS ht,
_timescaledb_catalog.hypertable_data_node AS s
WHERE
ht.schema_name = schema_name_in
AND ht.table_name = table_name_in
AND s.hypertable_id = ht.id) AS srv
LEFT OUTER JOIN LATERAL _timescaledb_functions.data_node_compressed_chunk_stats(
srv.node_name, schema_name_in, table_name_in) ch ON TRUE
WHERE ch.chunk_name IS NOT NULL;
$BODY$ SET search_path TO pg_catalog, pg_temp;
-- Get per chunk compression statistics for a hypertable that has
-- compression enabled
CREATE OR REPLACE FUNCTION @extschema@.chunk_compression_stats (hypertable REGCLASS)
RETURNS TABLE (
chunk_schema name,
chunk_name name,
compression_status text,
before_compression_table_bytes bigint,
before_compression_index_bytes bigint,
before_compression_toast_bytes bigint,
before_compression_total_bytes bigint,
after_compression_table_bytes bigint,
after_compression_index_bytes bigint,
after_compression_toast_bytes bigint,
after_compression_total_bytes bigint,
node_name name)
LANGUAGE PLPGSQL
STABLE STRICT
AS $BODY$
DECLARE
table_name name;
schema_name name;
is_distributed bool;
BEGIN
SELECT
relname,
nspname,
replication_factor > 0
INTO
table_name,
schema_name,
is_distributed
FROM
pg_class c
INNER JOIN pg_namespace n ON (n.OID = c.relnamespace)
INNER JOIN _timescaledb_catalog.hypertable ht ON (ht.schema_name = n.nspname
AND ht.table_name = c.relname)
WHERE
c.OID = hypertable;
IF table_name IS NULL THEN
RETURN;
END IF;
CASE WHEN is_distributed THEN
RETURN QUERY
SELECT
*
FROM
_timescaledb_functions.compressed_chunk_remote_stats(schema_name, table_name);
ELSE
RETURN QUERY
SELECT
*,
NULL::name
FROM
_timescaledb_functions.compressed_chunk_local_stats(schema_name, table_name);
END CASE;
END;
$BODY$ SET search_path TO pg_catalog, pg_temp;
-- Get compression statistics for a hypertable that has
-- compression enabled
CREATE OR REPLACE FUNCTION @extschema@.hypertable_compression_stats (hypertable REGCLASS)
RETURNS TABLE (
total_chunks bigint,
number_compressed_chunks bigint,
before_compression_table_bytes bigint,
before_compression_index_bytes bigint,
before_compression_toast_bytes bigint,
before_compression_total_bytes bigint,
after_compression_table_bytes bigint,
after_compression_index_bytes bigint,
after_compression_toast_bytes bigint,
after_compression_total_bytes bigint,
node_name name)
LANGUAGE SQL
STABLE STRICT
AS
$BODY$
SELECT
count(*)::bigint AS total_chunks,
(count(*) FILTER (WHERE ch.compression_status = 'Compressed'))::bigint AS number_compressed_chunks,
sum(ch.before_compression_table_bytes)::bigint AS before_compression_table_bytes,
sum(ch.before_compression_index_bytes)::bigint AS before_compression_index_bytes,
sum(ch.before_compression_toast_bytes)::bigint AS before_compression_toast_bytes,
sum(ch.before_compression_total_bytes)::bigint AS before_compression_total_bytes,
sum(ch.after_compression_table_bytes)::bigint AS after_compression_table_bytes,
sum(ch.after_compression_index_bytes)::bigint AS after_compression_index_bytes,
sum(ch.after_compression_toast_bytes)::bigint AS after_compression_toast_bytes,
sum(ch.after_compression_total_bytes)::bigint AS after_compression_total_bytes,
ch.node_name
FROM
@extschema@.chunk_compression_stats(hypertable) ch
GROUP BY
ch.node_name;
$BODY$ SET search_path TO pg_catalog, pg_temp;
-------------Get index size for hypertables -------
--schema_name - schema_name for hypertable index
-- index_name - index on hyper table
---note that the query matches against the hypertable's schema name as
-- the input is on the hypertable index and not the chunk index.
CREATE OR REPLACE FUNCTION _timescaledb_functions.indexes_local_size(
schema_name_in NAME,
index_name_in NAME
)
RETURNS TABLE ( hypertable_id INTEGER,
total_bytes BIGINT )
LANGUAGE SQL VOLATILE STRICT AS
$BODY$
WITH chunk_index_size (num_bytes) AS (
SELECT
COALESCE(sum(pg_relation_size(c.oid)), 0)::bigint
FROM
pg_class c,
pg_namespace n,
_timescaledb_catalog.chunk ch,
_timescaledb_catalog.chunk_index ci,
_timescaledb_catalog.hypertable h
WHERE ch.schema_name = n.nspname
AND c.relnamespace = n.oid
AND c.relname = ci.index_name
AND ch.id = ci.chunk_id
AND h.id = ci.hypertable_id
AND h.schema_name = schema_name_in
AND ci.hypertable_index_name = index_name_in
) SELECT
h.id,
-- Add size of index on all chunks + index size on root table
(SELECT num_bytes FROM chunk_index_size) + pg_relation_size(format('%I.%I', schema_name_in, index_name_in)::regclass)::bigint
FROM
pg_class c, pg_index i, _timescaledb_catalog.hypertable h
WHERE
i.indexrelid = format('%I.%I', schema_name_in, index_name_in)::regclass
AND c.oid = i.indrelid
AND h.schema_name = schema_name_in
AND h.table_name = c.relname;
$BODY$ SET search_path TO pg_catalog, pg_temp;
CREATE OR REPLACE FUNCTION _timescaledb_functions.data_node_index_size(node_name name, schema_name_in name, index_name_in name)
RETURNS TABLE ( hypertable_id INTEGER, total_bytes BIGINT)
AS '@MODULE_PATHNAME@' , 'ts_dist_remote_hypertable_index_info' LANGUAGE C VOLATILE STRICT;
CREATE OR REPLACE FUNCTION _timescaledb_functions.indexes_remote_size(
schema_name_in NAME,
table_name_in NAME,
index_name_in NAME
)
RETURNS BIGINT
LANGUAGE SQL VOLATILE STRICT AS
$BODY$
SELECT
sum(entry.total_bytes)::bigint AS total_bytes
FROM (
SELECT
s.node_name
FROM
_timescaledb_catalog.hypertable AS ht,
_timescaledb_catalog.hypertable_data_node AS s
WHERE
ht.schema_name = schema_name_in
AND ht.table_name = table_name_in
AND s.hypertable_id = ht.id
) AS srv
JOIN LATERAL _timescaledb_functions.data_node_index_size(
srv.node_name, schema_name_in, index_name_in) entry ON TRUE;
$BODY$ SET search_path TO pg_catalog, pg_temp;
-- Get sizes of indexes on a hypertable
--
-- index_name - index on hyper table
--
-- Returns:
-- total_bytes - size of index on disk
CREATE OR REPLACE FUNCTION @extschema@.hypertable_index_size(
index_name REGCLASS
)
RETURNS BIGINT
LANGUAGE PLPGSQL VOLATILE STRICT AS
$BODY$
DECLARE
ht_index_name NAME;
ht_schema_name NAME;
ht_name NAME;
is_distributed BOOL;
ht_id INTEGER;
index_bytes BIGINT;
BEGIN
SELECT c.relname, cl.relname, nsp.nspname, ht.replication_factor > 0
INTO ht_index_name, ht_name, ht_schema_name, is_distributed
FROM pg_class c, pg_index cind, pg_class cl,
pg_namespace nsp, _timescaledb_catalog.hypertable ht
WHERE c.oid = cind.indexrelid AND cind.indrelid = cl.oid
AND cl.relnamespace = nsp.oid AND c.oid = index_name
AND ht.schema_name = nsp.nspname ANd ht.table_name = cl.relname;
IF ht_index_name IS NULL THEN
RETURN NULL;
END IF;
-- get the local size or size of access node indexes
SELECT il.total_bytes
INTO index_bytes
FROM _timescaledb_functions.indexes_local_size(ht_schema_name, ht_index_name) il;
IF index_bytes IS NULL THEN
index_bytes = 0;
END IF;
-- Add size from data nodes
IF is_distributed THEN
index_bytes = index_bytes + _timescaledb_functions.indexes_remote_size(ht_schema_name, ht_name, ht_index_name);
END IF;
RETURN index_bytes;
END;
$BODY$ SET search_path TO pg_catalog, pg_temp;
-------------End index size for hypertables -------