mirror of
https://github.com/timescale/timescaledb.git
synced 2025-05-17 19:13:16 +08:00
Support pushing partial aggregates to data nodes
This change modifies the timescale_fdw to allow aggregates on a subset of partitioning dimensions to be pushed down to data node. The deparsing code has also been modified to wrap the pushed aggregate in a _timescaledb_internal.partialize_agg call, which will return the proper array of values which postgres can combine into the finalized value.
This commit is contained in:
parent
112a7cf8d0
commit
fd75dec223
@ -87,6 +87,7 @@
|
||||
#include "timescaledb_fdw.h"
|
||||
#include "extension_constants.h"
|
||||
#include "plan_expand_hypertable.h"
|
||||
#include "partialize_finalize.h"
|
||||
|
||||
/*
|
||||
* Global context for foreign_expr_walker's search of an expression tree.
|
||||
@ -755,10 +756,6 @@ foreign_expr_walker(Node *node, foreign_glob_cxt *glob_cxt, foreign_loc_cxt *out
|
||||
if (!IS_UPPER_REL(glob_cxt->foreignrel))
|
||||
return false;
|
||||
|
||||
/* Only non-split aggregates are pushable. */
|
||||
if (agg->aggsplit != AGGSPLIT_SIMPLE)
|
||||
return false;
|
||||
|
||||
/* As usual, it must be shippable. */
|
||||
if (!is_shippable(agg->aggfnoid, ProcedureRelationId, fpinfo))
|
||||
return false;
|
||||
@ -2762,14 +2759,16 @@ deparseAggref(Aggref *node, deparse_expr_cxt *context)
|
||||
{
|
||||
StringInfo buf = context->buf;
|
||||
bool use_variadic;
|
||||
|
||||
/* Only basic, non-split aggregation accepted. */
|
||||
Assert(node->aggsplit == AGGSPLIT_SIMPLE);
|
||||
bool partial_agg = node->aggsplit != AGGSPLIT_SIMPLE;
|
||||
Assert(node->aggsplit == AGGSPLIT_SIMPLE || node->aggsplit == AGGSPLIT_INITIAL_SERIAL);
|
||||
|
||||
/* Check if need to print VARIADIC (cf. ruleutils.c) */
|
||||
use_variadic = node->aggvariadic;
|
||||
|
||||
/* Find aggregate name from aggfnoid which is a pg_proc entry */
|
||||
if (partial_agg)
|
||||
appendStringInfoString(buf, INTERNAL_SCHEMA_NAME "." PARTIALIZE_FUNC_NAME "(");
|
||||
|
||||
appendFunctionName(node->aggfnoid, context);
|
||||
appendStringInfoChar(buf, '(');
|
||||
|
||||
@ -2843,7 +2842,7 @@ deparseAggref(Aggref *node, deparse_expr_cxt *context)
|
||||
deparseExpr((Expr *) node->aggfilter, context);
|
||||
}
|
||||
|
||||
appendStringInfoChar(buf, ')');
|
||||
appendStringInfoString(buf, partial_agg ? "))" : ")");
|
||||
}
|
||||
|
||||
/*
|
||||
|
@ -17,6 +17,7 @@
|
||||
#include <optimizer/restrictinfo.h>
|
||||
#include <access/sysattr.h>
|
||||
#include <utils/memutils.h>
|
||||
#include <foreign/fdwapi.h>
|
||||
|
||||
#include <export.h>
|
||||
#include <chunk_server.h>
|
||||
@ -116,6 +117,15 @@ build_server_rel(PlannerInfo *root, Index relid, Oid serverid, RelOptInfo *paren
|
||||
rel->lateral_relids = parent->lateral_relids;
|
||||
rel->serverid = serverid;
|
||||
|
||||
/* We need to use the FDW interface to get called by the planner for
|
||||
* partial aggs. For some reason, the standard upper_paths_hook is never
|
||||
* called for upper rels of type UPPERREL_PARTIAL_GROUP_AGG, which is odd
|
||||
* (see end of PostgreSQL planner.c:create_partial_grouping_paths). Until
|
||||
* this gets fixed in the PostgreSQL planner, we're forced to set
|
||||
* fdwroutine here although we will scan this rel with a ServerScan and
|
||||
* not a ForeignScan. */
|
||||
rel->fdwroutine = GetFdwRoutineByServerId(serverid);
|
||||
|
||||
return rel;
|
||||
}
|
||||
|
||||
|
@ -31,6 +31,7 @@
|
||||
#include <utils/selfuncs.h>
|
||||
#include <commands/defrem.h>
|
||||
#include <commands/explain.h>
|
||||
#include <commands/extension.h>
|
||||
#include <rewrite/rewriteManip.h>
|
||||
#include <miscadmin.h>
|
||||
#include <libpq-fe.h>
|
||||
@ -66,6 +67,7 @@
|
||||
#include "remote/data_format.h"
|
||||
#include "guc.h"
|
||||
#include "server_scan_plan.h"
|
||||
#include "extension_constants.h"
|
||||
|
||||
/* Default CPU cost to start up a foreign query. */
|
||||
#define DEFAULT_FDW_STARTUP_COST 100.0
|
||||
@ -1106,6 +1108,8 @@ get_foreign_paths(PlannerInfo *root, RelOptInfo *baserel, Oid foreigntableid)
|
||||
TsFdwRelationInfo *fpinfo = fdw_relation_info_get(baserel);
|
||||
Path *path;
|
||||
|
||||
Assert(fpinfo->type != TS_FDW_RELATION_INFO_HYPERTABLE_SERVER);
|
||||
|
||||
if (fpinfo->type == TS_FDW_RELATION_INFO_HYPERTABLE)
|
||||
{
|
||||
if (ts_guc_enable_per_server_queries)
|
||||
@ -2919,9 +2923,6 @@ add_foreign_grouping_paths(PlannerInfo *root, RelOptInfo *input_rel, RelOptInfo
|
||||
if (!parse->groupClause && !parse->groupingSets && !parse->hasAggs && !root->hasHavingQual)
|
||||
return;
|
||||
|
||||
Assert(extra->patype == PARTITIONWISE_AGGREGATE_NONE ||
|
||||
extra->patype == PARTITIONWISE_AGGREGATE_FULL);
|
||||
|
||||
/* save the input_rel as outerrel in fpinfo */
|
||||
fpinfo->outerrel = input_rel;
|
||||
|
||||
@ -2985,7 +2986,8 @@ fdw_create_upper_paths(TsFdwRelationInfo *input_fpinfo, PlannerInfo *root, Upper
|
||||
/* Ignore stages we don't support; and skip any duplicate calls (i.e.,
|
||||
* output_rel->fdw_private has already been set by a previous call to this
|
||||
* function). */
|
||||
if (stage != UPPERREL_GROUP_AGG || output_rel->fdw_private)
|
||||
if ((stage != UPPERREL_GROUP_AGG && stage != UPPERREL_PARTIAL_GROUP_AGG) ||
|
||||
output_rel->fdw_private)
|
||||
return;
|
||||
|
||||
input_fpinfo = fdw_relation_info_alloc(output_rel, input_fpinfo->type);
|
||||
@ -3034,6 +3036,13 @@ get_foreign_upper_paths(PlannerInfo *root, UpperRelationKind stage, RelOptInfo *
|
||||
if (fpinfo == NULL)
|
||||
return;
|
||||
|
||||
/* We abuse the FDW API's GetForeignUpperPaths callback because, for some
|
||||
* reason, the regular create_upper_paths_hook is never called for
|
||||
* partially grouped rels, so we cannot use if for server rels. See end of
|
||||
* PostgreSQL planner.c:create_partial_grouping_paths(). */
|
||||
if (fpinfo->type == TS_FDW_RELATION_INFO_HYPERTABLE_SERVER)
|
||||
return server_scan_create_upper_paths(root, stage, input_rel, output_rel, extra);
|
||||
|
||||
return fdw_create_upper_paths(fpinfo,
|
||||
root,
|
||||
stage,
|
||||
|
@ -12,8 +12,10 @@
|
||||
|
||||
#include "compat.h"
|
||||
|
||||
Datum tsl_finalize_agg_sfunc(PG_FUNCTION_ARGS);
|
||||
Datum tsl_finalize_agg_ffunc(PG_FUNCTION_ARGS);
|
||||
Datum tsl_partialize_agg(PG_FUNCTION_ARGS);
|
||||
#define PARTIALIZE_FUNC_NAME "partialize_agg"
|
||||
|
||||
extern Datum tsl_finalize_agg_sfunc(PG_FUNCTION_ARGS);
|
||||
extern Datum tsl_finalize_agg_ffunc(PG_FUNCTION_ARGS);
|
||||
extern Datum tsl_partialize_agg(PG_FUNCTION_ARGS);
|
||||
|
||||
#endif
|
||||
|
461
tsl/test/expected/dist_partial_agg.out
Normal file
461
tsl/test/expected/dist_partial_agg.out
Normal file
@ -0,0 +1,461 @@
|
||||
-- This file and its contents are licensed under the Timescale License.
|
||||
-- Please see the included NOTICE for copyright information and
|
||||
-- LICENSE-TIMESCALE for a copy of the license.
|
||||
-- Need to be super user to create extension and add servers
|
||||
\c :TEST_DBNAME :ROLE_SUPERUSER;
|
||||
-- Need explicit password for non-super users to connect
|
||||
ALTER ROLE :ROLE_DEFAULT_CLUSTER_USER CREATEDB PASSWORD 'pass';
|
||||
GRANT USAGE ON FOREIGN DATA WRAPPER timescaledb_fdw TO :ROLE_DEFAULT_CLUSTER_USER;
|
||||
SET ROLE :ROLE_DEFAULT_CLUSTER_USER;
|
||||
-- Cleanup from other potential tests that created these databases
|
||||
SET client_min_messages TO ERROR;
|
||||
DROP DATABASE IF EXISTS server_1;
|
||||
DROP DATABASE IF EXISTS server_2;
|
||||
DROP DATABASE IF EXISTS server_3;
|
||||
SET client_min_messages TO NOTICE;
|
||||
CREATE DATABASE server_1;
|
||||
CREATE DATABASE server_2;
|
||||
CREATE DATABASE server_3;
|
||||
\c server_1
|
||||
SET client_min_messages TO ERROR;
|
||||
CREATE EXTENSION timescaledb;
|
||||
CREATE TYPE custom_type AS (high int, low int);
|
||||
\c server_2
|
||||
SET client_min_messages TO ERROR;
|
||||
CREATE EXTENSION timescaledb;
|
||||
CREATE TYPE custom_type AS (high int, low int);
|
||||
\c server_3
|
||||
SET client_min_messages TO ERROR;
|
||||
CREATE EXTENSION timescaledb;
|
||||
CREATE TYPE custom_type AS (high int, low int);
|
||||
\c :TEST_DBNAME :ROLE_SUPERUSER;
|
||||
SET ROLE :ROLE_DEFAULT_CLUSTER_USER;
|
||||
\set TEST_TABLE 'conditions'
|
||||
\ir 'include/aggregate_table_create.sql'
|
||||
-- This file and its contents are licensed under the Timescale License.
|
||||
-- Please see the included NOTICE for copyright information and
|
||||
-- LICENSE-TIMESCALE for a copy of the license.
|
||||
-- This file creates a table with a lot of different types to allow a range of aggregate functions.
|
||||
-- This does not include the creation of a corresponding hypertable, as we may want to vary how that is done.
|
||||
CREATE TYPE custom_type AS (high int, low int);
|
||||
CREATE TABLE :TEST_TABLE (
|
||||
timec TIMESTAMPTZ NOT NULL,
|
||||
location TEXT NOT NULL,
|
||||
region TEXT NOT NULL,
|
||||
temperature DOUBLE PRECISION NULL,
|
||||
humidity DOUBLE PRECISION NULL,
|
||||
lowp double precision NULL,
|
||||
highp double precision null,
|
||||
allnull double precision null,
|
||||
highlow custom_type null,
|
||||
bit_int smallint,
|
||||
good_life boolean
|
||||
);
|
||||
-- Add servers using the TimescaleDB server management API
|
||||
SELECT * FROM add_server('server_1', database => 'server_1', password => 'pass', if_not_exists => true);
|
||||
NOTICE: remote server database "server_1" already exists, skipping
|
||||
server_name | host | port | database | username | server_username | created
|
||||
-------------+-----------+-------+----------+----------------------+----------------------+---------
|
||||
server_1 | localhost | 15432 | server_1 | default_cluster_user | default_cluster_user | t
|
||||
(1 row)
|
||||
|
||||
SELECT * FROM add_server('server_2', database => 'server_2', password => 'pass', if_not_exists => true);
|
||||
NOTICE: remote server database "server_2" already exists, skipping
|
||||
server_name | host | port | database | username | server_username | created
|
||||
-------------+-----------+-------+----------+----------------------+----------------------+---------
|
||||
server_2 | localhost | 15432 | server_2 | default_cluster_user | default_cluster_user | t
|
||||
(1 row)
|
||||
|
||||
SELECT * FROM add_server('server_3', database => 'server_3', password => 'pass', if_not_exists => true);
|
||||
NOTICE: remote server database "server_3" already exists, skipping
|
||||
server_name | host | port | database | username | server_username | created
|
||||
-------------+-----------+-------+----------+----------------------+----------------------+---------
|
||||
server_3 | localhost | 15432 | server_3 | default_cluster_user | default_cluster_user | t
|
||||
(1 row)
|
||||
|
||||
SELECT table_name FROM create_distributed_hypertable( 'conditions', 'timec', 'location', 3, chunk_time_interval => INTERVAL '1 day');
|
||||
table_name
|
||||
------------
|
||||
conditions
|
||||
(1 row)
|
||||
|
||||
-- We need a lot of data and a lot of chunks to make the planner push down all of the aggregates
|
||||
\ir 'include/aggregate_table_populate.sql'
|
||||
-- This file and its contents are licensed under the Timescale License.
|
||||
-- Please see the included NOTICE for copyright information and
|
||||
-- LICENSE-TIMESCALE for a copy of the license.
|
||||
-- This files assumes the existence of some table with definition as seen in the aggregate_table.sql file.
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'POR', 'west', generate_series(25, 85, 0.0625), 75, 40, 70, NULL, (1,2)::custom_type, 2, true;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'SFO', 'west', generate_series(25, 85, 0.0625), 75, 40, 70, NULL, (1,2)::custom_type, 2, true;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'SAC', 'west', generate_series(25, 85, 0.0625), 75, 40, 70, NULL, (1,2)::custom_type, 2, true;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'SEA', 'west', generate_series(25, 85, 0.0625), 75, 40, 70, NULL, (1,2)::custom_type, 2, true;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'TAC', 'west', generate_series(25, 85, 0.0625), 75, 40, 70, NULL, (1,2)::custom_type, 2, true;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'NYC', 'north-east', generate_series(29, 41, 0.0125), 45, 50, 40, NULL, (3,4)::custom_type, 4, false;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'BOS', 'north-east', generate_series(29, 41, 0.0125), 45, 50, 40, NULL, (3,4)::custom_type, 4, false;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'CHI', 'midwest', generate_series(29, 41, 0.0125), 45, 50, 40, NULL, (3,4)::custom_type, 4, false;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'MIN', 'midwest', generate_series(29, 41, 0.0125), 45, 50, 40, NULL, (3,4)::custom_type, 4, false;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'DET', 'midwest', generate_series(29, 41, 0.0125), 45, 50, 40, NULL, (3,4)::custom_type, 4, false;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'LA', 'west', generate_series(61, 85, 0.025), 55, NULL, 28, NULL, NULL, 8, true;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'SDG', 'west', generate_series(61, 85, 0.025), 55, NULL, 28, NULL, NULL, 8, true;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'PHX', 'west', generate_series(61, 85, 0.025), 55, NULL, 28, NULL, NULL, 8, true;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'DAL', 'south', generate_series(61, 85, 0.025), 55, NULL, 28, NULL, NULL, 8, true;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'AUS', 'south', generate_series(61, 85, 0.025), 55, NULL, 28, NULL, NULL, 8, true;
|
||||
SET enable_partitionwise_aggregate = ON;
|
||||
-- Run an explain on the aggregate queries to make sure expected aggregates are being pushed down.
|
||||
-- Grouping by the paritioning column should result in full aggregate pushdown where possible,
|
||||
-- while using a non-partitioning column should result in a partial pushdown
|
||||
\set PREFIX 'EXPLAIN (VERBOSE, COSTS OFF)'
|
||||
\set GROUPING 'location'
|
||||
\ir 'include/aggregate_queries.sql'
|
||||
-- This file and its contents are licensed under the Timescale License.
|
||||
-- Please see the included NOTICE for copyright information and
|
||||
-- LICENSE-TIMESCALE for a copy of the license.
|
||||
-- This files assumes the existence of some table with definition as seen in the aggregate_table.sql file.
|
||||
-- All of these should be able to be pushed down if enabled
|
||||
:PREFIX SELECT :GROUPING,
|
||||
min(allnull) as min_allnull,
|
||||
max(temperature) as max_temp,
|
||||
sum(temperature)+sum(humidity) as agg_sum_expr,
|
||||
avg(humidity),
|
||||
ROUND( CAST(stddev(humidity) AS NUMERIC), 1),
|
||||
bit_and(bit_int),
|
||||
bit_or(bit_int),
|
||||
bool_and(good_life),
|
||||
every(temperature > 0),
|
||||
bool_or(good_life),
|
||||
count(*) as count_rows,
|
||||
count(temperature) as count_temp,
|
||||
count(allnull) as count_zero,
|
||||
ROUND( CAST(corr(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(covar_pop(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(covar_samp(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(regr_avgx(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(regr_avgy(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(regr_count(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(regr_intercept(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(regr_r2(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(regr_slope(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(regr_sxx(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(regr_sxy(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(regr_syy(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(stddev(temperature) AS NUMERIC), 1) as stddev_temp,
|
||||
ROUND( CAST(stddev_pop(temperature) AS NUMERIC), 1),
|
||||
ROUND( CAST(stddev_samp(temperature) AS NUMERIC), 1),
|
||||
ROUND( CAST(variance(temperature) AS NUMERIC), 1),
|
||||
ROUND( CAST(var_pop(temperature) AS NUMERIC), 1),
|
||||
ROUND( CAST(var_samp(temperature) AS NUMERIC), 1),
|
||||
last(temperature, timec) as last_temp,
|
||||
histogram(temperature, 0, 100, 1)
|
||||
FROM :TEST_TABLE
|
||||
GROUP BY :GROUPING
|
||||
ORDER BY :GROUPING;
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
Sort
|
||||
Output: conditions.location, (min(conditions.allnull)), (max(conditions.temperature)), ((sum(conditions.temperature) + sum(conditions.humidity))), (avg(conditions.humidity)), (round((stddev(conditions.humidity))::numeric, 1)), (bit_and(conditions.bit_int)), (bit_or(conditions.bit_int)), (bool_and(conditions.good_life)), (every((conditions.temperature > '0'::double precision))), (bool_or(conditions.good_life)), (count(*)), (count(conditions.temperature)), (count(conditions.allnull)), (round((corr(conditions.temperature, conditions.humidity))::numeric, 1)), (round((covar_pop(conditions.temperature, conditions.humidity))::numeric, 1)), (round((covar_samp(conditions.temperature, conditions.humidity))::numeric, 1)), (round((regr_avgx(conditions.temperature, conditions.humidity))::numeric, 1)), (round((regr_avgy(conditions.temperature, conditions.humidity))::numeric, 1)), (round((regr_count(conditions.temperature, conditions.humidity))::numeric, 1)), (round((regr_intercept(conditions.temperature, conditions.humidity))::numeric, 1)), (round((regr_r2(conditions.temperature, conditions.humidity))::numeric, 1)), (round((regr_slope(conditions.temperature, conditions.humidity))::numeric, 1)), (round((regr_sxx(conditions.temperature, conditions.humidity))::numeric, 1)), (round((regr_sxy(conditions.temperature, conditions.humidity))::numeric, 1)), (round((regr_syy(conditions.temperature, conditions.humidity))::numeric, 1)), (round((stddev(conditions.temperature))::numeric, 1)), (round((stddev_pop(conditions.temperature))::numeric, 1)), (round((stddev_samp(conditions.temperature))::numeric, 1)), (round((variance(conditions.temperature))::numeric, 1)), (round((var_pop(conditions.temperature))::numeric, 1)), (round((var_samp(conditions.temperature))::numeric, 1)), (last(conditions.temperature, conditions.timec)), (histogram(conditions.temperature, '0'::double precision, '100'::double precision, 1))
|
||||
Sort Key: conditions.location
|
||||
-> Append
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: conditions.location, (min(conditions.allnull)), (max(conditions.temperature)), ((sum(conditions.temperature) + sum(conditions.humidity))), (avg(conditions.humidity)), (round((stddev(conditions.humidity))::numeric, 1)), (bit_and(conditions.bit_int)), (bit_or(conditions.bit_int)), (bool_and(conditions.good_life)), (every((conditions.temperature > '0'::double precision))), (bool_or(conditions.good_life)), (count(*)), (count(conditions.temperature)), (count(conditions.allnull)), (round((corr(conditions.temperature, conditions.humidity))::numeric, 1)), (round((covar_pop(conditions.temperature, conditions.humidity))::numeric, 1)), (round((covar_samp(conditions.temperature, conditions.humidity))::numeric, 1)), (round((regr_avgx(conditions.temperature, conditions.humidity))::numeric, 1)), (round((regr_avgy(conditions.temperature, conditions.humidity))::numeric, 1)), (round((regr_count(conditions.temperature, conditions.humidity))::numeric, 1)), (round((regr_intercept(conditions.temperature, conditions.humidity))::numeric, 1)), (round((regr_r2(conditions.temperature, conditions.humidity))::numeric, 1)), (round((regr_slope(conditions.temperature, conditions.humidity))::numeric, 1)), (round((regr_sxx(conditions.temperature, conditions.humidity))::numeric, 1)), (round((regr_sxy(conditions.temperature, conditions.humidity))::numeric, 1)), (round((regr_syy(conditions.temperature, conditions.humidity))::numeric, 1)), (round((stddev(conditions.temperature))::numeric, 1)), (round((stddev_pop(conditions.temperature))::numeric, 1)), (round((stddev_samp(conditions.temperature))::numeric, 1)), (round((variance(conditions.temperature))::numeric, 1)), (round((var_pop(conditions.temperature))::numeric, 1)), (round((var_samp(conditions.temperature))::numeric, 1)), (last(conditions.temperature, conditions.timec)), (histogram(conditions.temperature, '0'::double precision, '100'::double precision, 1))
|
||||
Relations: Aggregate on (public.conditions)
|
||||
Server: server_1
|
||||
Chunks: _hyper_1_1_dist_chunk, _hyper_1_2_dist_chunk, _hyper_1_3_dist_chunk, _hyper_1_4_dist_chunk
|
||||
Remote SQL: SELECT location, min(allnull), max(temperature), (sum(temperature) + sum(humidity)), avg(humidity), round(stddev(humidity)::numeric, 1), bit_and(bit_int), bit_or(bit_int), bool_and(good_life), every((temperature > 0::double precision)), bool_or(good_life), count(*), count(temperature), count(allnull), round(corr(temperature, humidity)::numeric, 1), round(covar_pop(temperature, humidity)::numeric, 1), round(covar_samp(temperature, humidity)::numeric, 1), round(regr_avgx(temperature, humidity)::numeric, 1), round(regr_avgy(temperature, humidity)::numeric, 1), round(regr_count(temperature, humidity)::numeric, 1), round(regr_intercept(temperature, humidity)::numeric, 1), round(regr_r2(temperature, humidity)::numeric, 1), round(regr_slope(temperature, humidity)::numeric, 1), round(regr_sxx(temperature, humidity)::numeric, 1), round(regr_sxy(temperature, humidity)::numeric, 1), round(regr_syy(temperature, humidity)::numeric, 1), round(stddev(temperature)::numeric, 1), round(stddev_pop(temperature)::numeric, 1), round(stddev_samp(temperature)::numeric, 1), round(variance(temperature)::numeric, 1), round(var_pop(temperature)::numeric, 1), round(var_samp(temperature)::numeric, 1), public.last(temperature, timec), public.histogram(temperature, 0::double precision, 100::double precision, 1) FROM public.conditions WHERE _timescaledb_internal.chunks_in(conditions, ARRAY[1, 2, 3, 4]) GROUP BY 1
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: conditions_1.location, (min(conditions_1.allnull)), (max(conditions_1.temperature)), ((sum(conditions_1.temperature) + sum(conditions_1.humidity))), (avg(conditions_1.humidity)), (round((stddev(conditions_1.humidity))::numeric, 1)), (bit_and(conditions_1.bit_int)), (bit_or(conditions_1.bit_int)), (bool_and(conditions_1.good_life)), (every((conditions_1.temperature > '0'::double precision))), (bool_or(conditions_1.good_life)), (count(*)), (count(conditions_1.temperature)), (count(conditions_1.allnull)), (round((corr(conditions_1.temperature, conditions_1.humidity))::numeric, 1)), (round((covar_pop(conditions_1.temperature, conditions_1.humidity))::numeric, 1)), (round((covar_samp(conditions_1.temperature, conditions_1.humidity))::numeric, 1)), (round((regr_avgx(conditions_1.temperature, conditions_1.humidity))::numeric, 1)), (round((regr_avgy(conditions_1.temperature, conditions_1.humidity))::numeric, 1)), (round((regr_count(conditions_1.temperature, conditions_1.humidity))::numeric, 1)), (round((regr_intercept(conditions_1.temperature, conditions_1.humidity))::numeric, 1)), (round((regr_r2(conditions_1.temperature, conditions_1.humidity))::numeric, 1)), (round((regr_slope(conditions_1.temperature, conditions_1.humidity))::numeric, 1)), (round((regr_sxx(conditions_1.temperature, conditions_1.humidity))::numeric, 1)), (round((regr_sxy(conditions_1.temperature, conditions_1.humidity))::numeric, 1)), (round((regr_syy(conditions_1.temperature, conditions_1.humidity))::numeric, 1)), (round((stddev(conditions_1.temperature))::numeric, 1)), (round((stddev_pop(conditions_1.temperature))::numeric, 1)), (round((stddev_samp(conditions_1.temperature))::numeric, 1)), (round((variance(conditions_1.temperature))::numeric, 1)), (round((var_pop(conditions_1.temperature))::numeric, 1)), (round((var_samp(conditions_1.temperature))::numeric, 1)), (last(conditions_1.temperature, conditions_1.timec)), (histogram(conditions_1.temperature, '0'::double precision, '100'::double precision, 1))
|
||||
Relations: Aggregate on (public.conditions)
|
||||
Server: server_2
|
||||
Chunks: _hyper_1_9_dist_chunk, _hyper_1_10_dist_chunk, _hyper_1_11_dist_chunk, _hyper_1_12_dist_chunk
|
||||
Remote SQL: SELECT location, min(allnull), max(temperature), (sum(temperature) + sum(humidity)), avg(humidity), round(stddev(humidity)::numeric, 1), bit_and(bit_int), bit_or(bit_int), bool_and(good_life), every((temperature > 0::double precision)), bool_or(good_life), count(*), count(temperature), count(allnull), round(corr(temperature, humidity)::numeric, 1), round(covar_pop(temperature, humidity)::numeric, 1), round(covar_samp(temperature, humidity)::numeric, 1), round(regr_avgx(temperature, humidity)::numeric, 1), round(regr_avgy(temperature, humidity)::numeric, 1), round(regr_count(temperature, humidity)::numeric, 1), round(regr_intercept(temperature, humidity)::numeric, 1), round(regr_r2(temperature, humidity)::numeric, 1), round(regr_slope(temperature, humidity)::numeric, 1), round(regr_sxx(temperature, humidity)::numeric, 1), round(regr_sxy(temperature, humidity)::numeric, 1), round(regr_syy(temperature, humidity)::numeric, 1), round(stddev(temperature)::numeric, 1), round(stddev_pop(temperature)::numeric, 1), round(stddev_samp(temperature)::numeric, 1), round(variance(temperature)::numeric, 1), round(var_pop(temperature)::numeric, 1), round(var_samp(temperature)::numeric, 1), public.last(temperature, timec), public.histogram(temperature, 0::double precision, 100::double precision, 1) FROM public.conditions WHERE _timescaledb_internal.chunks_in(conditions, ARRAY[1, 2, 3, 4]) GROUP BY 1
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: conditions_2.location, (min(conditions_2.allnull)), (max(conditions_2.temperature)), ((sum(conditions_2.temperature) + sum(conditions_2.humidity))), (avg(conditions_2.humidity)), (round((stddev(conditions_2.humidity))::numeric, 1)), (bit_and(conditions_2.bit_int)), (bit_or(conditions_2.bit_int)), (bool_and(conditions_2.good_life)), (every((conditions_2.temperature > '0'::double precision))), (bool_or(conditions_2.good_life)), (count(*)), (count(conditions_2.temperature)), (count(conditions_2.allnull)), (round((corr(conditions_2.temperature, conditions_2.humidity))::numeric, 1)), (round((covar_pop(conditions_2.temperature, conditions_2.humidity))::numeric, 1)), (round((covar_samp(conditions_2.temperature, conditions_2.humidity))::numeric, 1)), (round((regr_avgx(conditions_2.temperature, conditions_2.humidity))::numeric, 1)), (round((regr_avgy(conditions_2.temperature, conditions_2.humidity))::numeric, 1)), (round((regr_count(conditions_2.temperature, conditions_2.humidity))::numeric, 1)), (round((regr_intercept(conditions_2.temperature, conditions_2.humidity))::numeric, 1)), (round((regr_r2(conditions_2.temperature, conditions_2.humidity))::numeric, 1)), (round((regr_slope(conditions_2.temperature, conditions_2.humidity))::numeric, 1)), (round((regr_sxx(conditions_2.temperature, conditions_2.humidity))::numeric, 1)), (round((regr_sxy(conditions_2.temperature, conditions_2.humidity))::numeric, 1)), (round((regr_syy(conditions_2.temperature, conditions_2.humidity))::numeric, 1)), (round((stddev(conditions_2.temperature))::numeric, 1)), (round((stddev_pop(conditions_2.temperature))::numeric, 1)), (round((stddev_samp(conditions_2.temperature))::numeric, 1)), (round((variance(conditions_2.temperature))::numeric, 1)), (round((var_pop(conditions_2.temperature))::numeric, 1)), (round((var_samp(conditions_2.temperature))::numeric, 1)), (last(conditions_2.temperature, conditions_2.timec)), (histogram(conditions_2.temperature, '0'::double precision, '100'::double precision, 1))
|
||||
Relations: Aggregate on (public.conditions)
|
||||
Server: server_3
|
||||
Chunks: _hyper_1_5_dist_chunk, _hyper_1_6_dist_chunk, _hyper_1_7_dist_chunk, _hyper_1_8_dist_chunk
|
||||
Remote SQL: SELECT location, min(allnull), max(temperature), (sum(temperature) + sum(humidity)), avg(humidity), round(stddev(humidity)::numeric, 1), bit_and(bit_int), bit_or(bit_int), bool_and(good_life), every((temperature > 0::double precision)), bool_or(good_life), count(*), count(temperature), count(allnull), round(corr(temperature, humidity)::numeric, 1), round(covar_pop(temperature, humidity)::numeric, 1), round(covar_samp(temperature, humidity)::numeric, 1), round(regr_avgx(temperature, humidity)::numeric, 1), round(regr_avgy(temperature, humidity)::numeric, 1), round(regr_count(temperature, humidity)::numeric, 1), round(regr_intercept(temperature, humidity)::numeric, 1), round(regr_r2(temperature, humidity)::numeric, 1), round(regr_slope(temperature, humidity)::numeric, 1), round(regr_sxx(temperature, humidity)::numeric, 1), round(regr_sxy(temperature, humidity)::numeric, 1), round(regr_syy(temperature, humidity)::numeric, 1), round(stddev(temperature)::numeric, 1), round(stddev_pop(temperature)::numeric, 1), round(stddev_samp(temperature)::numeric, 1), round(variance(temperature)::numeric, 1), round(var_pop(temperature)::numeric, 1), round(var_samp(temperature)::numeric, 1), public.last(temperature, timec), public.histogram(temperature, 0::double precision, 100::double precision, 1) FROM public.conditions WHERE _timescaledb_internal.chunks_in(conditions, ARRAY[1, 2, 3, 4]) GROUP BY 1
|
||||
(22 rows)
|
||||
|
||||
-- Aggregates on custom types are not yet pushed down
|
||||
:PREFIX SELECT :GROUPING,
|
||||
last(highlow, timec) as last_hl,
|
||||
first(highlow, timec) as first_hl
|
||||
FROM :TEST_TABLE
|
||||
GROUP BY :GROUPING
|
||||
ORDER BY :GROUPING;
|
||||
QUERY PLAN
|
||||
----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
Merge Append
|
||||
Sort Key: conditions.location
|
||||
-> GroupAggregate
|
||||
Output: conditions.location, last(conditions.highlow, conditions.timec), first(conditions.highlow, conditions.timec)
|
||||
Group Key: conditions.location
|
||||
-> Custom Scan (ServerScan) on public.conditions
|
||||
Output: conditions.location, conditions.highlow, conditions.timec
|
||||
Server: server_1
|
||||
Chunks: _hyper_1_1_dist_chunk, _hyper_1_2_dist_chunk, _hyper_1_3_dist_chunk, _hyper_1_4_dist_chunk
|
||||
Remote SQL: SELECT timec, location, highlow FROM public.conditions WHERE _timescaledb_internal.chunks_in(conditions, ARRAY[1, 2, 3, 4]) ORDER BY location ASC NULLS LAST
|
||||
-> GroupAggregate
|
||||
Output: conditions_1.location, last(conditions_1.highlow, conditions_1.timec), first(conditions_1.highlow, conditions_1.timec)
|
||||
Group Key: conditions_1.location
|
||||
-> Custom Scan (ServerScan) on public.conditions conditions_1
|
||||
Output: conditions_1.location, conditions_1.highlow, conditions_1.timec
|
||||
Server: server_2
|
||||
Chunks: _hyper_1_9_dist_chunk, _hyper_1_10_dist_chunk, _hyper_1_11_dist_chunk, _hyper_1_12_dist_chunk
|
||||
Remote SQL: SELECT timec, location, highlow FROM public.conditions WHERE _timescaledb_internal.chunks_in(conditions, ARRAY[1, 2, 3, 4]) ORDER BY location ASC NULLS LAST
|
||||
-> GroupAggregate
|
||||
Output: conditions_2.location, last(conditions_2.highlow, conditions_2.timec), first(conditions_2.highlow, conditions_2.timec)
|
||||
Group Key: conditions_2.location
|
||||
-> Custom Scan (ServerScan) on public.conditions conditions_2
|
||||
Output: conditions_2.location, conditions_2.highlow, conditions_2.timec
|
||||
Server: server_3
|
||||
Chunks: _hyper_1_5_dist_chunk, _hyper_1_6_dist_chunk, _hyper_1_7_dist_chunk, _hyper_1_8_dist_chunk
|
||||
Remote SQL: SELECT timec, location, highlow FROM public.conditions WHERE _timescaledb_internal.chunks_in(conditions, ARRAY[1, 2, 3, 4]) ORDER BY location ASC NULLS LAST
|
||||
(26 rows)
|
||||
|
||||
-- Mix of aggregates that push down and those that don't
|
||||
:PREFIX SELECT :GROUPING,
|
||||
min(allnull) as min_allnull,
|
||||
max(temperature) as max_temp,
|
||||
sum(temperature)+sum(humidity) as agg_sum_expr,
|
||||
avg(humidity),
|
||||
ROUND( CAST(stddev(humidity) AS NUMERIC), 1),
|
||||
bit_and(bit_int),
|
||||
bit_or(bit_int),
|
||||
bool_and(good_life),
|
||||
every(temperature > 0),
|
||||
bool_or(good_life),
|
||||
first(highlow, timec) as first_hl
|
||||
FROM :TEST_TABLE
|
||||
GROUP BY :GROUPING
|
||||
ORDER BY :GROUPING;
|
||||
QUERY PLAN
|
||||
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
Merge Append
|
||||
Sort Key: conditions.location
|
||||
-> GroupAggregate
|
||||
Output: conditions.location, min(conditions.allnull), max(conditions.temperature), (sum(conditions.temperature) + sum(conditions.humidity)), avg(conditions.humidity), round((stddev(conditions.humidity))::numeric, 1), bit_and(conditions.bit_int), bit_or(conditions.bit_int), bool_and(conditions.good_life), every((conditions.temperature > '0'::double precision)), bool_or(conditions.good_life), first(conditions.highlow, conditions.timec)
|
||||
Group Key: conditions.location
|
||||
-> Custom Scan (ServerScan) on public.conditions
|
||||
Output: conditions.location, conditions.allnull, conditions.temperature, conditions.humidity, conditions.bit_int, conditions.good_life, conditions.highlow, conditions.timec
|
||||
Server: server_1
|
||||
Chunks: _hyper_1_1_dist_chunk, _hyper_1_2_dist_chunk, _hyper_1_3_dist_chunk, _hyper_1_4_dist_chunk
|
||||
Remote SQL: SELECT timec, location, temperature, humidity, allnull, highlow, bit_int, good_life FROM public.conditions WHERE _timescaledb_internal.chunks_in(conditions, ARRAY[1, 2, 3, 4]) ORDER BY location ASC NULLS LAST
|
||||
-> GroupAggregate
|
||||
Output: conditions_1.location, min(conditions_1.allnull), max(conditions_1.temperature), (sum(conditions_1.temperature) + sum(conditions_1.humidity)), avg(conditions_1.humidity), round((stddev(conditions_1.humidity))::numeric, 1), bit_and(conditions_1.bit_int), bit_or(conditions_1.bit_int), bool_and(conditions_1.good_life), every((conditions_1.temperature > '0'::double precision)), bool_or(conditions_1.good_life), first(conditions_1.highlow, conditions_1.timec)
|
||||
Group Key: conditions_1.location
|
||||
-> Custom Scan (ServerScan) on public.conditions conditions_1
|
||||
Output: conditions_1.location, conditions_1.allnull, conditions_1.temperature, conditions_1.humidity, conditions_1.bit_int, conditions_1.good_life, conditions_1.highlow, conditions_1.timec
|
||||
Server: server_2
|
||||
Chunks: _hyper_1_9_dist_chunk, _hyper_1_10_dist_chunk, _hyper_1_11_dist_chunk, _hyper_1_12_dist_chunk
|
||||
Remote SQL: SELECT timec, location, temperature, humidity, allnull, highlow, bit_int, good_life FROM public.conditions WHERE _timescaledb_internal.chunks_in(conditions, ARRAY[1, 2, 3, 4]) ORDER BY location ASC NULLS LAST
|
||||
-> GroupAggregate
|
||||
Output: conditions_2.location, min(conditions_2.allnull), max(conditions_2.temperature), (sum(conditions_2.temperature) + sum(conditions_2.humidity)), avg(conditions_2.humidity), round((stddev(conditions_2.humidity))::numeric, 1), bit_and(conditions_2.bit_int), bit_or(conditions_2.bit_int), bool_and(conditions_2.good_life), every((conditions_2.temperature > '0'::double precision)), bool_or(conditions_2.good_life), first(conditions_2.highlow, conditions_2.timec)
|
||||
Group Key: conditions_2.location
|
||||
-> Custom Scan (ServerScan) on public.conditions conditions_2
|
||||
Output: conditions_2.location, conditions_2.allnull, conditions_2.temperature, conditions_2.humidity, conditions_2.bit_int, conditions_2.good_life, conditions_2.highlow, conditions_2.timec
|
||||
Server: server_3
|
||||
Chunks: _hyper_1_5_dist_chunk, _hyper_1_6_dist_chunk, _hyper_1_7_dist_chunk, _hyper_1_8_dist_chunk
|
||||
Remote SQL: SELECT timec, location, temperature, humidity, allnull, highlow, bit_int, good_life FROM public.conditions WHERE _timescaledb_internal.chunks_in(conditions, ARRAY[1, 2, 3, 4]) ORDER BY location ASC NULLS LAST
|
||||
(26 rows)
|
||||
|
||||
\set GROUPING 'region'
|
||||
\ir 'include/aggregate_queries.sql'
|
||||
-- This file and its contents are licensed under the Timescale License.
|
||||
-- Please see the included NOTICE for copyright information and
|
||||
-- LICENSE-TIMESCALE for a copy of the license.
|
||||
-- This files assumes the existence of some table with definition as seen in the aggregate_table.sql file.
|
||||
-- All of these should be able to be pushed down if enabled
|
||||
:PREFIX SELECT :GROUPING,
|
||||
min(allnull) as min_allnull,
|
||||
max(temperature) as max_temp,
|
||||
sum(temperature)+sum(humidity) as agg_sum_expr,
|
||||
avg(humidity),
|
||||
ROUND( CAST(stddev(humidity) AS NUMERIC), 1),
|
||||
bit_and(bit_int),
|
||||
bit_or(bit_int),
|
||||
bool_and(good_life),
|
||||
every(temperature > 0),
|
||||
bool_or(good_life),
|
||||
count(*) as count_rows,
|
||||
count(temperature) as count_temp,
|
||||
count(allnull) as count_zero,
|
||||
ROUND( CAST(corr(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(covar_pop(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(covar_samp(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(regr_avgx(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(regr_avgy(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(regr_count(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(regr_intercept(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(regr_r2(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(regr_slope(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(regr_sxx(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(regr_sxy(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(regr_syy(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(stddev(temperature) AS NUMERIC), 1) as stddev_temp,
|
||||
ROUND( CAST(stddev_pop(temperature) AS NUMERIC), 1),
|
||||
ROUND( CAST(stddev_samp(temperature) AS NUMERIC), 1),
|
||||
ROUND( CAST(variance(temperature) AS NUMERIC), 1),
|
||||
ROUND( CAST(var_pop(temperature) AS NUMERIC), 1),
|
||||
ROUND( CAST(var_samp(temperature) AS NUMERIC), 1),
|
||||
last(temperature, timec) as last_temp,
|
||||
histogram(temperature, 0, 100, 1)
|
||||
FROM :TEST_TABLE
|
||||
GROUP BY :GROUPING
|
||||
ORDER BY :GROUPING;
|
||||
QUERY PLAN
|
||||
-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
Sort
|
||||
Output: conditions.region, (min(conditions.allnull)), (max(conditions.temperature)), ((sum(conditions.temperature) + sum(conditions.humidity))), (avg(conditions.humidity)), (round((stddev(conditions.humidity))::numeric, 1)), (bit_and(conditions.bit_int)), (bit_or(conditions.bit_int)), (bool_and(conditions.good_life)), (every((conditions.temperature > '0'::double precision))), (bool_or(conditions.good_life)), (count(*)), (count(conditions.temperature)), (count(conditions.allnull)), (round((corr(conditions.temperature, conditions.humidity))::numeric, 1)), (round((covar_pop(conditions.temperature, conditions.humidity))::numeric, 1)), (round((covar_samp(conditions.temperature, conditions.humidity))::numeric, 1)), (round((regr_avgx(conditions.temperature, conditions.humidity))::numeric, 1)), (round((regr_avgy(conditions.temperature, conditions.humidity))::numeric, 1)), (round((regr_count(conditions.temperature, conditions.humidity))::numeric, 1)), (round((regr_intercept(conditions.temperature, conditions.humidity))::numeric, 1)), (round((regr_r2(conditions.temperature, conditions.humidity))::numeric, 1)), (round((regr_slope(conditions.temperature, conditions.humidity))::numeric, 1)), (round((regr_sxx(conditions.temperature, conditions.humidity))::numeric, 1)), (round((regr_sxy(conditions.temperature, conditions.humidity))::numeric, 1)), (round((regr_syy(conditions.temperature, conditions.humidity))::numeric, 1)), (round((stddev(conditions.temperature))::numeric, 1)), (round((stddev_pop(conditions.temperature))::numeric, 1)), (round((stddev_samp(conditions.temperature))::numeric, 1)), (round((variance(conditions.temperature))::numeric, 1)), (round((var_pop(conditions.temperature))::numeric, 1)), (round((var_samp(conditions.temperature))::numeric, 1)), (last(conditions.temperature, conditions.timec)), (histogram(conditions.temperature, '0'::double precision, '100'::double precision, 1))
|
||||
Sort Key: conditions.region
|
||||
-> Finalize HashAggregate
|
||||
Output: conditions.region, min(conditions.allnull), max(conditions.temperature), (sum(conditions.temperature) + sum(conditions.humidity)), avg(conditions.humidity), round((stddev(conditions.humidity))::numeric, 1), bit_and(conditions.bit_int), bit_or(conditions.bit_int), bool_and(conditions.good_life), every((conditions.temperature > '0'::double precision)), bool_or(conditions.good_life), count(*), count(conditions.temperature), count(conditions.allnull), round((corr(conditions.temperature, conditions.humidity))::numeric, 1), round((covar_pop(conditions.temperature, conditions.humidity))::numeric, 1), round((covar_samp(conditions.temperature, conditions.humidity))::numeric, 1), round((regr_avgx(conditions.temperature, conditions.humidity))::numeric, 1), round((regr_avgy(conditions.temperature, conditions.humidity))::numeric, 1), round((regr_count(conditions.temperature, conditions.humidity))::numeric, 1), round((regr_intercept(conditions.temperature, conditions.humidity))::numeric, 1), round((regr_r2(conditions.temperature, conditions.humidity))::numeric, 1), round((regr_slope(conditions.temperature, conditions.humidity))::numeric, 1), round((regr_sxx(conditions.temperature, conditions.humidity))::numeric, 1), round((regr_sxy(conditions.temperature, conditions.humidity))::numeric, 1), round((regr_syy(conditions.temperature, conditions.humidity))::numeric, 1), round((stddev(conditions.temperature))::numeric, 1), round((stddev_pop(conditions.temperature))::numeric, 1), round((stddev_samp(conditions.temperature))::numeric, 1), round((variance(conditions.temperature))::numeric, 1), round((var_pop(conditions.temperature))::numeric, 1), round((var_samp(conditions.temperature))::numeric, 1), last(conditions.temperature, conditions.timec), histogram(conditions.temperature, '0'::double precision, '100'::double precision, 1)
|
||||
Group Key: conditions.region
|
||||
-> Append
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: conditions.region, (PARTIAL min(conditions.allnull)), (PARTIAL max(conditions.temperature)), (PARTIAL sum(conditions.temperature)), (PARTIAL sum(conditions.humidity)), (PARTIAL avg(conditions.humidity)), (PARTIAL stddev(conditions.humidity)), (PARTIAL bit_and(conditions.bit_int)), (PARTIAL bit_or(conditions.bit_int)), (PARTIAL bool_and(conditions.good_life)), (PARTIAL every((conditions.temperature > '0'::double precision))), (PARTIAL bool_or(conditions.good_life)), (PARTIAL count(*)), (PARTIAL count(conditions.temperature)), (PARTIAL count(conditions.allnull)), (PARTIAL corr(conditions.temperature, conditions.humidity)), (PARTIAL covar_pop(conditions.temperature, conditions.humidity)), (PARTIAL covar_samp(conditions.temperature, conditions.humidity)), (PARTIAL regr_avgx(conditions.temperature, conditions.humidity)), (PARTIAL regr_avgy(conditions.temperature, conditions.humidity)), (PARTIAL regr_count(conditions.temperature, conditions.humidity)), (PARTIAL regr_intercept(conditions.temperature, conditions.humidity)), (PARTIAL regr_r2(conditions.temperature, conditions.humidity)), (PARTIAL regr_slope(conditions.temperature, conditions.humidity)), (PARTIAL regr_sxx(conditions.temperature, conditions.humidity)), (PARTIAL regr_sxy(conditions.temperature, conditions.humidity)), (PARTIAL regr_syy(conditions.temperature, conditions.humidity)), (PARTIAL stddev(conditions.temperature)), (PARTIAL stddev_pop(conditions.temperature)), (PARTIAL stddev_samp(conditions.temperature)), (PARTIAL variance(conditions.temperature)), (PARTIAL var_pop(conditions.temperature)), (PARTIAL var_samp(conditions.temperature)), (PARTIAL last(conditions.temperature, conditions.timec)), (PARTIAL histogram(conditions.temperature, '0'::double precision, '100'::double precision, 1))
|
||||
Relations: Aggregate on (public.conditions)
|
||||
Server: server_1
|
||||
Chunks: _hyper_1_1_dist_chunk, _hyper_1_2_dist_chunk, _hyper_1_3_dist_chunk, _hyper_1_4_dist_chunk
|
||||
Remote SQL: SELECT region, _timescaledb_internal.partialize_agg(min(allnull)), _timescaledb_internal.partialize_agg(max(temperature)), _timescaledb_internal.partialize_agg(sum(temperature)), _timescaledb_internal.partialize_agg(sum(humidity)), _timescaledb_internal.partialize_agg(avg(humidity)), _timescaledb_internal.partialize_agg(stddev(humidity)), _timescaledb_internal.partialize_agg(bit_and(bit_int)), _timescaledb_internal.partialize_agg(bit_or(bit_int)), _timescaledb_internal.partialize_agg(bool_and(good_life)), _timescaledb_internal.partialize_agg(every((temperature > 0::double precision))), _timescaledb_internal.partialize_agg(bool_or(good_life)), _timescaledb_internal.partialize_agg(count(*)), _timescaledb_internal.partialize_agg(count(temperature)), _timescaledb_internal.partialize_agg(count(allnull)), _timescaledb_internal.partialize_agg(corr(temperature, humidity)), _timescaledb_internal.partialize_agg(covar_pop(temperature, humidity)), _timescaledb_internal.partialize_agg(covar_samp(temperature, humidity)), _timescaledb_internal.partialize_agg(regr_avgx(temperature, humidity)), _timescaledb_internal.partialize_agg(regr_avgy(temperature, humidity)), _timescaledb_internal.partialize_agg(regr_count(temperature, humidity)), _timescaledb_internal.partialize_agg(regr_intercept(temperature, humidity)), _timescaledb_internal.partialize_agg(regr_r2(temperature, humidity)), _timescaledb_internal.partialize_agg(regr_slope(temperature, humidity)), _timescaledb_internal.partialize_agg(regr_sxx(temperature, humidity)), _timescaledb_internal.partialize_agg(regr_sxy(temperature, humidity)), _timescaledb_internal.partialize_agg(regr_syy(temperature, humidity)), _timescaledb_internal.partialize_agg(stddev(temperature)), _timescaledb_internal.partialize_agg(stddev_pop(temperature)), _timescaledb_internal.partialize_agg(stddev_samp(temperature)), _timescaledb_internal.partialize_agg(variance(temperature)), _timescaledb_internal.partialize_agg(var_pop(temperature)), _timescaledb_internal.partialize_agg(var_samp(temperature)), _timescaledb_internal.partialize_agg(public.last(temperature, timec)), _timescaledb_internal.partialize_agg(public.histogram(temperature, 0::double precision, 100::double precision, 1)) FROM public.conditions WHERE _timescaledb_internal.chunks_in(conditions, ARRAY[1, 2, 3, 4]) GROUP BY 1
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: conditions_1.region, (PARTIAL min(conditions_1.allnull)), (PARTIAL max(conditions_1.temperature)), (PARTIAL sum(conditions_1.temperature)), (PARTIAL sum(conditions_1.humidity)), (PARTIAL avg(conditions_1.humidity)), (PARTIAL stddev(conditions_1.humidity)), (PARTIAL bit_and(conditions_1.bit_int)), (PARTIAL bit_or(conditions_1.bit_int)), (PARTIAL bool_and(conditions_1.good_life)), (PARTIAL every((conditions_1.temperature > '0'::double precision))), (PARTIAL bool_or(conditions_1.good_life)), (PARTIAL count(*)), (PARTIAL count(conditions_1.temperature)), (PARTIAL count(conditions_1.allnull)), (PARTIAL corr(conditions_1.temperature, conditions_1.humidity)), (PARTIAL covar_pop(conditions_1.temperature, conditions_1.humidity)), (PARTIAL covar_samp(conditions_1.temperature, conditions_1.humidity)), (PARTIAL regr_avgx(conditions_1.temperature, conditions_1.humidity)), (PARTIAL regr_avgy(conditions_1.temperature, conditions_1.humidity)), (PARTIAL regr_count(conditions_1.temperature, conditions_1.humidity)), (PARTIAL regr_intercept(conditions_1.temperature, conditions_1.humidity)), (PARTIAL regr_r2(conditions_1.temperature, conditions_1.humidity)), (PARTIAL regr_slope(conditions_1.temperature, conditions_1.humidity)), (PARTIAL regr_sxx(conditions_1.temperature, conditions_1.humidity)), (PARTIAL regr_sxy(conditions_1.temperature, conditions_1.humidity)), (PARTIAL regr_syy(conditions_1.temperature, conditions_1.humidity)), (PARTIAL stddev(conditions_1.temperature)), (PARTIAL stddev_pop(conditions_1.temperature)), (PARTIAL stddev_samp(conditions_1.temperature)), (PARTIAL variance(conditions_1.temperature)), (PARTIAL var_pop(conditions_1.temperature)), (PARTIAL var_samp(conditions_1.temperature)), (PARTIAL last(conditions_1.temperature, conditions_1.timec)), (PARTIAL histogram(conditions_1.temperature, '0'::double precision, '100'::double precision, 1))
|
||||
Relations: Aggregate on (public.conditions)
|
||||
Server: server_2
|
||||
Chunks: _hyper_1_9_dist_chunk, _hyper_1_10_dist_chunk, _hyper_1_11_dist_chunk, _hyper_1_12_dist_chunk
|
||||
Remote SQL: SELECT region, _timescaledb_internal.partialize_agg(min(allnull)), _timescaledb_internal.partialize_agg(max(temperature)), _timescaledb_internal.partialize_agg(sum(temperature)), _timescaledb_internal.partialize_agg(sum(humidity)), _timescaledb_internal.partialize_agg(avg(humidity)), _timescaledb_internal.partialize_agg(stddev(humidity)), _timescaledb_internal.partialize_agg(bit_and(bit_int)), _timescaledb_internal.partialize_agg(bit_or(bit_int)), _timescaledb_internal.partialize_agg(bool_and(good_life)), _timescaledb_internal.partialize_agg(every((temperature > 0::double precision))), _timescaledb_internal.partialize_agg(bool_or(good_life)), _timescaledb_internal.partialize_agg(count(*)), _timescaledb_internal.partialize_agg(count(temperature)), _timescaledb_internal.partialize_agg(count(allnull)), _timescaledb_internal.partialize_agg(corr(temperature, humidity)), _timescaledb_internal.partialize_agg(covar_pop(temperature, humidity)), _timescaledb_internal.partialize_agg(covar_samp(temperature, humidity)), _timescaledb_internal.partialize_agg(regr_avgx(temperature, humidity)), _timescaledb_internal.partialize_agg(regr_avgy(temperature, humidity)), _timescaledb_internal.partialize_agg(regr_count(temperature, humidity)), _timescaledb_internal.partialize_agg(regr_intercept(temperature, humidity)), _timescaledb_internal.partialize_agg(regr_r2(temperature, humidity)), _timescaledb_internal.partialize_agg(regr_slope(temperature, humidity)), _timescaledb_internal.partialize_agg(regr_sxx(temperature, humidity)), _timescaledb_internal.partialize_agg(regr_sxy(temperature, humidity)), _timescaledb_internal.partialize_agg(regr_syy(temperature, humidity)), _timescaledb_internal.partialize_agg(stddev(temperature)), _timescaledb_internal.partialize_agg(stddev_pop(temperature)), _timescaledb_internal.partialize_agg(stddev_samp(temperature)), _timescaledb_internal.partialize_agg(variance(temperature)), _timescaledb_internal.partialize_agg(var_pop(temperature)), _timescaledb_internal.partialize_agg(var_samp(temperature)), _timescaledb_internal.partialize_agg(public.last(temperature, timec)), _timescaledb_internal.partialize_agg(public.histogram(temperature, 0::double precision, 100::double precision, 1)) FROM public.conditions WHERE _timescaledb_internal.chunks_in(conditions, ARRAY[1, 2, 3, 4]) GROUP BY 1
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: conditions_2.region, (PARTIAL min(conditions_2.allnull)), (PARTIAL max(conditions_2.temperature)), (PARTIAL sum(conditions_2.temperature)), (PARTIAL sum(conditions_2.humidity)), (PARTIAL avg(conditions_2.humidity)), (PARTIAL stddev(conditions_2.humidity)), (PARTIAL bit_and(conditions_2.bit_int)), (PARTIAL bit_or(conditions_2.bit_int)), (PARTIAL bool_and(conditions_2.good_life)), (PARTIAL every((conditions_2.temperature > '0'::double precision))), (PARTIAL bool_or(conditions_2.good_life)), (PARTIAL count(*)), (PARTIAL count(conditions_2.temperature)), (PARTIAL count(conditions_2.allnull)), (PARTIAL corr(conditions_2.temperature, conditions_2.humidity)), (PARTIAL covar_pop(conditions_2.temperature, conditions_2.humidity)), (PARTIAL covar_samp(conditions_2.temperature, conditions_2.humidity)), (PARTIAL regr_avgx(conditions_2.temperature, conditions_2.humidity)), (PARTIAL regr_avgy(conditions_2.temperature, conditions_2.humidity)), (PARTIAL regr_count(conditions_2.temperature, conditions_2.humidity)), (PARTIAL regr_intercept(conditions_2.temperature, conditions_2.humidity)), (PARTIAL regr_r2(conditions_2.temperature, conditions_2.humidity)), (PARTIAL regr_slope(conditions_2.temperature, conditions_2.humidity)), (PARTIAL regr_sxx(conditions_2.temperature, conditions_2.humidity)), (PARTIAL regr_sxy(conditions_2.temperature, conditions_2.humidity)), (PARTIAL regr_syy(conditions_2.temperature, conditions_2.humidity)), (PARTIAL stddev(conditions_2.temperature)), (PARTIAL stddev_pop(conditions_2.temperature)), (PARTIAL stddev_samp(conditions_2.temperature)), (PARTIAL variance(conditions_2.temperature)), (PARTIAL var_pop(conditions_2.temperature)), (PARTIAL var_samp(conditions_2.temperature)), (PARTIAL last(conditions_2.temperature, conditions_2.timec)), (PARTIAL histogram(conditions_2.temperature, '0'::double precision, '100'::double precision, 1))
|
||||
Relations: Aggregate on (public.conditions)
|
||||
Server: server_3
|
||||
Chunks: _hyper_1_5_dist_chunk, _hyper_1_6_dist_chunk, _hyper_1_7_dist_chunk, _hyper_1_8_dist_chunk
|
||||
Remote SQL: SELECT region, _timescaledb_internal.partialize_agg(min(allnull)), _timescaledb_internal.partialize_agg(max(temperature)), _timescaledb_internal.partialize_agg(sum(temperature)), _timescaledb_internal.partialize_agg(sum(humidity)), _timescaledb_internal.partialize_agg(avg(humidity)), _timescaledb_internal.partialize_agg(stddev(humidity)), _timescaledb_internal.partialize_agg(bit_and(bit_int)), _timescaledb_internal.partialize_agg(bit_or(bit_int)), _timescaledb_internal.partialize_agg(bool_and(good_life)), _timescaledb_internal.partialize_agg(every((temperature > 0::double precision))), _timescaledb_internal.partialize_agg(bool_or(good_life)), _timescaledb_internal.partialize_agg(count(*)), _timescaledb_internal.partialize_agg(count(temperature)), _timescaledb_internal.partialize_agg(count(allnull)), _timescaledb_internal.partialize_agg(corr(temperature, humidity)), _timescaledb_internal.partialize_agg(covar_pop(temperature, humidity)), _timescaledb_internal.partialize_agg(covar_samp(temperature, humidity)), _timescaledb_internal.partialize_agg(regr_avgx(temperature, humidity)), _timescaledb_internal.partialize_agg(regr_avgy(temperature, humidity)), _timescaledb_internal.partialize_agg(regr_count(temperature, humidity)), _timescaledb_internal.partialize_agg(regr_intercept(temperature, humidity)), _timescaledb_internal.partialize_agg(regr_r2(temperature, humidity)), _timescaledb_internal.partialize_agg(regr_slope(temperature, humidity)), _timescaledb_internal.partialize_agg(regr_sxx(temperature, humidity)), _timescaledb_internal.partialize_agg(regr_sxy(temperature, humidity)), _timescaledb_internal.partialize_agg(regr_syy(temperature, humidity)), _timescaledb_internal.partialize_agg(stddev(temperature)), _timescaledb_internal.partialize_agg(stddev_pop(temperature)), _timescaledb_internal.partialize_agg(stddev_samp(temperature)), _timescaledb_internal.partialize_agg(variance(temperature)), _timescaledb_internal.partialize_agg(var_pop(temperature)), _timescaledb_internal.partialize_agg(var_samp(temperature)), _timescaledb_internal.partialize_agg(public.last(temperature, timec)), _timescaledb_internal.partialize_agg(public.histogram(temperature, 0::double precision, 100::double precision, 1)) FROM public.conditions WHERE _timescaledb_internal.chunks_in(conditions, ARRAY[1, 2, 3, 4]) GROUP BY 1
|
||||
(25 rows)
|
||||
|
||||
-- Aggregates on custom types are not yet pushed down
|
||||
:PREFIX SELECT :GROUPING,
|
||||
last(highlow, timec) as last_hl,
|
||||
first(highlow, timec) as first_hl
|
||||
FROM :TEST_TABLE
|
||||
GROUP BY :GROUPING
|
||||
ORDER BY :GROUPING;
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
Sort
|
||||
Output: conditions.region, (last(conditions.highlow, conditions.timec)), (first(conditions.highlow, conditions.timec))
|
||||
Sort Key: conditions.region
|
||||
-> Finalize HashAggregate
|
||||
Output: conditions.region, last(conditions.highlow, conditions.timec), first(conditions.highlow, conditions.timec)
|
||||
Group Key: conditions.region
|
||||
-> Append
|
||||
-> Partial HashAggregate
|
||||
Output: conditions.region, PARTIAL last(conditions.highlow, conditions.timec), PARTIAL first(conditions.highlow, conditions.timec)
|
||||
Group Key: conditions.region
|
||||
-> Custom Scan (ServerScan) on public.conditions
|
||||
Output: conditions.region, conditions.highlow, conditions.timec
|
||||
Server: server_1
|
||||
Chunks: _hyper_1_1_dist_chunk, _hyper_1_2_dist_chunk, _hyper_1_3_dist_chunk, _hyper_1_4_dist_chunk
|
||||
Remote SQL: SELECT timec, region, highlow FROM public.conditions WHERE _timescaledb_internal.chunks_in(conditions, ARRAY[1, 2, 3, 4])
|
||||
-> Partial HashAggregate
|
||||
Output: conditions_1.region, PARTIAL last(conditions_1.highlow, conditions_1.timec), PARTIAL first(conditions_1.highlow, conditions_1.timec)
|
||||
Group Key: conditions_1.region
|
||||
-> Custom Scan (ServerScan) on public.conditions conditions_1
|
||||
Output: conditions_1.region, conditions_1.highlow, conditions_1.timec
|
||||
Server: server_2
|
||||
Chunks: _hyper_1_9_dist_chunk, _hyper_1_10_dist_chunk, _hyper_1_11_dist_chunk, _hyper_1_12_dist_chunk
|
||||
Remote SQL: SELECT timec, region, highlow FROM public.conditions WHERE _timescaledb_internal.chunks_in(conditions, ARRAY[1, 2, 3, 4])
|
||||
-> Partial HashAggregate
|
||||
Output: conditions_2.region, PARTIAL last(conditions_2.highlow, conditions_2.timec), PARTIAL first(conditions_2.highlow, conditions_2.timec)
|
||||
Group Key: conditions_2.region
|
||||
-> Custom Scan (ServerScan) on public.conditions conditions_2
|
||||
Output: conditions_2.region, conditions_2.highlow, conditions_2.timec
|
||||
Server: server_3
|
||||
Chunks: _hyper_1_5_dist_chunk, _hyper_1_6_dist_chunk, _hyper_1_7_dist_chunk, _hyper_1_8_dist_chunk
|
||||
Remote SQL: SELECT timec, region, highlow FROM public.conditions WHERE _timescaledb_internal.chunks_in(conditions, ARRAY[1, 2, 3, 4])
|
||||
(31 rows)
|
||||
|
||||
-- Mix of aggregates that push down and those that don't
|
||||
:PREFIX SELECT :GROUPING,
|
||||
min(allnull) as min_allnull,
|
||||
max(temperature) as max_temp,
|
||||
sum(temperature)+sum(humidity) as agg_sum_expr,
|
||||
avg(humidity),
|
||||
ROUND( CAST(stddev(humidity) AS NUMERIC), 1),
|
||||
bit_and(bit_int),
|
||||
bit_or(bit_int),
|
||||
bool_and(good_life),
|
||||
every(temperature > 0),
|
||||
bool_or(good_life),
|
||||
first(highlow, timec) as first_hl
|
||||
FROM :TEST_TABLE
|
||||
GROUP BY :GROUPING
|
||||
ORDER BY :GROUPING;
|
||||
QUERY PLAN
|
||||
-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
Sort
|
||||
Output: conditions.region, (min(conditions.allnull)), (max(conditions.temperature)), ((sum(conditions.temperature) + sum(conditions.humidity))), (avg(conditions.humidity)), (round((stddev(conditions.humidity))::numeric, 1)), (bit_and(conditions.bit_int)), (bit_or(conditions.bit_int)), (bool_and(conditions.good_life)), (every((conditions.temperature > '0'::double precision))), (bool_or(conditions.good_life)), (first(conditions.highlow, conditions.timec))
|
||||
Sort Key: conditions.region
|
||||
-> Finalize HashAggregate
|
||||
Output: conditions.region, min(conditions.allnull), max(conditions.temperature), (sum(conditions.temperature) + sum(conditions.humidity)), avg(conditions.humidity), round((stddev(conditions.humidity))::numeric, 1), bit_and(conditions.bit_int), bit_or(conditions.bit_int), bool_and(conditions.good_life), every((conditions.temperature > '0'::double precision)), bool_or(conditions.good_life), first(conditions.highlow, conditions.timec)
|
||||
Group Key: conditions.region
|
||||
-> Append
|
||||
-> Partial HashAggregate
|
||||
Output: conditions.region, PARTIAL min(conditions.allnull), PARTIAL max(conditions.temperature), PARTIAL sum(conditions.temperature), PARTIAL sum(conditions.humidity), PARTIAL avg(conditions.humidity), PARTIAL stddev(conditions.humidity), PARTIAL bit_and(conditions.bit_int), PARTIAL bit_or(conditions.bit_int), PARTIAL bool_and(conditions.good_life), PARTIAL every((conditions.temperature > '0'::double precision)), PARTIAL bool_or(conditions.good_life), PARTIAL first(conditions.highlow, conditions.timec)
|
||||
Group Key: conditions.region
|
||||
-> Custom Scan (ServerScan) on public.conditions
|
||||
Output: conditions.region, conditions.allnull, conditions.temperature, conditions.humidity, conditions.bit_int, conditions.good_life, conditions.highlow, conditions.timec
|
||||
Server: server_1
|
||||
Chunks: _hyper_1_1_dist_chunk, _hyper_1_2_dist_chunk, _hyper_1_3_dist_chunk, _hyper_1_4_dist_chunk
|
||||
Remote SQL: SELECT timec, region, temperature, humidity, allnull, highlow, bit_int, good_life FROM public.conditions WHERE _timescaledb_internal.chunks_in(conditions, ARRAY[1, 2, 3, 4])
|
||||
-> Partial HashAggregate
|
||||
Output: conditions_1.region, PARTIAL min(conditions_1.allnull), PARTIAL max(conditions_1.temperature), PARTIAL sum(conditions_1.temperature), PARTIAL sum(conditions_1.humidity), PARTIAL avg(conditions_1.humidity), PARTIAL stddev(conditions_1.humidity), PARTIAL bit_and(conditions_1.bit_int), PARTIAL bit_or(conditions_1.bit_int), PARTIAL bool_and(conditions_1.good_life), PARTIAL every((conditions_1.temperature > '0'::double precision)), PARTIAL bool_or(conditions_1.good_life), PARTIAL first(conditions_1.highlow, conditions_1.timec)
|
||||
Group Key: conditions_1.region
|
||||
-> Custom Scan (ServerScan) on public.conditions conditions_1
|
||||
Output: conditions_1.region, conditions_1.allnull, conditions_1.temperature, conditions_1.humidity, conditions_1.bit_int, conditions_1.good_life, conditions_1.highlow, conditions_1.timec
|
||||
Server: server_2
|
||||
Chunks: _hyper_1_9_dist_chunk, _hyper_1_10_dist_chunk, _hyper_1_11_dist_chunk, _hyper_1_12_dist_chunk
|
||||
Remote SQL: SELECT timec, region, temperature, humidity, allnull, highlow, bit_int, good_life FROM public.conditions WHERE _timescaledb_internal.chunks_in(conditions, ARRAY[1, 2, 3, 4])
|
||||
-> Partial HashAggregate
|
||||
Output: conditions_2.region, PARTIAL min(conditions_2.allnull), PARTIAL max(conditions_2.temperature), PARTIAL sum(conditions_2.temperature), PARTIAL sum(conditions_2.humidity), PARTIAL avg(conditions_2.humidity), PARTIAL stddev(conditions_2.humidity), PARTIAL bit_and(conditions_2.bit_int), PARTIAL bit_or(conditions_2.bit_int), PARTIAL bool_and(conditions_2.good_life), PARTIAL every((conditions_2.temperature > '0'::double precision)), PARTIAL bool_or(conditions_2.good_life), PARTIAL first(conditions_2.highlow, conditions_2.timec)
|
||||
Group Key: conditions_2.region
|
||||
-> Custom Scan (ServerScan) on public.conditions conditions_2
|
||||
Output: conditions_2.region, conditions_2.allnull, conditions_2.temperature, conditions_2.humidity, conditions_2.bit_int, conditions_2.good_life, conditions_2.highlow, conditions_2.timec
|
||||
Server: server_3
|
||||
Chunks: _hyper_1_5_dist_chunk, _hyper_1_6_dist_chunk, _hyper_1_7_dist_chunk, _hyper_1_8_dist_chunk
|
||||
Remote SQL: SELECT timec, region, temperature, humidity, allnull, highlow, bit_int, good_life FROM public.conditions WHERE _timescaledb_internal.chunks_in(conditions, ARRAY[1, 2, 3, 4])
|
||||
(31 rows)
|
||||
|
||||
-- Full aggregate pushdown correctness check, compare location grouped query results with partionwise aggregates on and off
|
||||
\set GROUPING 'location'
|
||||
SELECT format('%s/results/dist_agg_loc_results_test.out', :'TEST_OUTPUT_DIR') as "RESULTS_TEST1",
|
||||
format('%s/results/dist_agg_loc_results_control.out', :'TEST_OUTPUT_DIR') as "RESULTS_CONTROL1"
|
||||
\gset
|
||||
SELECT format('\! diff %s %s', :'RESULTS_CONTROL1', :'RESULTS_TEST1') as "DIFF_CMD1"
|
||||
\gset
|
||||
--generate the results into two different files
|
||||
\set ECHO errors
|
||||
:DIFF_CMD1
|
||||
-- Partial aggregate pushdown correctness check, compare region grouped query results with partionwise aggregates on and off
|
||||
\set GROUPING 'region'
|
||||
SELECT format('%s/results/dist_agg_region_results_test.out', :'TEST_OUTPUT_DIR') as "RESULTS_TEST2",
|
||||
format('%s/results/dist_agg_region_results_control.out', :'TEST_OUTPUT_DIR') as "RESULTS_CONTROL2"
|
||||
\gset
|
||||
SELECT format('\! diff %s %s', :'RESULTS_CONTROL2', :'RESULTS_TEST2') as "DIFF_CMD2"
|
||||
\gset
|
||||
--generate the results into two different files
|
||||
\set ECHO errors
|
||||
:DIFF_CMD2
|
@ -473,7 +473,7 @@ WHERE time < '2018-06-01 00:00'
|
||||
GROUP BY 1
|
||||
ORDER BY 1;
|
||||
QUERY PLAN
|
||||
-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
Sort
|
||||
Output: hyper."time", (avg(hyper.temp))
|
||||
Sort Key: hyper."time"
|
||||
@ -481,23 +481,19 @@ ORDER BY 1;
|
||||
Output: hyper."time", avg(hyper.temp)
|
||||
Group Key: hyper."time"
|
||||
-> Append
|
||||
-> Partial HashAggregate
|
||||
Output: hyper."time", PARTIAL avg(hyper.temp)
|
||||
Group Key: hyper."time"
|
||||
-> Custom Scan (ServerScan) on public.hyper
|
||||
Output: hyper."time", hyper.temp
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: hyper."time", (PARTIAL avg(hyper.temp))
|
||||
Relations: Aggregate on (public.hyper)
|
||||
Server: server_1
|
||||
Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk
|
||||
Remote SQL: SELECT "time", temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone))
|
||||
-> Partial HashAggregate
|
||||
Output: hyper_1."time", PARTIAL avg(hyper_1.temp)
|
||||
Group Key: hyper_1."time"
|
||||
-> Custom Scan (ServerScan) on public.hyper hyper_1
|
||||
Output: hyper_1."time", hyper_1.temp
|
||||
Remote SQL: SELECT "time", _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: hyper_1."time", (PARTIAL avg(hyper_1.temp))
|
||||
Relations: Aggregate on (public.hyper)
|
||||
Server: server_2
|
||||
Chunks: _hyper_1_2_dist_chunk, _hyper_1_4_dist_chunk
|
||||
Remote SQL: SELECT "time", temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone))
|
||||
(23 rows)
|
||||
Remote SQL: SELECT "time", _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1
|
||||
(19 rows)
|
||||
|
||||
-- Show result
|
||||
SELECT time, avg(temp)
|
||||
@ -699,28 +695,24 @@ FROM hyper
|
||||
WHERE time < '2018-06-01 00:00'
|
||||
GROUP BY 1;
|
||||
QUERY PLAN
|
||||
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
Finalize HashAggregate
|
||||
Output: hyper.location, avg(hyper.temp)
|
||||
Group Key: hyper.location
|
||||
-> Append
|
||||
-> Partial HashAggregate
|
||||
Output: hyper.location, PARTIAL avg(hyper.temp)
|
||||
Group Key: hyper.location
|
||||
-> Custom Scan (ServerScan) on public.hyper
|
||||
Output: hyper.location, hyper.temp
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: hyper.location, (PARTIAL avg(hyper.temp))
|
||||
Relations: Aggregate on (public.hyper)
|
||||
Server: server_1
|
||||
Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk
|
||||
Remote SQL: SELECT location, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone))
|
||||
-> Partial HashAggregate
|
||||
Output: hyper_1.location, PARTIAL avg(hyper_1.temp)
|
||||
Group Key: hyper_1.location
|
||||
-> Custom Scan (ServerScan) on public.hyper hyper_1
|
||||
Output: hyper_1.location, hyper_1.temp
|
||||
Remote SQL: SELECT location, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: hyper_1.location, (PARTIAL avg(hyper_1.temp))
|
||||
Relations: Aggregate on (public.hyper)
|
||||
Server: server_2
|
||||
Chunks: _hyper_1_4_dist_chunk, _hyper_1_2_dist_chunk
|
||||
Remote SQL: SELECT location, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone))
|
||||
(20 rows)
|
||||
Remote SQL: SELECT location, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1
|
||||
(16 rows)
|
||||
|
||||
-- Expand query across repartition boundary. This makes it unsafe to
|
||||
-- push down the FULL agg, so should expect a PARTIAL agg on
|
||||
@ -804,7 +796,7 @@ FROM hyper
|
||||
GROUP BY 1
|
||||
ORDER BY 1;
|
||||
QUERY PLAN
|
||||
---------------------------------------------------------------------------------------------------------------------------------------------
|
||||
---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
Sort
|
||||
Output: hyper.device, (avg(hyper.temp))
|
||||
Sort Key: hyper.device
|
||||
@ -812,23 +804,19 @@ ORDER BY 1;
|
||||
Output: hyper.device, avg(hyper.temp)
|
||||
Group Key: hyper.device
|
||||
-> Append
|
||||
-> Partial HashAggregate
|
||||
Output: hyper.device, PARTIAL avg(hyper.temp)
|
||||
Group Key: hyper.device
|
||||
-> Custom Scan (ServerScan) on public.hyper
|
||||
Output: hyper.device, hyper.temp
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: hyper.device, (PARTIAL avg(hyper.temp))
|
||||
Relations: Aggregate on (public.hyper)
|
||||
Server: server_1
|
||||
Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk, _hyper_1_5_dist_chunk, _hyper_1_7_dist_chunk
|
||||
Remote SQL: SELECT device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[1, 2, 3, 4])
|
||||
-> Partial HashAggregate
|
||||
Output: hyper_1.device, PARTIAL avg(hyper_1.temp)
|
||||
Group Key: hyper_1.device
|
||||
-> Custom Scan (ServerScan) on public.hyper hyper_1
|
||||
Output: hyper_1.device, hyper_1.temp
|
||||
Remote SQL: SELECT device, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[1, 2, 3, 4]) GROUP BY 1
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: hyper_1.device, (PARTIAL avg(hyper_1.temp))
|
||||
Relations: Aggregate on (public.hyper)
|
||||
Server: server_2
|
||||
Chunks: _hyper_1_2_dist_chunk, _hyper_1_4_dist_chunk, _hyper_1_6_dist_chunk
|
||||
Remote SQL: SELECT device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[1, 2, 3])
|
||||
(23 rows)
|
||||
Remote SQL: SELECT device, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[1, 2, 3]) GROUP BY 1
|
||||
(19 rows)
|
||||
|
||||
-- Show result
|
||||
SELECT device, avg(temp)
|
||||
@ -898,7 +886,7 @@ WHERE time > '2018-04-19 00:01'
|
||||
GROUP BY 1
|
||||
ORDER BY 1;
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
Sort
|
||||
Output: hyper.device, (avg(hyper.temp))
|
||||
Sort Key: hyper.device
|
||||
@ -906,23 +894,19 @@ ORDER BY 1;
|
||||
Output: hyper.device, avg(hyper.temp)
|
||||
Group Key: hyper.device
|
||||
-> Append
|
||||
-> Partial HashAggregate
|
||||
Output: hyper.device, PARTIAL avg(hyper.temp)
|
||||
Group Key: hyper.device
|
||||
-> Custom Scan (ServerScan) on public.hyper
|
||||
Output: hyper.device, hyper.temp
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: hyper.device, (PARTIAL avg(hyper.temp))
|
||||
Relations: Aggregate on (public.hyper)
|
||||
Server: server_1
|
||||
Chunks: _hyper_1_3_dist_chunk, _hyper_1_5_dist_chunk, _hyper_1_7_dist_chunk
|
||||
Remote SQL: SELECT device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[2, 3, 4]) AND (("time" > '2018-04-19 00:01:00-07'::timestamp with time zone))
|
||||
-> Partial HashAggregate
|
||||
Output: hyper_1.device, PARTIAL avg(hyper_1.temp)
|
||||
Group Key: hyper_1.device
|
||||
-> Custom Scan (ServerScan) on public.hyper hyper_1
|
||||
Output: hyper_1.device, hyper_1.temp
|
||||
Remote SQL: SELECT device, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[2, 3, 4]) AND (("time" > '2018-04-19 00:01:00-07'::timestamp with time zone)) GROUP BY 1
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: hyper_1.device, (PARTIAL avg(hyper_1.temp))
|
||||
Relations: Aggregate on (public.hyper)
|
||||
Server: server_2
|
||||
Chunks: _hyper_1_4_dist_chunk, _hyper_1_6_dist_chunk
|
||||
Remote SQL: SELECT device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[2, 3]) AND (("time" > '2018-04-19 00:01:00-07'::timestamp with time zone))
|
||||
(23 rows)
|
||||
Remote SQL: SELECT device, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[2, 3]) AND (("time" > '2018-04-19 00:01:00-07'::timestamp with time zone)) GROUP BY 1
|
||||
(19 rows)
|
||||
|
||||
SELECT device, avg(temp)
|
||||
FROM hyper
|
||||
@ -2184,28 +2168,24 @@ SELECT time_bucket('1 day', time), avg(temp)
|
||||
FROM hyper1d
|
||||
GROUP BY 1;
|
||||
QUERY PLAN
|
||||
-------------------------------------------------------------------------------------------------------------------------------------
|
||||
------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
Finalize HashAggregate
|
||||
Output: (time_bucket('@ 1 day'::interval, hyper1d."time")), avg(hyper1d.temp)
|
||||
Group Key: (time_bucket('@ 1 day'::interval, hyper1d."time"))
|
||||
-> Append
|
||||
-> Partial HashAggregate
|
||||
Output: (time_bucket('@ 1 day'::interval, hyper1d."time")), PARTIAL avg(hyper1d.temp)
|
||||
Group Key: time_bucket('@ 1 day'::interval, hyper1d."time")
|
||||
-> Custom Scan (ServerScan) on public.hyper1d
|
||||
Output: time_bucket('@ 1 day'::interval, hyper1d."time"), hyper1d.temp
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: (time_bucket('@ 1 day'::interval, hyper1d."time")), (PARTIAL avg(hyper1d.temp))
|
||||
Relations: Aggregate on (public.hyper1d)
|
||||
Server: server_1
|
||||
Chunks: _hyper_2_8_dist_chunk, _hyper_2_10_dist_chunk
|
||||
Remote SQL: SELECT "time", temp FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(hyper1d, ARRAY[5, 6])
|
||||
-> Partial HashAggregate
|
||||
Output: (time_bucket('@ 1 day'::interval, hyper1d_1."time")), PARTIAL avg(hyper1d_1.temp)
|
||||
Group Key: time_bucket('@ 1 day'::interval, hyper1d_1."time")
|
||||
-> Custom Scan (ServerScan) on public.hyper1d hyper1d_1
|
||||
Output: time_bucket('@ 1 day'::interval, hyper1d_1."time"), hyper1d_1.temp
|
||||
Remote SQL: SELECT public.time_bucket('@ 1 day'::interval, "time"), _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(hyper1d, ARRAY[5, 6]) GROUP BY 1
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: (time_bucket('@ 1 day'::interval, hyper1d_1."time")), (PARTIAL avg(hyper1d_1.temp))
|
||||
Relations: Aggregate on (public.hyper1d)
|
||||
Server: server_2
|
||||
Chunks: _hyper_2_9_dist_chunk
|
||||
Remote SQL: SELECT "time", temp FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(hyper1d, ARRAY[4])
|
||||
(20 rows)
|
||||
Remote SQL: SELECT public.time_bucket('@ 1 day'::interval, "time"), _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(hyper1d, ARRAY[4]) GROUP BY 1
|
||||
(16 rows)
|
||||
|
||||
--- Only one chunk in query => safe to fully push down although not on
|
||||
--- a partitioning dimension.
|
||||
@ -2233,20 +2213,22 @@ FROM hyper1d
|
||||
WHERE time < '2018-06-01 00:00'
|
||||
GROUP BY 1;
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
HashAggregate
|
||||
------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
Finalize HashAggregate
|
||||
Output: hyper1d.device, avg(hyper1d.temp)
|
||||
Group Key: hyper1d.device
|
||||
-> Append
|
||||
-> Custom Scan (ServerScan) on public.hyper1d
|
||||
Output: hyper1d.device, hyper1d.temp
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: hyper1d.device, (PARTIAL avg(hyper1d.temp))
|
||||
Relations: Aggregate on (public.hyper1d)
|
||||
Server: server_1
|
||||
Chunks: _hyper_2_8_dist_chunk
|
||||
Remote SQL: SELECT device, temp FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(hyper1d, ARRAY[5]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone))
|
||||
-> Custom Scan (ServerScan) on public.hyper1d hyper1d_1
|
||||
Output: hyper1d_1.device, hyper1d_1.temp
|
||||
Remote SQL: SELECT device, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(hyper1d, ARRAY[5]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: hyper1d_1.device, (PARTIAL avg(hyper1d_1.temp))
|
||||
Relations: Aggregate on (public.hyper1d)
|
||||
Server: server_2
|
||||
Chunks: _hyper_2_9_dist_chunk
|
||||
Remote SQL: SELECT device, temp FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(hyper1d, ARRAY[4]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone))
|
||||
(14 rows)
|
||||
Remote SQL: SELECT device, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(hyper1d, ARRAY[4]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1
|
||||
(16 rows)
|
||||
|
||||
|
@ -473,7 +473,7 @@ WHERE time < '2018-06-01 00:00'
|
||||
GROUP BY 1
|
||||
ORDER BY 1;
|
||||
QUERY PLAN
|
||||
-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
Sort
|
||||
Output: hyper."time", (avg(hyper.temp))
|
||||
Sort Key: hyper."time"
|
||||
@ -481,23 +481,19 @@ ORDER BY 1;
|
||||
Output: hyper."time", avg(hyper.temp)
|
||||
Group Key: hyper."time"
|
||||
-> Append
|
||||
-> Partial HashAggregate
|
||||
Output: hyper."time", PARTIAL avg(hyper.temp)
|
||||
Group Key: hyper."time"
|
||||
-> Custom Scan (ServerScan) on public.hyper
|
||||
Output: hyper."time", hyper.temp
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: hyper."time", (PARTIAL avg(hyper.temp))
|
||||
Relations: Aggregate on (public.hyper)
|
||||
Server: server_1
|
||||
Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk
|
||||
Remote SQL: SELECT "time", temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone))
|
||||
-> Partial HashAggregate
|
||||
Output: hyper_1."time", PARTIAL avg(hyper_1.temp)
|
||||
Group Key: hyper_1."time"
|
||||
-> Custom Scan (ServerScan) on public.hyper hyper_1
|
||||
Output: hyper_1."time", hyper_1.temp
|
||||
Remote SQL: SELECT "time", _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: hyper_1."time", (PARTIAL avg(hyper_1.temp))
|
||||
Relations: Aggregate on (public.hyper)
|
||||
Server: server_2
|
||||
Chunks: _hyper_1_2_dist_chunk, _hyper_1_4_dist_chunk
|
||||
Remote SQL: SELECT "time", temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone))
|
||||
(23 rows)
|
||||
Remote SQL: SELECT "time", _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1
|
||||
(19 rows)
|
||||
|
||||
-- Show result
|
||||
SELECT time, avg(temp)
|
||||
@ -699,28 +695,24 @@ FROM hyper
|
||||
WHERE time < '2018-06-01 00:00'
|
||||
GROUP BY 1;
|
||||
QUERY PLAN
|
||||
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
Finalize HashAggregate
|
||||
Output: hyper.location, avg(hyper.temp)
|
||||
Group Key: hyper.location
|
||||
-> Append
|
||||
-> Partial HashAggregate
|
||||
Output: hyper.location, PARTIAL avg(hyper.temp)
|
||||
Group Key: hyper.location
|
||||
-> Custom Scan (ServerScan) on public.hyper
|
||||
Output: hyper.location, hyper.temp
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: hyper.location, (PARTIAL avg(hyper.temp))
|
||||
Relations: Aggregate on (public.hyper)
|
||||
Server: server_1
|
||||
Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk
|
||||
Remote SQL: SELECT location, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone))
|
||||
-> Partial HashAggregate
|
||||
Output: hyper_1.location, PARTIAL avg(hyper_1.temp)
|
||||
Group Key: hyper_1.location
|
||||
-> Custom Scan (ServerScan) on public.hyper hyper_1
|
||||
Output: hyper_1.location, hyper_1.temp
|
||||
Remote SQL: SELECT location, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: hyper_1.location, (PARTIAL avg(hyper_1.temp))
|
||||
Relations: Aggregate on (public.hyper)
|
||||
Server: server_2
|
||||
Chunks: _hyper_1_4_dist_chunk, _hyper_1_2_dist_chunk
|
||||
Remote SQL: SELECT location, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone))
|
||||
(20 rows)
|
||||
Remote SQL: SELECT location, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1
|
||||
(16 rows)
|
||||
|
||||
-- Expand query across repartition boundary. This makes it unsafe to
|
||||
-- push down the FULL agg, so should expect a PARTIAL agg on
|
||||
@ -804,7 +796,7 @@ FROM hyper
|
||||
GROUP BY 1
|
||||
ORDER BY 1;
|
||||
QUERY PLAN
|
||||
---------------------------------------------------------------------------------------------------------------------------------------------
|
||||
---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
Sort
|
||||
Output: hyper.device, (avg(hyper.temp))
|
||||
Sort Key: hyper.device
|
||||
@ -812,23 +804,19 @@ ORDER BY 1;
|
||||
Output: hyper.device, avg(hyper.temp)
|
||||
Group Key: hyper.device
|
||||
-> Append
|
||||
-> Partial HashAggregate
|
||||
Output: hyper.device, PARTIAL avg(hyper.temp)
|
||||
Group Key: hyper.device
|
||||
-> Custom Scan (ServerScan) on public.hyper
|
||||
Output: hyper.device, hyper.temp
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: hyper.device, (PARTIAL avg(hyper.temp))
|
||||
Relations: Aggregate on (public.hyper)
|
||||
Server: server_1
|
||||
Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk, _hyper_1_5_dist_chunk, _hyper_1_7_dist_chunk
|
||||
Remote SQL: SELECT device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[1, 2, 3, 4])
|
||||
-> Partial HashAggregate
|
||||
Output: hyper_1.device, PARTIAL avg(hyper_1.temp)
|
||||
Group Key: hyper_1.device
|
||||
-> Custom Scan (ServerScan) on public.hyper hyper_1
|
||||
Output: hyper_1.device, hyper_1.temp
|
||||
Remote SQL: SELECT device, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[1, 2, 3, 4]) GROUP BY 1
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: hyper_1.device, (PARTIAL avg(hyper_1.temp))
|
||||
Relations: Aggregate on (public.hyper)
|
||||
Server: server_2
|
||||
Chunks: _hyper_1_2_dist_chunk, _hyper_1_4_dist_chunk, _hyper_1_6_dist_chunk
|
||||
Remote SQL: SELECT device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[1, 2, 3])
|
||||
(23 rows)
|
||||
Remote SQL: SELECT device, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[1, 2, 3]) GROUP BY 1
|
||||
(19 rows)
|
||||
|
||||
-- Show result
|
||||
SELECT device, avg(temp)
|
||||
@ -898,7 +886,7 @@ WHERE time > '2018-04-19 00:01'
|
||||
GROUP BY 1
|
||||
ORDER BY 1;
|
||||
QUERY PLAN
|
||||
--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
Sort
|
||||
Output: hyper.device, (avg(hyper.temp))
|
||||
Sort Key: hyper.device
|
||||
@ -906,23 +894,19 @@ ORDER BY 1;
|
||||
Output: hyper.device, avg(hyper.temp)
|
||||
Group Key: hyper.device
|
||||
-> Append
|
||||
-> Partial HashAggregate
|
||||
Output: hyper.device, PARTIAL avg(hyper.temp)
|
||||
Group Key: hyper.device
|
||||
-> Custom Scan (ServerScan) on public.hyper
|
||||
Output: hyper.device, hyper.temp
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: hyper.device, (PARTIAL avg(hyper.temp))
|
||||
Relations: Aggregate on (public.hyper)
|
||||
Server: server_1
|
||||
Chunks: _hyper_1_3_dist_chunk, _hyper_1_5_dist_chunk, _hyper_1_7_dist_chunk
|
||||
Remote SQL: SELECT device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[2, 3, 4]) AND (("time" > '2018-04-19 00:01:00-07'::timestamp with time zone))
|
||||
-> Partial HashAggregate
|
||||
Output: hyper_1.device, PARTIAL avg(hyper_1.temp)
|
||||
Group Key: hyper_1.device
|
||||
-> Custom Scan (ServerScan) on public.hyper hyper_1
|
||||
Output: hyper_1.device, hyper_1.temp
|
||||
Remote SQL: SELECT device, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[2, 3, 4]) AND (("time" > '2018-04-19 00:01:00-07'::timestamp with time zone)) GROUP BY 1
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: hyper_1.device, (PARTIAL avg(hyper_1.temp))
|
||||
Relations: Aggregate on (public.hyper)
|
||||
Server: server_2
|
||||
Chunks: _hyper_1_4_dist_chunk, _hyper_1_6_dist_chunk
|
||||
Remote SQL: SELECT device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[2, 3]) AND (("time" > '2018-04-19 00:01:00-07'::timestamp with time zone))
|
||||
(23 rows)
|
||||
Remote SQL: SELECT device, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(hyper, ARRAY[2, 3]) AND (("time" > '2018-04-19 00:01:00-07'::timestamp with time zone)) GROUP BY 1
|
||||
(19 rows)
|
||||
|
||||
SELECT device, avg(temp)
|
||||
FROM hyper
|
||||
@ -2174,28 +2158,24 @@ SELECT time_bucket('1 day', time), avg(temp)
|
||||
FROM hyper1d
|
||||
GROUP BY 1;
|
||||
QUERY PLAN
|
||||
-------------------------------------------------------------------------------------------------------------------------------------
|
||||
------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
Finalize HashAggregate
|
||||
Output: (time_bucket('@ 1 day'::interval, hyper1d."time")), avg(hyper1d.temp)
|
||||
Group Key: (time_bucket('@ 1 day'::interval, hyper1d."time"))
|
||||
-> Append
|
||||
-> Partial HashAggregate
|
||||
Output: (time_bucket('@ 1 day'::interval, hyper1d."time")), PARTIAL avg(hyper1d.temp)
|
||||
Group Key: time_bucket('@ 1 day'::interval, hyper1d."time")
|
||||
-> Custom Scan (ServerScan) on public.hyper1d
|
||||
Output: time_bucket('@ 1 day'::interval, hyper1d."time"), hyper1d.temp
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: (time_bucket('@ 1 day'::interval, hyper1d."time")), (PARTIAL avg(hyper1d.temp))
|
||||
Relations: Aggregate on (public.hyper1d)
|
||||
Server: server_1
|
||||
Chunks: _hyper_2_8_dist_chunk, _hyper_2_10_dist_chunk
|
||||
Remote SQL: SELECT "time", temp FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(hyper1d, ARRAY[5, 6])
|
||||
-> Partial HashAggregate
|
||||
Output: (time_bucket('@ 1 day'::interval, hyper1d_1."time")), PARTIAL avg(hyper1d_1.temp)
|
||||
Group Key: time_bucket('@ 1 day'::interval, hyper1d_1."time")
|
||||
-> Custom Scan (ServerScan) on public.hyper1d hyper1d_1
|
||||
Output: time_bucket('@ 1 day'::interval, hyper1d_1."time"), hyper1d_1.temp
|
||||
Remote SQL: SELECT public.time_bucket('@ 1 day'::interval, "time"), _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(hyper1d, ARRAY[5, 6]) GROUP BY 1
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: (time_bucket('@ 1 day'::interval, hyper1d_1."time")), (PARTIAL avg(hyper1d_1.temp))
|
||||
Relations: Aggregate on (public.hyper1d)
|
||||
Server: server_2
|
||||
Chunks: _hyper_2_9_dist_chunk
|
||||
Remote SQL: SELECT "time", temp FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(hyper1d, ARRAY[4])
|
||||
(20 rows)
|
||||
Remote SQL: SELECT public.time_bucket('@ 1 day'::interval, "time"), _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(hyper1d, ARRAY[4]) GROUP BY 1
|
||||
(16 rows)
|
||||
|
||||
--- Only one chunk in query => safe to fully push down although not on
|
||||
--- a partitioning dimension.
|
||||
@ -2222,20 +2202,22 @@ FROM hyper1d
|
||||
WHERE time < '2018-06-01 00:00'
|
||||
GROUP BY 1;
|
||||
QUERY PLAN
|
||||
------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
HashAggregate
|
||||
------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
|
||||
Finalize HashAggregate
|
||||
Output: hyper1d.device, avg(hyper1d.temp)
|
||||
Group Key: hyper1d.device
|
||||
-> Append
|
||||
-> Custom Scan (ServerScan) on public.hyper1d
|
||||
Output: hyper1d.device, hyper1d.temp
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: hyper1d.device, (PARTIAL avg(hyper1d.temp))
|
||||
Relations: Aggregate on (public.hyper1d)
|
||||
Server: server_1
|
||||
Chunks: _hyper_2_8_dist_chunk
|
||||
Remote SQL: SELECT device, temp FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(hyper1d, ARRAY[5]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone))
|
||||
-> Custom Scan (ServerScan) on public.hyper1d hyper1d_1
|
||||
Output: hyper1d_1.device, hyper1d_1.temp
|
||||
Remote SQL: SELECT device, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(hyper1d, ARRAY[5]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1
|
||||
-> Custom Scan (ServerScan)
|
||||
Output: hyper1d_1.device, (PARTIAL avg(hyper1d_1.temp))
|
||||
Relations: Aggregate on (public.hyper1d)
|
||||
Server: server_2
|
||||
Chunks: _hyper_2_9_dist_chunk
|
||||
Remote SQL: SELECT device, temp FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(hyper1d, ARRAY[4]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone))
|
||||
(14 rows)
|
||||
Remote SQL: SELECT device, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(hyper1d, ARRAY[4]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1
|
||||
(16 rows)
|
||||
|
||||
|
@ -83,6 +83,7 @@ if (PG_VERSION_SUPPORTS_MULTINODE)
|
||||
list(APPEND TEST_FILES
|
||||
chunk_api.sql
|
||||
timescaledb_fdw.sql
|
||||
dist_partial_agg.sql
|
||||
)
|
||||
list(APPEND TEST_FILES_DEBUG
|
||||
deparse.sql
|
||||
|
114
tsl/test/sql/dist_partial_agg.sql
Normal file
114
tsl/test/sql/dist_partial_agg.sql
Normal file
@ -0,0 +1,114 @@
|
||||
-- This file and its contents are licensed under the Timescale License.
|
||||
-- Please see the included NOTICE for copyright information and
|
||||
-- LICENSE-TIMESCALE for a copy of the license.
|
||||
|
||||
-- Need to be super user to create extension and add servers
|
||||
\c :TEST_DBNAME :ROLE_SUPERUSER;
|
||||
|
||||
-- Need explicit password for non-super users to connect
|
||||
ALTER ROLE :ROLE_DEFAULT_CLUSTER_USER CREATEDB PASSWORD 'pass';
|
||||
GRANT USAGE ON FOREIGN DATA WRAPPER timescaledb_fdw TO :ROLE_DEFAULT_CLUSTER_USER;
|
||||
|
||||
SET ROLE :ROLE_DEFAULT_CLUSTER_USER;
|
||||
|
||||
-- Cleanup from other potential tests that created these databases
|
||||
SET client_min_messages TO ERROR;
|
||||
DROP DATABASE IF EXISTS server_1;
|
||||
DROP DATABASE IF EXISTS server_2;
|
||||
DROP DATABASE IF EXISTS server_3;
|
||||
SET client_min_messages TO NOTICE;
|
||||
|
||||
CREATE DATABASE server_1;
|
||||
CREATE DATABASE server_2;
|
||||
CREATE DATABASE server_3;
|
||||
|
||||
\c server_1
|
||||
SET client_min_messages TO ERROR;
|
||||
CREATE EXTENSION timescaledb;
|
||||
CREATE TYPE custom_type AS (high int, low int);
|
||||
\c server_2
|
||||
SET client_min_messages TO ERROR;
|
||||
CREATE EXTENSION timescaledb;
|
||||
CREATE TYPE custom_type AS (high int, low int);
|
||||
\c server_3
|
||||
SET client_min_messages TO ERROR;
|
||||
CREATE EXTENSION timescaledb;
|
||||
CREATE TYPE custom_type AS (high int, low int);
|
||||
\c :TEST_DBNAME :ROLE_SUPERUSER;
|
||||
SET ROLE :ROLE_DEFAULT_CLUSTER_USER;
|
||||
|
||||
\set TEST_TABLE 'conditions'
|
||||
\ir 'include/aggregate_table_create.sql'
|
||||
|
||||
-- Add servers using the TimescaleDB server management API
|
||||
SELECT * FROM add_server('server_1', database => 'server_1', password => 'pass', if_not_exists => true);
|
||||
SELECT * FROM add_server('server_2', database => 'server_2', password => 'pass', if_not_exists => true);
|
||||
SELECT * FROM add_server('server_3', database => 'server_3', password => 'pass', if_not_exists => true);
|
||||
|
||||
SELECT table_name FROM create_distributed_hypertable( 'conditions', 'timec', 'location', 3, chunk_time_interval => INTERVAL '1 day');
|
||||
|
||||
-- We need a lot of data and a lot of chunks to make the planner push down all of the aggregates
|
||||
\ir 'include/aggregate_table_populate.sql'
|
||||
|
||||
SET enable_partitionwise_aggregate = ON;
|
||||
|
||||
-- Run an explain on the aggregate queries to make sure expected aggregates are being pushed down.
|
||||
-- Grouping by the paritioning column should result in full aggregate pushdown where possible,
|
||||
-- while using a non-partitioning column should result in a partial pushdown
|
||||
\set PREFIX 'EXPLAIN (VERBOSE, COSTS OFF)'
|
||||
|
||||
\set GROUPING 'location'
|
||||
\ir 'include/aggregate_queries.sql'
|
||||
|
||||
\set GROUPING 'region'
|
||||
\ir 'include/aggregate_queries.sql'
|
||||
|
||||
-- Full aggregate pushdown correctness check, compare location grouped query results with partionwise aggregates on and off
|
||||
\set GROUPING 'location'
|
||||
SELECT format('%s/results/dist_agg_loc_results_test.out', :'TEST_OUTPUT_DIR') as "RESULTS_TEST1",
|
||||
format('%s/results/dist_agg_loc_results_control.out', :'TEST_OUTPUT_DIR') as "RESULTS_CONTROL1"
|
||||
\gset
|
||||
SELECT format('\! diff %s %s', :'RESULTS_CONTROL1', :'RESULTS_TEST1') as "DIFF_CMD1"
|
||||
\gset
|
||||
|
||||
--generate the results into two different files
|
||||
\set ECHO errors
|
||||
SET client_min_messages TO error;
|
||||
--make output contain query results
|
||||
\set PREFIX ''
|
||||
\o :RESULTS_CONTROL1
|
||||
SET enable_partitionwise_aggregate = OFF;
|
||||
\ir 'include/aggregate_queries.sql'
|
||||
\o
|
||||
\o :RESULTS_TEST1
|
||||
SET enable_partitionwise_aggregate = ON;
|
||||
\ir 'include/aggregate_queries.sql'
|
||||
\o
|
||||
\set ECHO all
|
||||
|
||||
:DIFF_CMD1
|
||||
|
||||
-- Partial aggregate pushdown correctness check, compare region grouped query results with partionwise aggregates on and off
|
||||
\set GROUPING 'region'
|
||||
SELECT format('%s/results/dist_agg_region_results_test.out', :'TEST_OUTPUT_DIR') as "RESULTS_TEST2",
|
||||
format('%s/results/dist_agg_region_results_control.out', :'TEST_OUTPUT_DIR') as "RESULTS_CONTROL2"
|
||||
\gset
|
||||
SELECT format('\! diff %s %s', :'RESULTS_CONTROL2', :'RESULTS_TEST2') as "DIFF_CMD2"
|
||||
\gset
|
||||
|
||||
--generate the results into two different files
|
||||
\set ECHO errors
|
||||
SET client_min_messages TO error;
|
||||
--make output contain query results
|
||||
\set PREFIX ''
|
||||
\o :RESULTS_CONTROL2
|
||||
SET enable_partitionwise_aggregate = OFF;
|
||||
\ir 'include/aggregate_queries.sql'
|
||||
\o
|
||||
\o :RESULTS_TEST2
|
||||
SET enable_partitionwise_aggregate = ON;
|
||||
\ir 'include/aggregate_queries.sql'
|
||||
\o
|
||||
\set ECHO all
|
||||
|
||||
:DIFF_CMD2
|
69
tsl/test/sql/include/aggregate_queries.sql
Normal file
69
tsl/test/sql/include/aggregate_queries.sql
Normal file
@ -0,0 +1,69 @@
|
||||
-- This file and its contents are licensed under the Timescale License.
|
||||
-- Please see the included NOTICE for copyright information and
|
||||
-- LICENSE-TIMESCALE for a copy of the license.
|
||||
|
||||
-- This files assumes the existence of some table with definition as seen in the aggregate_table.sql file.
|
||||
|
||||
-- All of these should be able to be pushed down if enabled
|
||||
:PREFIX SELECT :GROUPING,
|
||||
min(allnull) as min_allnull,
|
||||
max(temperature) as max_temp,
|
||||
sum(temperature)+sum(humidity) as agg_sum_expr,
|
||||
avg(humidity),
|
||||
ROUND( CAST(stddev(humidity) AS NUMERIC), 1),
|
||||
bit_and(bit_int),
|
||||
bit_or(bit_int),
|
||||
bool_and(good_life),
|
||||
every(temperature > 0),
|
||||
bool_or(good_life),
|
||||
count(*) as count_rows,
|
||||
count(temperature) as count_temp,
|
||||
count(allnull) as count_zero,
|
||||
ROUND( CAST(corr(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(covar_pop(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(covar_samp(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(regr_avgx(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(regr_avgy(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(regr_count(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(regr_intercept(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(regr_r2(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(regr_slope(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(regr_sxx(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(regr_sxy(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(regr_syy(temperature, humidity) AS NUMERIC), 1),
|
||||
ROUND( CAST(stddev(temperature) AS NUMERIC), 1) as stddev_temp,
|
||||
ROUND( CAST(stddev_pop(temperature) AS NUMERIC), 1),
|
||||
ROUND( CAST(stddev_samp(temperature) AS NUMERIC), 1),
|
||||
ROUND( CAST(variance(temperature) AS NUMERIC), 1),
|
||||
ROUND( CAST(var_pop(temperature) AS NUMERIC), 1),
|
||||
ROUND( CAST(var_samp(temperature) AS NUMERIC), 1),
|
||||
last(temperature, timec) as last_temp,
|
||||
histogram(temperature, 0, 100, 1)
|
||||
FROM :TEST_TABLE
|
||||
GROUP BY :GROUPING
|
||||
ORDER BY :GROUPING;
|
||||
|
||||
-- Aggregates on custom types are not yet pushed down
|
||||
:PREFIX SELECT :GROUPING,
|
||||
last(highlow, timec) as last_hl,
|
||||
first(highlow, timec) as first_hl
|
||||
FROM :TEST_TABLE
|
||||
GROUP BY :GROUPING
|
||||
ORDER BY :GROUPING;
|
||||
|
||||
-- Mix of aggregates that push down and those that don't
|
||||
:PREFIX SELECT :GROUPING,
|
||||
min(allnull) as min_allnull,
|
||||
max(temperature) as max_temp,
|
||||
sum(temperature)+sum(humidity) as agg_sum_expr,
|
||||
avg(humidity),
|
||||
ROUND( CAST(stddev(humidity) AS NUMERIC), 1),
|
||||
bit_and(bit_int),
|
||||
bit_or(bit_int),
|
||||
bool_and(good_life),
|
||||
every(temperature > 0),
|
||||
bool_or(good_life),
|
||||
first(highlow, timec) as first_hl
|
||||
FROM :TEST_TABLE
|
||||
GROUP BY :GROUPING
|
||||
ORDER BY :GROUPING;
|
22
tsl/test/sql/include/aggregate_table_create.sql
Normal file
22
tsl/test/sql/include/aggregate_table_create.sql
Normal file
@ -0,0 +1,22 @@
|
||||
-- This file and its contents are licensed under the Timescale License.
|
||||
-- Please see the included NOTICE for copyright information and
|
||||
-- LICENSE-TIMESCALE for a copy of the license.
|
||||
|
||||
-- This file creates a table with a lot of different types to allow a range of aggregate functions.
|
||||
-- This does not include the creation of a corresponding hypertable, as we may want to vary how that is done.
|
||||
|
||||
CREATE TYPE custom_type AS (high int, low int);
|
||||
|
||||
CREATE TABLE :TEST_TABLE (
|
||||
timec TIMESTAMPTZ NOT NULL,
|
||||
location TEXT NOT NULL,
|
||||
region TEXT NOT NULL,
|
||||
temperature DOUBLE PRECISION NULL,
|
||||
humidity DOUBLE PRECISION NULL,
|
||||
lowp double precision NULL,
|
||||
highp double precision null,
|
||||
allnull double precision null,
|
||||
highlow custom_type null,
|
||||
bit_int smallint,
|
||||
good_life boolean
|
||||
);
|
35
tsl/test/sql/include/aggregate_table_populate.sql
Normal file
35
tsl/test/sql/include/aggregate_table_populate.sql
Normal file
@ -0,0 +1,35 @@
|
||||
-- This file and its contents are licensed under the Timescale License.
|
||||
-- Please see the included NOTICE for copyright information and
|
||||
-- LICENSE-TIMESCALE for a copy of the license.
|
||||
|
||||
-- This files assumes the existence of some table with definition as seen in the aggregate_table.sql file.
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'POR', 'west', generate_series(25, 85, 0.0625), 75, 40, 70, NULL, (1,2)::custom_type, 2, true;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'SFO', 'west', generate_series(25, 85, 0.0625), 75, 40, 70, NULL, (1,2)::custom_type, 2, true;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'SAC', 'west', generate_series(25, 85, 0.0625), 75, 40, 70, NULL, (1,2)::custom_type, 2, true;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'SEA', 'west', generate_series(25, 85, 0.0625), 75, 40, 70, NULL, (1,2)::custom_type, 2, true;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'TAC', 'west', generate_series(25, 85, 0.0625), 75, 40, 70, NULL, (1,2)::custom_type, 2, true;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'NYC', 'north-east', generate_series(29, 41, 0.0125), 45, 50, 40, NULL, (3,4)::custom_type, 4, false;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'BOS', 'north-east', generate_series(29, 41, 0.0125), 45, 50, 40, NULL, (3,4)::custom_type, 4, false;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'CHI', 'midwest', generate_series(29, 41, 0.0125), 45, 50, 40, NULL, (3,4)::custom_type, 4, false;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'MIN', 'midwest', generate_series(29, 41, 0.0125), 45, 50, 40, NULL, (3,4)::custom_type, 4, false;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'DET', 'midwest', generate_series(29, 41, 0.0125), 45, 50, 40, NULL, (3,4)::custom_type, 4, false;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'LA', 'west', generate_series(61, 85, 0.025), 55, NULL, 28, NULL, NULL, 8, true;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'SDG', 'west', generate_series(61, 85, 0.025), 55, NULL, 28, NULL, NULL, 8, true;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'PHX', 'west', generate_series(61, 85, 0.025), 55, NULL, 28, NULL, NULL, 8, true;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'DAL', 'south', generate_series(61, 85, 0.025), 55, NULL, 28, NULL, NULL, 8, true;
|
||||
INSERT INTO :TEST_TABLE
|
||||
SELECT generate_series('2018-12-01 00:00'::timestamp, '2018-12-04 08:00'::timestamp, '5 minute'), 'AUS', 'south', generate_series(61, 85, 0.025), 55, NULL, 28, NULL, NULL, 8, true;
|
Loading…
x
Reference in New Issue
Block a user