diff --git a/src/chunk.c b/src/chunk.c index 9dc3d71dd..60c6b10d1 100644 --- a/src/chunk.c +++ b/src/chunk.c @@ -2299,6 +2299,36 @@ ts_chunk_get_by_id(int32 id, bool fail_if_not_found) return chunk_scan_find(CHUNK_ID_INDEX, scankey, 1, CurrentMemoryContext, fail_if_not_found); } +/* + * Number of chunks created after given chunk. + * If chunk2.id > chunk1.id then chunk2 is created after chunk1 + */ +int +ts_chunk_num_of_chunks_created_after(const Chunk *chunk) +{ + ScanKeyData scankey[1]; + + /* + * Try to find chunks with a greater Id then a given chunk + */ + ScanKeyInit(&scankey[0], + Anum_chunk_idx_id, + BTGreaterStrategyNumber, + F_INT4GT, + Int32GetDatum(chunk->fd.id)); + + return chunk_scan_internal(CHUNK_ID_INDEX, + scankey, + 1, + NULL, + NULL, + NULL, + 0, + ForwardScanDirection, + AccessShareLock, + CurrentMemoryContext); +} + /* * Simple scans provide lightweight ways to access chunk information without the * overhead of getting a full chunk (i.e., no extra metadata, like constraints, diff --git a/src/chunk.h b/src/chunk.h index 10e7c582f..9916d1886 100644 --- a/src/chunk.h +++ b/src/chunk.h @@ -136,6 +136,7 @@ extern Oid ts_chunk_get_schema_id(int32 chunk_id, bool missing_ok); extern bool ts_chunk_get_id(const char *schema, const char *table, int32 *chunk_id, bool missing_ok); extern bool ts_chunk_exists_relid(Oid relid); +extern TSDLLEXPORT int ts_chunk_num_of_chunks_created_after(const Chunk *chunk); extern TSDLLEXPORT bool ts_chunk_exists_with_compression(int32 hypertable_id); extern void ts_chunk_recreate_all_constraints_for_dimension(Hyperspace *hs, int32 dimension_id); extern TSDLLEXPORT void ts_chunk_drop_fks(Chunk *const chunk); @@ -144,7 +145,8 @@ extern int ts_chunk_delete_by_hypertable_id(int32 hypertable_id); extern int ts_chunk_delete_by_name(const char *schema, const char *table, DropBehavior behavior); extern bool ts_chunk_set_name(Chunk *chunk, const char *newname); extern bool ts_chunk_set_schema(Chunk *chunk, const char *newschema); -extern List *ts_chunk_get_window(int32 dimension_id, int64 point, int count, MemoryContext mctx); +extern TSDLLEXPORT List *ts_chunk_get_window(int32 dimension_id, int64 point, int count, + MemoryContext mctx); extern void ts_chunks_rename_schema_name(char *old_schema, char *new_schema); extern TSDLLEXPORT bool ts_chunk_set_compressed_chunk(Chunk *chunk, int32 compressed_chunk_id, bool isnull); diff --git a/src/chunk_adaptive.c b/src/chunk_adaptive.c index af14565bb..50cdda342 100644 --- a/src/chunk_adaptive.c +++ b/src/chunk_adaptive.c @@ -114,8 +114,8 @@ get_memory_cache_size(void) */ #define DEFAULT_CACHE_MEMORY_SLACK 0.9 -static inline int64 -calculate_initial_chunk_target_size(void) +extern inline int64 +ts_chunk_calculate_initial_chunk_target_size(void) { return (int64)((double) get_memory_cache_size() * DEFAULT_CACHE_MEMORY_SLACK); } @@ -662,7 +662,7 @@ chunk_target_size_in_bytes(const text *target_size_text) return 0; if (pg_strcasecmp(target_size, "estimate") == 0) - target_size_bytes = calculate_initial_chunk_target_size(); + target_size_bytes = ts_chunk_calculate_initial_chunk_target_size(); else target_size_bytes = convert_text_memory_amount_to_bytes(target_size); diff --git a/src/chunk_adaptive.h b/src/chunk_adaptive.h index 4e6508f8a..f18075687 100644 --- a/src/chunk_adaptive.h +++ b/src/chunk_adaptive.h @@ -27,4 +27,6 @@ extern void ts_chunk_adaptive_sizing_info_validate(ChunkSizingInfo *info); extern void ts_chunk_sizing_func_validate(regproc func, ChunkSizingInfo *info); extern TSDLLEXPORT ChunkSizingInfo *ts_chunk_sizing_info_get_default_disabled(Oid table_relid); +extern TSDLLEXPORT int64 ts_chunk_calculate_initial_chunk_target_size(void); + #endif /* TIMESCALEDB_CHUNK_ADAPTIVE_H */ diff --git a/test/sql/utils/testsupport.sql b/test/sql/utils/testsupport.sql index c9d42030f..e30cb51f3 100644 --- a/test/sql/utils/testsupport.sql +++ b/test/sql/utils/testsupport.sql @@ -279,3 +279,6 @@ $BODY$ BEGIN END $BODY$; + +CREATE OR REPLACE FUNCTION ts_test_override_current_timestamptz(new_value TIMESTAMPTZ) +RETURNS VOID AS :TSL_MODULE_PATHNAME, 'ts_test_override_current_timestamptz' LANGUAGE C VOLATILE STRICT; diff --git a/tsl/src/fdw/estimate.c b/tsl/src/fdw/estimate.c index 930f83f8c..65664ac66 100644 --- a/tsl/src/fdw/estimate.c +++ b/tsl/src/fdw/estimate.c @@ -103,7 +103,7 @@ get_upper_rel_estimate(PlannerInfo *root, RelOptInfo *rel, CostEstimate *ce) ce->startup_cost = ofpinfo->rel_startup_cost; ce->startup_cost += aggcosts.transCost.startup; ce->startup_cost += aggcosts.transCost.per_tuple * input_rows; - ce->startup_cost += (cpu_operator_cost * num_group_cols) * input_rows; + ce->startup_cost += cpu_operator_cost * num_group_cols * input_rows; ce->startup_cost += ptarget->cost.startup; /*----- diff --git a/tsl/src/fdw/relinfo.c b/tsl/src/fdw/relinfo.c index a2ca5fbfe..6b12433c1 100644 --- a/tsl/src/fdw/relinfo.c +++ b/tsl/src/fdw/relinfo.c @@ -12,6 +12,7 @@ #include #include #include +#include #include #include @@ -22,6 +23,15 @@ #include "deparse.h" #include "relinfo.h" #include "estimate.h" +#include "chunk_adaptive.h" +#include "cache.h" +#include "hypertable.h" +#include "hypertable_cache.h" +#include "dimension.h" +#include "chunk.h" +#include "hypercube.h" +#include "errors.h" +#include "scan_exec.h" /* Default CPU cost to start up a foreign query. */ #define DEFAULT_FDW_STARTUP_COST 100.0 @@ -31,6 +41,8 @@ #define DEFAULT_FDW_FETCH_SIZE 10000 +#define DEFAULT_CHUNK_LOOKBACK_WINDOW 10 + /* * Parse options from foreign server and apply them to fpinfo. * @@ -97,6 +109,257 @@ get_relation_qualified_name(Oid relid) return name->data; } +static const double FILL_FACTOR_CURRENT_CHUNK = 0.5; +static const double FILL_FACTOR_HISTORICAL_CHUNK = 1; + +static DimensionSlice * +get_chunk_time_slice(Chunk *chunk, Hyperspace *space) +{ + int32 time_dim_id = hyperspace_get_open_dimension(space, 0)->fd.id; + return ts_hypercube_get_slice_by_dimension_id(chunk->cube, time_dim_id); +} + +/* + * Sums of slices belonging to closed dimensions + */ +static int +get_total_number_of_slices(Hyperspace *space) +{ + int dim_idx; + int total_slices = 0; + + for (dim_idx = 0; dim_idx < space->num_dimensions; dim_idx++) + { + Dimension *dim = &space->dimensions[dim_idx]; + if (IS_CLOSED_DIMENSION(dim)) + total_slices += dim->fd.num_slices; + } + + return total_slices; +} + +/* + * Fillfactor values are between 0 and 1. It's an indication of how much data is in the chunk. + * + * Two major drivers for estimation is current time and number of chunks created after. + * + * Fill factor estimation assumes that data written is 'recent' in regards to time dimension (eg. + * almost real-time). For the case when writing historical data, given estimates might be more off + * as we assume that historical chunks have fill factor 1 unless the number of chunks created after + * is smaller then total number of slices. Even for writing historical data we might not be totally + * wrong since most probably data has monotonically increasing time. + * + * Estimation handles two possible hypertable configurations: 1. time dimension is of timestamp + * type 2. time dimension is of integer type. If hypertable uses timestamp type to partition data + * then there are three possible scenarios here: we are beyond chunk end time (historical chunk), we + * are somewhere in between chunk time boundaries (current chunk) or chunk start time is in the + * future (highly unlikely). For integer type we assume that all chunks execpt for current have + * factor 1. + * + * To explain how number of chunks created after the chunk affects estimation + * let's imagine that table is space partitioned with one dimension and having 3 partitions. If data + * is equaliy distributed amount partitions then there will be 3 current chunks. If there are two + * new chunks created after chunk X then chunk X is the current chunk. + */ +static double +estimate_chunk_fillfactor(Chunk *chunk, Hyperspace *space) +{ + Dimension *time_dim = hyperspace_get_open_dimension(space, 0); + DimensionSlice *time_slice = get_chunk_time_slice(chunk, space); + Oid time_dim_type = ts_dimension_get_partition_type(time_dim); + int num_created_after = ts_chunk_num_of_chunks_created_after(chunk); + int total_slices = get_total_number_of_slices(space); + + if (IS_TIMESTAMP_TYPE(time_dim_type)) + { + TimestampTz now = GetSQLCurrentTimestamp(-1); + int64 now_internal_time; + +#ifdef TS_DEBUG + if (ts_current_timestamp_override_value >= 0) + now = ts_current_timestamp_override_value; +#endif + now_internal_time = ts_time_value_to_internal(TimestampTzGetDatum(now), TIMESTAMPTZOID); + + /* if we are beyond end range then chunk can possibly be totally filled */ + if (time_slice->fd.range_end <= now_internal_time) + { + /* If there are less newly created chunks then the number of slices then this is current + * chunk. This also works better when writing historical data */ + return num_created_after < total_slices ? FILL_FACTOR_CURRENT_CHUNK : + FILL_FACTOR_HISTORICAL_CHUNK; + } + + /* for chunks in future (highly unlikely) we assume same as for `current` chunk */ + if (time_slice->fd.range_start >= now_internal_time) + return FILL_FACTOR_CURRENT_CHUNK; + + /* current time falls within chunk time constraints */ + double elapsed = (now_internal_time - time_slice->fd.range_start); + double interval = (time_slice->fd.range_end - time_slice->fd.range_start); + + Assert(interval != 0); + + return elapsed / interval; + } + else + { + /* if current chunk is the last created we assume it has 0.5 fill factor */ + return num_created_after < total_slices ? FILL_FACTOR_CURRENT_CHUNK : + FILL_FACTOR_HISTORICAL_CHUNK; + } +} + +typedef struct RelEstimates +{ + double tuples; + BlockNumber pages; +} RelEstimates; + +/* + * The idea is to look into number of tuples and pages for N previous chunks + * and calculate an average. Ideally we could add weights to this calculation + * and give more importance to newer chunks but a ballpark estimate should be + * just fine. + */ +static RelEstimates * +estimate_tuples_and_pages_using_prev_chunks(PlannerInfo *root, Hyperspace *space, + Chunk *current_chunk) +{ + RelEstimates *estimates = palloc0(sizeof(RelEstimates)); + ListCell *lc; + float4 total_tuples = 0; + int32 total_pages = 0; + int non_zero_reltuples_cnt = 0; + int non_zero_relpages_cnt = 0; + DimensionSlice *time_slice = get_chunk_time_slice(current_chunk, space); + List *prev_chunks = ts_chunk_get_window(time_slice->fd.dimension_id, + time_slice->fd.range_start, + DEFAULT_CHUNK_LOOKBACK_WINDOW, + CurrentMemoryContext); + + foreach (lc, prev_chunks) + { + Chunk *pc = lfirst(lc); + HeapTuple rel_tuple = SearchSysCache1(RELOID, ObjectIdGetDatum(pc->table_id)); + Form_pg_class rel_form; + + if (!HeapTupleIsValid(rel_tuple)) + ereport(ERROR, + (errcode(ERRCODE_TS_INTERNAL_ERROR), + errmsg("cache lookup failed for chunk relation %u", pc->fd.id), + errdetail("Failed to estimate number of tuples and pages for chunk %d.", + pc->table_id))); + + rel_form = (Form_pg_class) GETSTRUCT(rel_tuple); + if (rel_form->reltuples > 0) + { + total_tuples += rel_form->reltuples; + non_zero_reltuples_cnt++; + } + if (rel_form->relpages > 0) + { + total_pages += rel_form->relpages; + non_zero_relpages_cnt++; + } + ReleaseSysCache(rel_tuple); + } + + if (non_zero_reltuples_cnt > 0) + estimates->tuples = total_tuples / non_zero_reltuples_cnt; + if (non_zero_relpages_cnt > 0) + estimates->pages = total_pages / non_zero_relpages_cnt; + + return estimates; +} + +static RelEstimates * +estimate_tuples_and_pages_using_shared_buffers(PlannerInfo *root, Hypertable *ht, int result_width) +{ + RelEstimates *estimates = palloc(sizeof(RelEstimates)); + int64 chunk_size_estimate = ts_chunk_calculate_initial_chunk_target_size(); + + if (ht != NULL) + { + int total_slices = get_total_number_of_slices(ht->space); + if (total_slices > 0) + chunk_size_estimate /= total_slices; + } + else + /* half-size seems to be the safest bet */ + chunk_size_estimate /= 2; + + estimates->tuples = chunk_size_estimate / (result_width + MAXALIGN(SizeofHeapTupleHeader)); + estimates->pages = chunk_size_estimate / BLCKSZ; + return estimates; +} + +static void +set_rel_estimates(RelOptInfo *rel, RelEstimates *estimates) +{ + rel->tuples = estimates->tuples; + rel->pages = estimates->pages; +} + +static void +rel_estimates_apply_fillfactor(RelEstimates *estimates, double fillfactor) +{ + estimates->pages *= fillfactor; + estimates->tuples *= fillfactor; +} + +/* + * When there are no local stats we try estimating by either using stats from previous chunks (if + * they exist) or shared buffers size. + */ +static void +estimate_tuples_and_pages(PlannerInfo *root, RelOptInfo *rel) +{ + int parent_relid; + RangeTblEntry *hyper_rte; + Cache *hcache; + Hypertable *ht; + double chunk_fillfactor; + RangeTblEntry *chunk_rte; + Chunk *chunk; + Hyperspace *hyperspace; + RelEstimates *estimates; + + Assert(rel->tuples == 0); + Assert(rel->pages == 0); + + /* In some cases (e.g., UPDATE stmt) top_parent_relids is not set so the best + we can do is using shared buffers size without partitioning information. + Since updates are not something we generaly optimize for this should be fine. */ + if (rel->top_parent_relids == NULL) + { + estimates = + estimate_tuples_and_pages_using_shared_buffers(root, NULL, rel->reltarget->width); + set_rel_estimates(rel, estimates); + return; + } + + parent_relid = bms_next_member(rel->top_parent_relids, -1); + hyper_rte = planner_rt_fetch(parent_relid, root); + hcache = ts_hypertable_cache_pin(); + ht = ts_hypertable_cache_get_entry(hcache, hyper_rte->relid, CACHE_FLAG_NONE); + hyperspace = ht->space; + chunk_rte = planner_rt_fetch(rel->relid, root); + chunk = ts_chunk_get_by_relid(chunk_rte->relid, true); + + /* Let's first try figuring out number of tuples/pages using stats from previous chunks, + otherwise make an estimation based on shared buffers size */ + estimates = estimate_tuples_and_pages_using_prev_chunks(root, hyperspace, chunk); + if (estimates->tuples == 0 || estimates->pages == 0) + estimates = estimate_tuples_and_pages_using_shared_buffers(root, ht, rel->reltarget->width); + + chunk_fillfactor = estimate_chunk_fillfactor(chunk, hyperspace); + /* adjust tuples/pages using chunk_fillfactor */ + rel_estimates_apply_fillfactor(estimates, chunk_fillfactor); + set_rel_estimates(rel, estimates); + + ts_cache_release(hcache); +} TsFdwRelInfo * fdw_relinfo_create(PlannerInfo *root, RelOptInfo *rel, Oid server_oid, Oid local_table_id, @@ -199,17 +462,10 @@ fdw_relinfo_create(PlannerInfo *root, RelOptInfo *rel, Oid server_oid, Oid local /* * If the foreign table has never been ANALYZEd, it will have relpages * and reltuples equal to zero, which most likely has nothing to do - * with reality. We can't do a whole lot about that if we're not - * allowed to consult the data node, but we can use a hack similar - * to plancat.c's treatment of empty relations: use a minimum size - * estimate of 10 pages, and divide by the column-datatype-based width - * estimate to get the corresponding number of tuples. + * with reality. The best we can do is estimate number of tuples/pages. */ - if (rel->pages == 0 && rel->tuples == 0) - { - rel->pages = 10; - rel->tuples = (10 * BLCKSZ) / (rel->reltarget->width + MAXALIGN(SizeofHeapTupleHeader)); - } + if (rel->pages == 0 && rel->tuples == 0 && type == TS_FDW_RELINFO_FOREIGN_TABLE) + estimate_tuples_and_pages(root, rel); /* Estimate rel size as best we can with local statistics. There are * no local statistics for data node rels since they aren't real base diff --git a/tsl/src/fdw/scan_exec.c b/tsl/src/fdw/scan_exec.c index 4fa318f83..5c99331fc 100644 --- a/tsl/src/fdw/scan_exec.c +++ b/tsl/src/fdw/scan_exec.c @@ -191,7 +191,7 @@ prepare_query_params(PlanState *node, List *fdw_exprs, int num_params, FmgrInfo TimestampTz ts_current_timestamp_override_value = -1; extern void -fdw_scan_debug_override_pushdown_timestamp(TimestampTz time) +fdw_scan_debug_override_current_timestamp(TimestampTz time) { ts_current_timestamp_override_value = time; } diff --git a/tsl/src/fdw/scan_exec.h b/tsl/src/fdw/scan_exec.h index d87566c55..552b7b2cb 100644 --- a/tsl/src/fdw/scan_exec.h +++ b/tsl/src/fdw/scan_exec.h @@ -51,8 +51,10 @@ extern void fdw_scan_explain(ScanState *ss, List *fdw_private, ExplainState *es, extern DataFetcher *create_data_fetcher(ScanState *ss, TsFdwScanState *fsstate, FetchMode mode); #ifdef TS_DEBUG + +extern TimestampTz ts_current_timestamp_override_value; /* Allow tests to specify the time to push down in place of now() */ -extern void fdw_scan_debug_override_pushdown_timestamp(TimestampTz time); +extern void fdw_scan_debug_override_current_timestamp(TimestampTz time); #endif #endif /* TIMESCALEDB_TSL_FDW_SCAN_EXEC_H */ diff --git a/tsl/test/expected/debug_notice.out b/tsl/test/expected/debug_notice.out index b77147256..7f7a511c1 100644 --- a/tsl/test/expected/debug_notice.out +++ b/tsl/test/expected/debug_notice.out @@ -61,6 +61,7 @@ INSERT INTO hyper VALUES ('2018-05-19 13:01', 4, 4, 5.1), ('2018-05-20 15:08', 5, 1, 9.4), ('2018-05-30 13:02', 3, 2, 9.0); +-- Update table stats ANALYZE hyper; -- Optimizer debug messages shown at debug level 2 SET client_min_messages TO DEBUG2; diff --git a/tsl/test/expected/dist_partial_agg.out b/tsl/test/expected/dist_partial_agg.out index 42cd4d7e9..06c709fdc 100644 --- a/tsl/test/expected/dist_partial_agg.out +++ b/tsl/test/expected/dist_partial_agg.out @@ -208,37 +208,39 @@ SET enable_partitionwise_aggregate = ON; FROM :TEST_TABLE GROUP BY :GROUPING ORDER BY :GROUPING; - QUERY PLAN -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- - Custom Scan (AsyncAppend) + QUERY PLAN +---------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + Sort Output: location, (last(highlow, timec)), (first(highlow, timec)) - -> 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 (DataNodeScan) on public.conditions - Output: conditions.location, conditions.highlow, conditions.timec - Data node: data_node_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(public.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 (DataNodeScan) on public.conditions conditions_1 - Output: conditions_1.location, conditions_1.highlow, conditions_1.timec - Data node: data_node_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(public.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 (DataNodeScan) on public.conditions conditions_2 - Output: conditions_2.location, conditions_2.highlow, conditions_2.timec - Data node: data_node_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(public.conditions.*, ARRAY[1, 2, 3, 4]) ORDER BY location ASC NULLS LAST -(28 rows) + Sort Key: location + -> Custom Scan (AsyncAppend) + Output: location, (last(highlow, timec)), (first(highlow, timec)) + -> Append + -> HashAggregate + Output: conditions.location, last(conditions.highlow, conditions.timec), first(conditions.highlow, conditions.timec) + Group Key: conditions.location + -> Custom Scan (DataNodeScan) on public.conditions + Output: conditions.location, conditions.highlow, conditions.timec + Data node: data_node_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(public.conditions.*, ARRAY[1, 2, 3, 4]) + -> HashAggregate + 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 (DataNodeScan) on public.conditions conditions_1 + Output: conditions_1.location, conditions_1.highlow, conditions_1.timec + Data node: data_node_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(public.conditions.*, ARRAY[1, 2, 3, 4]) + -> HashAggregate + 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 (DataNodeScan) on public.conditions conditions_2 + Output: conditions_2.location, conditions_2.highlow, conditions_2.timec + Data node: data_node_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(public.conditions.*, ARRAY[1, 2, 3, 4]) +(30 rows) -- Mix of aggregates that push down and those that don't :PREFIX SELECT :GROUPING, @@ -256,37 +258,39 @@ SET enable_partitionwise_aggregate = ON; FROM :TEST_TABLE GROUP BY :GROUPING ORDER BY :GROUPING; - QUERY PLAN -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- - Custom Scan (AsyncAppend) + QUERY PLAN +------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + Sort Output: location, (min(allnull)), (max(temperature)), ((sum(temperature) + sum(humidity))), (avg(humidity)), (round(stddev((humidity)::integer), 5)), (bit_and(bit_int)), (bit_or(bit_int)), (bool_and(good_life)), (every((temperature > '0'::double precision))), (bool_or(good_life)), (first(highlow, timec)) - -> 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)::integer), 5), 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 (DataNodeScan) on public.conditions - Output: conditions.location, conditions.allnull, conditions.temperature, conditions.humidity, conditions.bit_int, conditions.good_life, conditions.highlow, conditions.timec - Data node: data_node_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(public.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)::integer), 5), 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 (DataNodeScan) 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 - Data node: data_node_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(public.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)::integer), 5), 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 (DataNodeScan) 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 - Data node: data_node_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(public.conditions.*, ARRAY[1, 2, 3, 4]) ORDER BY location ASC NULLS LAST -(28 rows) + Sort Key: location + -> Custom Scan (AsyncAppend) + Output: location, (min(allnull)), (max(temperature)), ((sum(temperature) + sum(humidity))), (avg(humidity)), (round(stddev((humidity)::integer), 5)), (bit_and(bit_int)), (bit_or(bit_int)), (bool_and(good_life)), (every((temperature > '0'::double precision))), (bool_or(good_life)), (first(highlow, timec)) + -> Append + -> HashAggregate + Output: conditions.location, min(conditions.allnull), max(conditions.temperature), (sum(conditions.temperature) + sum(conditions.humidity)), avg(conditions.humidity), round(stddev((conditions.humidity)::integer), 5), 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 (DataNodeScan) on public.conditions + Output: conditions.location, conditions.allnull, conditions.temperature, conditions.humidity, conditions.bit_int, conditions.good_life, conditions.highlow, conditions.timec + Data node: data_node_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(public.conditions.*, ARRAY[1, 2, 3, 4]) + -> HashAggregate + 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)::integer), 5), 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 (DataNodeScan) 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 + Data node: data_node_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(public.conditions.*, ARRAY[1, 2, 3, 4]) + -> HashAggregate + 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)::integer), 5), 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 (DataNodeScan) 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 + Data node: data_node_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(public.conditions.*, ARRAY[1, 2, 3, 4]) +(30 rows) \set GROUPING 'region' \ir 'include/aggregate_queries.sql' @@ -370,40 +374,42 @@ SET enable_partitionwise_aggregate = ON; FROM :TEST_TABLE GROUP BY :GROUPING ORDER BY :GROUPING; - QUERY PLAN ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + QUERY PLAN +-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- Finalize GroupAggregate Output: region, last(highlow, timec), first(highlow, timec) Group Key: region - -> Custom Scan (AsyncAppend) + -> Sort Output: region, (PARTIAL last(highlow, timec)), (PARTIAL first(highlow, timec)) - -> Merge Append - Sort Key: conditions.region - -> Partial GroupAggregate - Output: conditions.region, PARTIAL last(conditions.highlow, conditions.timec), PARTIAL first(conditions.highlow, conditions.timec) - Group Key: conditions.region - -> Custom Scan (DataNodeScan) on public.conditions - Output: conditions.region, conditions.highlow, conditions.timec - Data node: data_node_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(public.conditions.*, ARRAY[1, 2, 3, 4]) ORDER BY region ASC NULLS LAST - -> Partial GroupAggregate - 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 (DataNodeScan) on public.conditions conditions_1 - Output: conditions_1.region, conditions_1.highlow, conditions_1.timec - Data node: data_node_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(public.conditions.*, ARRAY[1, 2, 3, 4]) ORDER BY region ASC NULLS LAST - -> Partial GroupAggregate - 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 (DataNodeScan) on public.conditions conditions_2 - Output: conditions_2.region, conditions_2.highlow, conditions_2.timec - Data node: data_node_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(public.conditions.*, ARRAY[1, 2, 3, 4]) ORDER BY region ASC NULLS LAST -(31 rows) + Sort Key: region + -> Custom Scan (AsyncAppend) + Output: region, (PARTIAL last(highlow, timec)), (PARTIAL first(highlow, timec)) + -> Append + -> Partial HashAggregate + Output: conditions.region, PARTIAL last(conditions.highlow, conditions.timec), PARTIAL first(conditions.highlow, conditions.timec) + Group Key: conditions.region + -> Custom Scan (DataNodeScan) on public.conditions + Output: conditions.region, conditions.highlow, conditions.timec + Data node: data_node_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(public.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 (DataNodeScan) on public.conditions conditions_1 + Output: conditions_1.region, conditions_1.highlow, conditions_1.timec + Data node: data_node_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(public.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 (DataNodeScan) on public.conditions conditions_2 + Output: conditions_2.region, conditions_2.highlow, conditions_2.timec + Data node: data_node_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(public.conditions.*, ARRAY[1, 2, 3, 4]) +(33 rows) -- Mix of aggregates that push down and those that don't :PREFIX SELECT :GROUPING, @@ -421,40 +427,42 @@ SET enable_partitionwise_aggregate = ON; FROM :TEST_TABLE GROUP BY :GROUPING ORDER BY :GROUPING; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + QUERY PLAN +---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- Finalize GroupAggregate Output: region, min(allnull), max(temperature), (sum(temperature) + sum(humidity)), avg(humidity), round(stddev((humidity)::integer), 5), bit_and(bit_int), bit_or(bit_int), bool_and(good_life), every((temperature > '0'::double precision)), bool_or(good_life), first(highlow, timec) Group Key: region - -> Custom Scan (AsyncAppend) + -> Sort Output: region, (PARTIAL min(allnull)), (PARTIAL max(temperature)), (PARTIAL sum(temperature)), (PARTIAL sum(humidity)), (PARTIAL avg(humidity)), (PARTIAL stddev((humidity)::integer)), (PARTIAL bit_and(bit_int)), (PARTIAL bit_or(bit_int)), (PARTIAL bool_and(good_life)), (PARTIAL every((temperature > '0'::double precision))), (PARTIAL bool_or(good_life)), (PARTIAL first(highlow, timec)) - -> Merge Append - Sort Key: conditions.region - -> Partial GroupAggregate - 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)::integer), 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 (DataNodeScan) on public.conditions - Output: conditions.region, conditions.allnull, conditions.temperature, conditions.humidity, conditions.bit_int, conditions.good_life, conditions.highlow, conditions.timec - Data node: data_node_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(public.conditions.*, ARRAY[1, 2, 3, 4]) ORDER BY region ASC NULLS LAST - -> Partial GroupAggregate - 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)::integer), 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 (DataNodeScan) 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 - Data node: data_node_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(public.conditions.*, ARRAY[1, 2, 3, 4]) ORDER BY region ASC NULLS LAST - -> Partial GroupAggregate - 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)::integer), 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 (DataNodeScan) 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 - Data node: data_node_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(public.conditions.*, ARRAY[1, 2, 3, 4]) ORDER BY region ASC NULLS LAST -(31 rows) + Sort Key: region + -> Custom Scan (AsyncAppend) + Output: region, (PARTIAL min(allnull)), (PARTIAL max(temperature)), (PARTIAL sum(temperature)), (PARTIAL sum(humidity)), (PARTIAL avg(humidity)), (PARTIAL stddev((humidity)::integer)), (PARTIAL bit_and(bit_int)), (PARTIAL bit_or(bit_int)), (PARTIAL bool_and(good_life)), (PARTIAL every((temperature > '0'::double precision))), (PARTIAL bool_or(good_life)), (PARTIAL first(highlow, timec)) + -> 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)::integer), 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 (DataNodeScan) on public.conditions + Output: conditions.region, conditions.allnull, conditions.temperature, conditions.humidity, conditions.bit_int, conditions.good_life, conditions.highlow, conditions.timec + Data node: data_node_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(public.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)::integer), 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 (DataNodeScan) 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 + Data node: data_node_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(public.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)::integer), 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 (DataNodeScan) 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 + Data node: data_node_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(public.conditions.*, ARRAY[1, 2, 3, 4]) +(33 rows) -- Full aggregate pushdown correctness check, compare location grouped query results with partionwise aggregates on and off \set GROUPING 'location' @@ -480,11 +488,3 @@ SELECT format('\! diff %s %s', :'RESULTS_CONTROL2', :'RESULTS_TEST2') as "DIFF_C -- multiple values for "col" that has the same timestamp, so the -- output depends on the order of arriving tuples. :DIFF_CMD2 -14c14 -< west | (1,2) | (1,2) ---- -> west | (1,2) | -22c22 -< west | | 85 | 993674 | 67.5 | 9.68309 | 0 | 10 | t | t | t | (1,2) ---- -> west | | 85 | 993674 | 67.5 | 9.68309 | 0 | 10 | t | t | t | diff --git a/tsl/test/expected/hypertable_distributed-11.out b/tsl/test/expected/hypertable_distributed-11.out index f7be38c12..5f0474232 100644 --- a/tsl/test/expected/hypertable_distributed-11.out +++ b/tsl/test/expected/hypertable_distributed-11.out @@ -3,20 +3,10 @@ -- LICENSE-TIMESCALE for a copy of the license. -- Need to be super user to create extension and add data nodes \c :TEST_DBNAME :ROLE_CLUSTER_SUPERUSER; -\ir include/remote_exec.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. -CREATE SCHEMA IF NOT EXISTS test; -psql:include/remote_exec.sql:5: NOTICE: schema "test" already exists, skipping -GRANT USAGE ON SCHEMA test TO PUBLIC; -CREATE OR REPLACE FUNCTION test.remote_exec(srv_name name[], command text) -RETURNS VOID -AS :TSL_MODULE_PATHNAME, 'ts_remote_exec' -LANGUAGE C; --- Support for execute_sql_and_filter_data_node_name_on_error() \unset ECHO psql:include/filter_exec.sql:5: NOTICE: schema "test" already exists, skipping +psql:include/remote_exec.sql:5: NOTICE: schema "test" already exists, skipping +psql:utils/testsupport.sql:8: NOTICE: schema "test" already exists, skipping -- Cleanup from other potential tests that created these databases SET client_min_messages TO ERROR; DROP DATABASE IF EXISTS data_node_1; @@ -2358,27 +2348,27 @@ UPDATE disttable_replicated SET device = 2 WHERE device = (SELECT device FROM de -> Foreign Scan on _timescaledb_internal._hyper_6_11_dist_chunk _hyper_6_11_dist_chunk_1 (actual rows=2 loops=1) Output: _hyper_6_11_dist_chunk_1.device Data node: data_node_1 - Remote SQL: SELECT device FROM _timescaledb_internal._hyper_6_11_dist_chunk ORDER BY device ASC NULLS LAST + Remote SQL: SELECT device FROM _timescaledb_internal._hyper_6_11_dist_chunk -> Foreign Scan on _timescaledb_internal._hyper_6_12_dist_chunk _hyper_6_12_dist_chunk_1 (actual rows=1 loops=1) Output: _hyper_6_12_dist_chunk_1.device Data node: data_node_2 - Remote SQL: SELECT device FROM _timescaledb_internal._hyper_6_12_dist_chunk ORDER BY device ASC NULLS LAST + Remote SQL: SELECT device FROM _timescaledb_internal._hyper_6_12_dist_chunk -> Foreign Scan on _timescaledb_internal._hyper_6_13_dist_chunk _hyper_6_13_dist_chunk_1 (actual rows=1 loops=1) Output: _hyper_6_13_dist_chunk_1.device Data node: data_node_3 - Remote SQL: SELECT device FROM _timescaledb_internal._hyper_6_13_dist_chunk ORDER BY device ASC NULLS LAST + Remote SQL: SELECT device FROM _timescaledb_internal._hyper_6_13_dist_chunk -> Foreign Scan on _timescaledb_internal._hyper_6_14_dist_chunk _hyper_6_14_dist_chunk_1 (actual rows=2 loops=1) Output: _hyper_6_14_dist_chunk_1.device Data node: data_node_1 - Remote SQL: SELECT device FROM _timescaledb_internal._hyper_6_14_dist_chunk ORDER BY device ASC NULLS LAST + Remote SQL: SELECT device FROM _timescaledb_internal._hyper_6_14_dist_chunk -> Foreign Scan on _timescaledb_internal._hyper_6_15_dist_chunk _hyper_6_15_dist_chunk_1 (actual rows=1 loops=1) Output: _hyper_6_15_dist_chunk_1.device Data node: data_node_2 - Remote SQL: SELECT device FROM _timescaledb_internal._hyper_6_15_dist_chunk ORDER BY device ASC NULLS LAST + Remote SQL: SELECT device FROM _timescaledb_internal._hyper_6_15_dist_chunk -> Foreign Scan on _timescaledb_internal._hyper_6_16_dist_chunk _hyper_6_16_dist_chunk_1 (actual rows=1 loops=1) Output: _hyper_6_16_dist_chunk_1.device Data node: data_node_3 - Remote SQL: SELECT device FROM _timescaledb_internal._hyper_6_16_dist_chunk ORDER BY device ASC NULLS LAST + Remote SQL: SELECT device FROM _timescaledb_internal._hyper_6_16_dist_chunk InitPlan 2 (returns $1) -> Limit (actual rows=1 loops=1) Output: devices.device @@ -3295,12 +3285,12 @@ INSERT INTO dist_device VALUES ('2017-01-01 06:01', 1, 1.1), ('2017-01-01 09:11', 3, 2.1), ('2017-01-01 08:01', 1, 1.2); -EXPLAIN VERBOSE +EXPLAIN (VERBOSE, COSTS OFF) SELECT * FROM dist_device; QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------- - Append (cost=100.00..175.14 rows=1861 width=20) - -> Custom Scan (DataNodeScan) on public.dist_device (cost=100.00..165.83 rows=1861 width=20) + Append + -> Custom Scan (DataNodeScan) on public.dist_device Output: dist_device."time", dist_device.dist_device, dist_device.temp Data node: data_node_1 Chunks: _hyper_15_36_dist_chunk @@ -3315,3 +3305,89 @@ SELECT * FROM dist_device; Sun Jan 01 08:01:00 2017 PST | 1 | 1.2 (3 rows) +-- Test estimating relation size without stats +CREATE TABLE hyper_estimate(time timestamptz, device int, temp float); +SELECT * FROM create_distributed_hypertable('hyper_estimate', 'time', 'device', number_partitions => 3, replication_factor => 1, chunk_time_interval => INTERVAL '7 days'); +NOTICE: adding not-null constraint to column "time" + hypertable_id | schema_name | table_name | created +---------------+-------------+----------------+--------- + 16 | public | hyper_estimate | t +(1 row) + +-- This will enable us to more easily see estimates per chunk +SET timescaledb.enable_per_data_node_queries = false; +-- Estimating chunk progress uses current timestamp so we override it for test purposes +SELECT ts_test_override_current_timestamptz('2019-11-11 00:00'::timestamptz); + ts_test_override_current_timestamptz +-------------------------------------- + +(1 row) + +-- Test estimates when backfilling. 3 chunks should be historical and 3 should be considered current when estimating. +-- Note that estimate numbers are way off since we are using shared buffer size as starting point. This will not be +-- an issue in 'production' like env since chunk size should be similar to shared buffer size. +INSERT INTO hyper_estimate VALUES + ('2017-01-01 06:01', 1, 1.1), + ('2017-01-01 09:11', 1, 2.1), + ('2017-01-01 08:01', 1, 1.2), + ('2017-01-02 08:01', 1, 1.3), + ('2017-01-02 08:01', 2, 1.6), + ('2017-01-02 06:01', 2, 1.4), + ('2017-01-03 01:01', 3, 2), + ('2017-01-03 01:16', 3, 3), + ('2017-01-03 01:17', 3, 4), + ('2018-01-13 01:01', 1, 2), + ('2018-01-13 01:10', 1, 0.4), + ('2018-01-13 02:10', 2, 1.4), + ('2018-01-13 05:01', 2, 2), + ('2018-01-13 05:50', 2, 4), + ('2018-01-13 16:01', 3, 2); +-- Since there are no stats we use shared buffers size to estimate number of rows +EXPLAIN (COSTS ON) +SELECT * +FROM hyper_estimate; + QUERY PLAN +--------------------------------------------------------------------------------------------- + Append (cost=100.00..166847.40 rows=4118040 width=20) + -> Foreign Scan on _hyper_16_37_dist_chunk (cost=100.00..32468.60 rows=915120 width=20) + -> Foreign Scan on _hyper_16_38_dist_chunk (cost=100.00..32468.60 rows=915120 width=20) + -> Foreign Scan on _hyper_16_39_dist_chunk (cost=100.00..32468.60 rows=915120 width=20) + -> Foreign Scan on _hyper_16_40_dist_chunk (cost=100.00..16283.80 rows=457560 width=20) + -> Foreign Scan on _hyper_16_41_dist_chunk (cost=100.00..16283.80 rows=457560 width=20) + -> Foreign Scan on _hyper_16_42_dist_chunk (cost=100.00..16283.80 rows=457560 width=20) +(7 rows) + +-- This will calculate the stats +ANALYZE hyper_estimate; +EXPLAIN (COSTS ON) +SELECT * +FROM hyper_estimate; + QUERY PLAN +-------------------------------------------------------------------------------------- + Append (cost=100.00..606.52 rows=15 width=20) + -> Foreign Scan on _hyper_16_37_dist_chunk (cost=100.00..101.12 rows=4 width=20) + -> Foreign Scan on _hyper_16_38_dist_chunk (cost=100.00..101.06 rows=2 width=20) + -> Foreign Scan on _hyper_16_39_dist_chunk (cost=100.00..101.09 rows=3 width=20) + -> Foreign Scan on _hyper_16_40_dist_chunk (cost=100.00..101.06 rows=2 width=20) + -> Foreign Scan on _hyper_16_41_dist_chunk (cost=100.00..101.09 rows=3 width=20) + -> Foreign Scan on _hyper_16_42_dist_chunk (cost=100.00..101.03 rows=1 width=20) +(7 rows) + +-- Let's insert data into a new chunk. This will result in chunk creation. +INSERT INTO hyper_estimate VALUES ('2019-11-11 06:01', 1, 1.1); +-- We have stats for previous chunks so we can interpolate number of rows for the new chunk +EXPLAIN (COSTS ON) +SELECT * +FROM hyper_estimate; + QUERY PLAN +-------------------------------------------------------------------------------------- + Append (cost=100.00..706.58 rows=17 width=20) + -> Foreign Scan on _hyper_16_37_dist_chunk (cost=100.00..101.12 rows=4 width=20) + -> Foreign Scan on _hyper_16_38_dist_chunk (cost=100.00..101.06 rows=2 width=20) + -> Foreign Scan on _hyper_16_39_dist_chunk (cost=100.00..101.09 rows=3 width=20) + -> Foreign Scan on _hyper_16_40_dist_chunk (cost=100.00..101.06 rows=2 width=20) + -> Foreign Scan on _hyper_16_41_dist_chunk (cost=100.00..101.09 rows=3 width=20) + -> Foreign Scan on _hyper_16_42_dist_chunk (cost=100.00..101.03 rows=1 width=20) + -> Foreign Scan on _hyper_16_43_dist_chunk (cost=100.00..100.05 rows=2 width=20) +(8 rows) + diff --git a/tsl/test/expected/hypertable_distributed-12.out b/tsl/test/expected/hypertable_distributed-12.out index e15c2d0d7..ce3271222 100644 --- a/tsl/test/expected/hypertable_distributed-12.out +++ b/tsl/test/expected/hypertable_distributed-12.out @@ -3,20 +3,10 @@ -- LICENSE-TIMESCALE for a copy of the license. -- Need to be super user to create extension and add data nodes \c :TEST_DBNAME :ROLE_CLUSTER_SUPERUSER; -\ir include/remote_exec.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. -CREATE SCHEMA IF NOT EXISTS test; -psql:include/remote_exec.sql:5: NOTICE: schema "test" already exists, skipping -GRANT USAGE ON SCHEMA test TO PUBLIC; -CREATE OR REPLACE FUNCTION test.remote_exec(srv_name name[], command text) -RETURNS VOID -AS :TSL_MODULE_PATHNAME, 'ts_remote_exec' -LANGUAGE C; --- Support for execute_sql_and_filter_data_node_name_on_error() \unset ECHO psql:include/filter_exec.sql:5: NOTICE: schema "test" already exists, skipping +psql:include/remote_exec.sql:5: NOTICE: schema "test" already exists, skipping +psql:utils/testsupport.sql:8: NOTICE: schema "test" already exists, skipping -- Cleanup from other potential tests that created these databases SET client_min_messages TO ERROR; DROP DATABASE IF EXISTS data_node_1; @@ -3289,11 +3279,11 @@ INSERT INTO dist_device VALUES ('2017-01-01 06:01', 1, 1.1), ('2017-01-01 09:11', 3, 2.1), ('2017-01-01 08:01', 1, 1.2); -EXPLAIN VERBOSE +EXPLAIN (VERBOSE, COSTS OFF) SELECT * FROM dist_device; QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------- - Custom Scan (DataNodeScan) on public.dist_device (cost=100.00..165.83 rows=1861 width=20) + Custom Scan (DataNodeScan) on public.dist_device Output: dist_device."time", dist_device.dist_device, dist_device.temp Data node: data_node_1 Chunks: _hyper_15_36_dist_chunk @@ -3308,3 +3298,89 @@ SELECT * FROM dist_device; Sun Jan 01 08:01:00 2017 PST | 1 | 1.2 (3 rows) +-- Test estimating relation size without stats +CREATE TABLE hyper_estimate(time timestamptz, device int, temp float); +SELECT * FROM create_distributed_hypertable('hyper_estimate', 'time', 'device', number_partitions => 3, replication_factor => 1, chunk_time_interval => INTERVAL '7 days'); +NOTICE: adding not-null constraint to column "time" + hypertable_id | schema_name | table_name | created +---------------+-------------+----------------+--------- + 16 | public | hyper_estimate | t +(1 row) + +-- This will enable us to more easily see estimates per chunk +SET timescaledb.enable_per_data_node_queries = false; +-- Estimating chunk progress uses current timestamp so we override it for test purposes +SELECT ts_test_override_current_timestamptz('2019-11-11 00:00'::timestamptz); + ts_test_override_current_timestamptz +-------------------------------------- + +(1 row) + +-- Test estimates when backfilling. 3 chunks should be historical and 3 should be considered current when estimating. +-- Note that estimate numbers are way off since we are using shared buffer size as starting point. This will not be +-- an issue in 'production' like env since chunk size should be similar to shared buffer size. +INSERT INTO hyper_estimate VALUES + ('2017-01-01 06:01', 1, 1.1), + ('2017-01-01 09:11', 1, 2.1), + ('2017-01-01 08:01', 1, 1.2), + ('2017-01-02 08:01', 1, 1.3), + ('2017-01-02 08:01', 2, 1.6), + ('2017-01-02 06:01', 2, 1.4), + ('2017-01-03 01:01', 3, 2), + ('2017-01-03 01:16', 3, 3), + ('2017-01-03 01:17', 3, 4), + ('2018-01-13 01:01', 1, 2), + ('2018-01-13 01:10', 1, 0.4), + ('2018-01-13 02:10', 2, 1.4), + ('2018-01-13 05:01', 2, 2), + ('2018-01-13 05:50', 2, 4), + ('2018-01-13 16:01', 3, 2); +-- Since there are no stats we use shared buffers size to estimate number of rows +EXPLAIN (COSTS ON) +SELECT * +FROM hyper_estimate; + QUERY PLAN +--------------------------------------------------------------------------------------------- + Append (cost=100.00..166847.40 rows=4118040 width=20) + -> Foreign Scan on _hyper_16_37_dist_chunk (cost=100.00..32468.60 rows=915120 width=20) + -> Foreign Scan on _hyper_16_38_dist_chunk (cost=100.00..32468.60 rows=915120 width=20) + -> Foreign Scan on _hyper_16_39_dist_chunk (cost=100.00..32468.60 rows=915120 width=20) + -> Foreign Scan on _hyper_16_40_dist_chunk (cost=100.00..16283.80 rows=457560 width=20) + -> Foreign Scan on _hyper_16_41_dist_chunk (cost=100.00..16283.80 rows=457560 width=20) + -> Foreign Scan on _hyper_16_42_dist_chunk (cost=100.00..16283.80 rows=457560 width=20) +(7 rows) + +-- This will calculate the stats +ANALYZE hyper_estimate; +EXPLAIN (COSTS ON) +SELECT * +FROM hyper_estimate; + QUERY PLAN +-------------------------------------------------------------------------------------- + Append (cost=100.00..606.52 rows=15 width=20) + -> Foreign Scan on _hyper_16_37_dist_chunk (cost=100.00..101.12 rows=4 width=20) + -> Foreign Scan on _hyper_16_38_dist_chunk (cost=100.00..101.06 rows=2 width=20) + -> Foreign Scan on _hyper_16_39_dist_chunk (cost=100.00..101.09 rows=3 width=20) + -> Foreign Scan on _hyper_16_40_dist_chunk (cost=100.00..101.06 rows=2 width=20) + -> Foreign Scan on _hyper_16_41_dist_chunk (cost=100.00..101.09 rows=3 width=20) + -> Foreign Scan on _hyper_16_42_dist_chunk (cost=100.00..101.03 rows=1 width=20) +(7 rows) + +-- Let's insert data into a new chunk. This will result in chunk creation. +INSERT INTO hyper_estimate VALUES ('2019-11-11 06:01', 1, 1.1); +-- We have stats for previous chunks so we can interpolate number of rows for the new chunk +EXPLAIN (COSTS ON) +SELECT * +FROM hyper_estimate; + QUERY PLAN +-------------------------------------------------------------------------------------- + Append (cost=100.00..706.58 rows=17 width=20) + -> Foreign Scan on _hyper_16_37_dist_chunk (cost=100.00..101.12 rows=4 width=20) + -> Foreign Scan on _hyper_16_38_dist_chunk (cost=100.00..101.06 rows=2 width=20) + -> Foreign Scan on _hyper_16_39_dist_chunk (cost=100.00..101.09 rows=3 width=20) + -> Foreign Scan on _hyper_16_40_dist_chunk (cost=100.00..101.06 rows=2 width=20) + -> Foreign Scan on _hyper_16_41_dist_chunk (cost=100.00..101.09 rows=3 width=20) + -> Foreign Scan on _hyper_16_42_dist_chunk (cost=100.00..101.03 rows=1 width=20) + -> Foreign Scan on _hyper_16_43_dist_chunk (cost=100.00..100.05 rows=2 width=20) +(8 rows) + diff --git a/tsl/test/expected/partitionwise_distributed-11.out b/tsl/test/expected/partitionwise_distributed-11.out index 875fa4278..31f62ab03 100644 --- a/tsl/test/expected/partitionwise_distributed-11.out +++ b/tsl/test/expected/partitionwise_distributed-11.out @@ -14,9 +14,279 @@ CREATE OR REPLACE FUNCTION test.remote_exec(srv_name name[], command text) RETURNS VOID AS :TSL_MODULE_PATHNAME, 'ts_remote_exec' LANGUAGE C; -CREATE OR REPLACE FUNCTION ts_test_override_pushdown_timestamptz(new_value TIMESTAMPTZ) RETURNS VOID -AS :TSL_MODULE_PATHNAME, 'ts_test_override_pushdown_timestamptz' -LANGUAGE C VOLATILE STRICT; +\ir utils/testsupport.sql +-- This file and its contents are licensed under the Apache License 2.0. +-- Please see the included NOTICE for copyright information and +-- LICENSE-APACHE for a copy of the license. +SELECT _timescaledb_internal.stop_background_workers(); + stop_background_workers +------------------------- + t +(1 row) + +CREATE SCHEMA IF NOT EXISTS test; +psql:utils/testsupport.sql:8: NOTICE: schema "test" already exists, skipping +GRANT USAGE ON SCHEMA test TO PUBLIC; +-- Utility functions to show relation information in tests. These +-- functions generate output which is the same across PostgreSQL +-- versions. Their usage is preferred over psql's '\d ', +-- since that output typically changes across PostgreSQL versions. +-- this function is duplicated in test/isolation/specs/multi_transaction_indexing.spec +-- if it changes, that copy may need to change as well +CREATE OR REPLACE FUNCTION test.show_columns(rel regclass) +RETURNS TABLE("Column" name, + "Type" text, + "NotNull" boolean) LANGUAGE SQL STABLE AS +$BODY$ + SELECT a.attname, + format_type(t.oid, t.typtypmod), + a.attnotnull + FROM pg_attribute a, pg_type t + WHERE a.attrelid = rel + AND a.atttypid = t.oid + AND a.attnum >= 0 + ORDER BY a.attnum; +$BODY$; +CREATE OR REPLACE FUNCTION test.show_columnsp(pattern text) +RETURNS TABLE("Relation" regclass, + "Kind" "char", + "Column" name, + "Column type" text, + "NotNull" boolean) LANGUAGE PLPGSQL STABLE AS +$BODY$ +DECLARE + schema_name name = split_part(pattern, '.', 1); + table_name name = split_part(pattern, '.', 2); +BEGIN + IF schema_name = '' OR table_name = '' THEN + schema_name := current_schema(); + table_name := pattern; + END IF; + + RETURN QUERY + SELECT c.oid::regclass, + c.relkind, + a.attname, + format_type(t.oid, t.typtypmod), + a.attnotnull + FROM pg_class c, pg_attribute a, pg_type t + WHERE format('%I.%I', c.relnamespace::regnamespace::name, c.relname) LIKE format('%I.%s', schema_name, table_name) + AND a.attrelid = c.oid + AND a.atttypid = t.oid + AND a.attnum >= 0 + ORDER BY c.relname, a.attnum; +END +$BODY$; +CREATE OR REPLACE FUNCTION test.show_indexes(rel regclass) +RETURNS TABLE("Index" regclass, + "Columns" name[], + "Expr" text, + "Unique" boolean, + "Primary" boolean, + "Exclusion" boolean, + "Tablespace" name) LANGUAGE SQL STABLE AS +$BODY$ + SELECT c.oid::regclass, + array(SELECT "Column" FROM test.show_columns(i.indexrelid)), + pg_get_expr(i.indexprs, c.oid, true), + i.indisunique, + i.indisprimary, + i.indisexclusion, + (SELECT t.spcname FROM pg_tablespace t WHERE t.oid = c.reltablespace) + FROM pg_class c, pg_index i + WHERE c.oid = i.indexrelid AND i.indrelid = rel + ORDER BY c.relname; +$BODY$; +-- this function is duplicated in test/isolation/specs/multi_transaction_indexing.spec +-- if it changes, that copy may need to change as well +CREATE OR REPLACE FUNCTION test.show_indexesp(pattern text) +RETURNS TABLE("Table" regclass, + "Index" regclass, + "Columns" name[], + "Expr" text, + "Unique" boolean, + "Primary" boolean, + "Exclusion" boolean, + "Tablespace" name) LANGUAGE PLPGSQL STABLE AS +$BODY$ +DECLARE + schema_name name = split_part(pattern, '.', 1); + table_name name = split_part(pattern, '.', 2); +BEGIN + IF schema_name = '' OR table_name = '' THEN + schema_name := current_schema(); + table_name := pattern; + END IF; + + RETURN QUERY + SELECT c.oid::regclass, + i.indexrelid::regclass, + array(SELECT "Column" FROM test.show_columns(i.indexrelid)), + pg_get_expr(i.indexprs, c.oid, true), + i.indisunique, + i.indisprimary, + i.indisexclusion, + (SELECT t.spcname FROM pg_class cc, pg_tablespace t WHERE cc.oid = i.indexrelid AND t.oid = cc.reltablespace) + FROM pg_class c, pg_index i + WHERE format('%I.%I', c.relnamespace::regnamespace::name, c.relname) LIKE format('%I.%s', schema_name, table_name) + AND c.oid = i.indrelid + ORDER BY c.oid, i.indexrelid; +END +$BODY$; +CREATE OR REPLACE FUNCTION test.show_constraints(rel regclass) +RETURNS TABLE("Constraint" name, + "Type" "char", + "Columns" name[], + "Index" regclass, + "Expr" text, + "Deferrable" bool, + "Deferred" bool, + "Validated" bool) LANGUAGE SQL STABLE AS +$BODY$ + SELECT c.conname, + c.contype, + array(SELECT attname FROM pg_attribute a, unnest(conkey) k WHERE a.attrelid = rel AND k = a.attnum), + c.conindid::regclass, + pg_get_expr(c.conbin, c.conrelid), + c.condeferrable, + c.condeferred, + c.convalidated + FROM pg_constraint c + WHERE c.conrelid = rel + ORDER BY c.conname; +$BODY$; +CREATE OR REPLACE FUNCTION test.show_constraintsp(pattern text) +RETURNS TABLE("Table" regclass, + "Constraint" name, + "Type" "char", + "Columns" name[], + "Index" regclass, + "Expr" text, + "Deferrable" bool, + "Deferred" bool, + "Validated" bool) LANGUAGE PLPGSQL STABLE AS +$BODY$ +DECLARE + schema_name name = split_part(pattern, '.', 1); + table_name name = split_part(pattern, '.', 2); +BEGIN + IF schema_name = '' OR table_name = '' THEN + schema_name := current_schema(); + table_name := pattern; + END IF; + + RETURN QUERY + SELECT cl.oid::regclass, + c.conname, + c.contype, + array(SELECT attname FROM pg_attribute a, unnest(conkey) k WHERE a.attrelid = cl.oid AND k = a.attnum), + c.conindid::regclass, + pg_get_expr(c.conbin, c.conrelid), + c.condeferrable, + c.condeferred, + c.convalidated + FROM pg_class cl, pg_constraint c + WHERE format('%I.%I', cl.relnamespace::regnamespace::name, cl.relname) LIKE format('%I.%s', schema_name, table_name) + AND c.conrelid = cl.oid + ORDER BY cl.relname, c.conname; +END +$BODY$; +CREATE OR REPLACE FUNCTION test.show_triggers(rel regclass, show_internal boolean = false) +RETURNS TABLE("Trigger" name, + "Type" smallint, + "Function" regproc) LANGUAGE SQL STABLE AS +$BODY$ + SELECT t.tgname, + t.tgtype, + t.tgfoid::regproc + FROM pg_trigger t + WHERE t.tgrelid = rel + AND t.tgisinternal = show_internal + ORDER BY t.tgname; +$BODY$; +CREATE OR REPLACE FUNCTION test.show_triggersp(pattern text, show_internal boolean = false) +RETURNS TABLE("Table" regclass, + "Trigger" name, + "Type" smallint, + "Function" regproc) LANGUAGE PLPGSQL STABLE AS +$BODY$ +DECLARE + schema_name name = split_part(pattern, '.', 1); + table_name name = split_part(pattern, '.', 2); +BEGIN + IF schema_name = '' OR table_name = '' THEN + schema_name := current_schema(); + table_name := pattern; + END IF; + + RETURN QUERY + SELECT t.tgrelid::regclass, + t.tgname, + t.tgtype, + t.tgfoid::regproc + FROM pg_class cl, pg_trigger t + WHERE format('%I.%I', cl.relnamespace::regnamespace::name, cl.relname) LIKE format('%I.%s', schema_name, table_name) + AND t.tgrelid = cl.oid + AND t.tgisinternal = show_internal + ORDER BY t.tgrelid, t.tgname; +END +$BODY$; +CREATE OR REPLACE FUNCTION test.show_subtables(rel regclass) +RETURNS TABLE("Child" regclass, + "Tablespace" name) LANGUAGE SQL STABLE AS +$BODY$ + SELECT objid::regclass, (SELECT t.spcname FROM pg_tablespace t WHERE t.oid = c.reltablespace) + FROM pg_depend d, pg_class c + WHERE d.refobjid = rel + AND d.deptype = 'n' + AND d.classid = 'pg_class'::regclass + AND d.objid = c.oid + ORDER BY d.refobjid, d.objid; +$BODY$; +CREATE OR REPLACE FUNCTION test.show_subtablesp(pattern text) +RETURNS TABLE("Parent" regclass, + "Child" regclass, + "Tablespace" name) LANGUAGE PLPGSQL STABLE AS +$BODY$ +DECLARE + schema_name name = split_part(pattern, '.', 1); + table_name name = split_part(pattern, '.', 2); +BEGIN + IF schema_name = '' OR table_name = '' THEN + schema_name := current_schema(); + table_name := pattern; + END IF; + + RETURN QUERY + SELECT refobjid::regclass, + objid::regclass, + (SELECT t.spcname FROM pg_class cc, pg_tablespace t WHERE cc.oid = d.objid AND t.oid = cc.reltablespace) + FROM pg_class c, pg_depend d + WHERE format('%I.%I', c.relnamespace::regnamespace::name, c.relname) LIKE format('%I.%s', schema_name, table_name) + AND d.refobjid = c.oid + AND d.deptype = 'n' + AND d.classid = 'pg_class'::regclass + ORDER BY d.refobjid, d.objid; +END +$BODY$; +CREATE OR REPLACE FUNCTION test.execute_sql(cmd TEXT) +RETURNS TEXT LANGUAGE PLPGSQL AS $BODY$ +BEGIN + EXECUTE cmd; + RETURN cmd; +END +$BODY$; +-- Used to set a deterministic memory setting during tests +CREATE OR REPLACE FUNCTION test.set_memory_cache_size(memory_amount text) +RETURNS BIGINT AS :MODULE_PATHNAME, 'ts_set_memory_cache_size' LANGUAGE C VOLATILE STRICT; +CREATE OR REPLACE FUNCTION test.empty_trigger_func() + RETURNS TRIGGER LANGUAGE PLPGSQL AS +$BODY$ +BEGIN +END +$BODY$; +CREATE OR REPLACE FUNCTION ts_test_override_current_timestamptz(new_value TIMESTAMPTZ) +RETURNS VOID AS :TSL_MODULE_PATHNAME, 'ts_test_override_current_timestamptz' LANGUAGE C VOLATILE STRICT; -- Cleanup from other potential tests that created these databases SET client_min_messages TO ERROR; DROP DATABASE IF EXISTS data_node_1; @@ -401,28 +671,30 @@ FROM hyper WHERE time < '2018-06-01 00:00' GROUP BY 1 ORDER BY 1; - QUERY PLAN ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + QUERY PLAN +-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- Finalize GroupAggregate Output: "time", avg(temp) Group Key: "time" - -> Custom Scan (AsyncAppend) + -> Sort Output: "time", (PARTIAL avg(temp)) - -> Merge Append - Sort Key: hyper."time" - -> Custom Scan (DataNodeScan) - Output: hyper."time", (PARTIAL avg(hyper.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_1 - Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk - Remote SQL: SELECT "time", _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 ORDER BY "time" ASC NULLS LAST - -> Custom Scan (DataNodeScan) - Output: hyper_1."time", (PARTIAL avg(hyper_1.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_2 - Chunks: _hyper_1_2_dist_chunk, _hyper_1_4_dist_chunk - Remote SQL: SELECT "time", _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 ORDER BY "time" ASC NULLS LAST -(19 rows) + Sort Key: "time" + -> Custom Scan (AsyncAppend) + Output: "time", (PARTIAL avg(temp)) + -> Append + -> Custom Scan (DataNodeScan) + Output: hyper."time", (PARTIAL avg(hyper.temp)) + Relations: Aggregate on (public.hyper) + Data node: data_node_1 + Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk + Remote SQL: SELECT "time", _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 + -> Custom Scan (DataNodeScan) + Output: hyper_1."time", (PARTIAL avg(hyper_1.temp)) + Relations: Aggregate on (public.hyper) + Data node: data_node_2 + Chunks: _hyper_1_2_dist_chunk, _hyper_1_4_dist_chunk + Remote SQL: SELECT "time", _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 +(21 rows) -- Show result SELECT time, avg(temp) @@ -615,25 +887,27 @@ FROM hyper WHERE time < '2018-06-01 00:00' GROUP BY 1 ORDER BY 1; - QUERY PLAN -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- - Custom Scan (AsyncAppend) + QUERY PLAN +------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ + Sort Output: device, (avg(temp)) - -> Merge Append - Sort Key: hyper.device - -> Custom Scan (DataNodeScan) - Output: hyper.device, (avg(hyper.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_1 - Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk - Remote SQL: SELECT device, avg(temp) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 ORDER BY device ASC NULLS LAST - -> Custom Scan (DataNodeScan) - Output: hyper_1.device, (avg(hyper_1.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_2 - Chunks: _hyper_1_4_dist_chunk, _hyper_1_2_dist_chunk - Remote SQL: SELECT device, avg(temp) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 ORDER BY device ASC NULLS LAST -(16 rows) + Sort Key: device + -> Custom Scan (AsyncAppend) + Output: device, (avg(temp)) + -> Append + -> Custom Scan (DataNodeScan) + Output: hyper.device, (avg(hyper.temp)) + Relations: Aggregate on (public.hyper) + Data node: data_node_1 + Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk + Remote SQL: SELECT device, avg(temp) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 + -> Custom Scan (DataNodeScan) + Output: hyper_1.device, (avg(hyper_1.temp)) + Relations: Aggregate on (public.hyper) + Data node: data_node_2 + Chunks: _hyper_1_4_dist_chunk, _hyper_1_2_dist_chunk + Remote SQL: SELECT device, avg(temp) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 +(18 rows) -- Show result SELECT device, avg(temp) @@ -659,27 +933,30 @@ SELECT location, avg(temp) FROM hyper WHERE time < '2018-06-01 00:00' GROUP BY 1; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- - Finalize HashAggregate + QUERY PLAN +---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + Finalize GroupAggregate Output: location, avg(temp) Group Key: location - -> Custom Scan (AsyncAppend) + -> Sort Output: location, (PARTIAL avg(temp)) - -> Append - -> Custom Scan (DataNodeScan) - Output: hyper.location, (PARTIAL avg(hyper.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_1 - Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk - Remote SQL: SELECT location, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 - -> Custom Scan (DataNodeScan) - Output: hyper_1.location, (PARTIAL avg(hyper_1.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_2 - Chunks: _hyper_1_4_dist_chunk, _hyper_1_2_dist_chunk - Remote SQL: SELECT location, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 -(18 rows) + Sort Key: location + -> Custom Scan (AsyncAppend) + Output: location, (PARTIAL avg(temp)) + -> Append + -> Custom Scan (DataNodeScan) + Output: hyper.location, (PARTIAL avg(hyper.temp)) + Relations: Aggregate on (public.hyper) + Data node: data_node_1 + Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk + Remote SQL: SELECT location, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 + -> Custom Scan (DataNodeScan) + Output: hyper_1.location, (PARTIAL avg(hyper_1.temp)) + Relations: Aggregate on (public.hyper) + Data node: data_node_2 + Chunks: _hyper_1_4_dist_chunk, _hyper_1_2_dist_chunk + Remote SQL: SELECT location, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 +(21 rows) -- Expand query across repartition boundary. This makes it unsafe to -- push down the FULL agg, so should expect a PARTIAL agg on @@ -750,12 +1027,12 @@ GROUP BY 1 ORDER BY 1; QUERY PLAN ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ - Sort - Output: device, (avg(temp)) - Sort Key: device - -> Finalize HashAggregate - Output: device, avg(temp) - Group Key: device + Finalize GroupAggregate + Output: device, avg(temp) + Group Key: device + -> Sort + Output: device, (PARTIAL avg(temp)) + Sort Key: device -> Custom Scan (AsyncAppend) Output: device, (PARTIAL avg(temp)) -> Append @@ -843,28 +1120,30 @@ FROM hyper WHERE time > '2018-04-19 00:01' GROUP BY 1 ORDER BY 1; - QUERY PLAN ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + QUERY PLAN +----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- Finalize GroupAggregate Output: device, avg(temp) Group Key: device - -> Custom Scan (AsyncAppend) + -> Sort Output: device, (PARTIAL avg(temp)) - -> Merge Append - Sort Key: hyper.device - -> Custom Scan (DataNodeScan) - Output: hyper.device, (PARTIAL avg(hyper.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_1 - Chunks: _hyper_1_3_dist_chunk, _hyper_1_5_dist_chunk, _hyper_1_7_dist_chunk - Remote SQL: SELECT device, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 3, 4]) AND (("time" > '2018-04-19 00:01:00-07'::timestamp with time zone)) GROUP BY 1 ORDER BY device ASC NULLS LAST - -> Custom Scan (DataNodeScan) - Output: hyper_1.device, (PARTIAL avg(hyper_1.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_2 - Chunks: _hyper_1_4_dist_chunk, _hyper_1_6_dist_chunk - Remote SQL: SELECT device, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 3]) AND (("time" > '2018-04-19 00:01:00-07'::timestamp with time zone)) GROUP BY 1 ORDER BY device ASC NULLS LAST -(19 rows) + Sort Key: device + -> Custom Scan (AsyncAppend) + Output: device, (PARTIAL avg(temp)) + -> Append + -> Custom Scan (DataNodeScan) + Output: hyper.device, (PARTIAL avg(hyper.temp)) + Relations: Aggregate on (public.hyper) + Data node: data_node_1 + Chunks: _hyper_1_3_dist_chunk, _hyper_1_5_dist_chunk, _hyper_1_7_dist_chunk + Remote SQL: SELECT device, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 3, 4]) AND (("time" > '2018-04-19 00:01:00-07'::timestamp with time zone)) GROUP BY 1 + -> Custom Scan (DataNodeScan) + Output: hyper_1.device, (PARTIAL avg(hyper_1.temp)) + Relations: Aggregate on (public.hyper) + Data node: data_node_2 + Chunks: _hyper_1_4_dist_chunk, _hyper_1_6_dist_chunk + Remote SQL: SELECT device, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 3]) AND (("time" > '2018-04-19 00:01:00-07'::timestamp with time zone)) GROUP BY 1 +(21 rows) SELECT device, avg(temp) FROM hyper @@ -944,28 +1223,26 @@ FROM hyper WHERE time < '2018-06-01 00:00' GROUP BY 1, 2 ORDER BY 1, 2; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- - Sort - Output: hyper."time", hyper.device, (avg(hyper.temp)) - Sort Key: hyper."time", hyper.device - -> HashAggregate - Output: hyper."time", hyper.device, avg(hyper.temp) - Group Key: hyper."time", hyper.device - -> Custom Scan (AsyncAppend) - Output: hyper."time", hyper.device, hyper.temp - -> Append - -> Custom Scan (DataNodeScan) on public.hyper hyper_1 - Output: hyper_1."time", hyper_1.device, hyper_1.temp - Data node: data_node_1 - Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk - Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) - -> Custom Scan (DataNodeScan) on public.hyper hyper_2 - Output: hyper_2."time", hyper_2.device, hyper_2.temp - Data node: data_node_2 - Chunks: _hyper_1_2_dist_chunk, _hyper_1_4_dist_chunk - Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) -(19 rows) + QUERY PLAN +---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + GroupAggregate + Output: hyper."time", hyper.device, avg(hyper.temp) + Group Key: hyper."time", hyper.device + -> Custom Scan (AsyncAppend) + Output: hyper."time", hyper.device, hyper.temp + -> Merge Append + Sort Key: hyper_1."time", hyper_1.device + -> Custom Scan (DataNodeScan) on public.hyper hyper_1 + Output: hyper_1."time", hyper_1.device, hyper_1.temp + Data node: data_node_1 + Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST + -> Custom Scan (DataNodeScan) on public.hyper hyper_2 + Output: hyper_2."time", hyper_2.device, hyper_2.temp + Data node: data_node_2 + Chunks: _hyper_1_2_dist_chunk, _hyper_1_4_dist_chunk + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST +(17 rows) SELECT time, device, avg(temp) FROM hyper @@ -1271,29 +1548,28 @@ FROM hyper WHERE time BETWEEN '2018-04-19 00:01' AND '2018-06-01 00:00' GROUP BY 1, 2 ORDER BY 1, 2; - QUERY PLAN ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ - Custom Scan (AsyncAppend) - Output: "time", device, (avg(temp)) - -> Merge Append - Sort Key: hyper."time", hyper.device - -> GroupAggregate - Output: hyper."time", hyper.device, avg(hyper.temp) - Group Key: hyper."time", hyper.device - -> Custom Scan (DataNodeScan) on public.hyper - Output: hyper."time", hyper.device, hyper.temp - Data node: data_node_1 - Chunks: _hyper_1_3_dist_chunk - Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2]) AND (("time" >= '2018-04-19 00:01:00-07'::timestamp with time zone)) AND (("time" <= '2018-06-01 00:00:00-07'::timestamp with time zone)) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST - -> GroupAggregate - Output: hyper_1."time", hyper_1.device, avg(hyper_1.temp) - Group Key: hyper_1."time", hyper_1.device - -> Custom Scan (DataNodeScan) on public.hyper hyper_1 - Output: hyper_1."time", hyper_1.device, hyper_1.temp - Data node: data_node_2 - Chunks: _hyper_1_4_dist_chunk - Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2]) AND (("time" >= '2018-04-19 00:01:00-07'::timestamp with time zone)) AND (("time" <= '2018-06-01 00:00:00-07'::timestamp with time zone)) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST -(20 rows) + QUERY PLAN +----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + Sort + Output: hyper."time", hyper.device, (avg(hyper.temp)) + Sort Key: hyper."time", hyper.device + -> HashAggregate + Output: hyper."time", hyper.device, avg(hyper.temp) + Group Key: hyper."time", hyper.device + -> Custom Scan (AsyncAppend) + Output: hyper."time", hyper.device, hyper.temp + -> Append + -> Custom Scan (DataNodeScan) on public.hyper hyper_1 + Output: hyper_1."time", hyper_1.device, hyper_1.temp + Data node: data_node_1 + Chunks: _hyper_1_3_dist_chunk + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2]) AND (("time" >= '2018-04-19 00:01:00-07'::timestamp with time zone)) AND (("time" <= '2018-06-01 00:00:00-07'::timestamp with time zone)) + -> Custom Scan (DataNodeScan) on public.hyper hyper_2 + Output: hyper_2."time", hyper_2.device, hyper_2.temp + Data node: data_node_2 + Chunks: _hyper_1_4_dist_chunk + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2]) AND (("time" >= '2018-04-19 00:01:00-07'::timestamp with time zone)) AND (("time" <= '2018-06-01 00:00:00-07'::timestamp with time zone)) +(19 rows) -- Test HAVING qual EXPLAIN (VERBOSE, COSTS OFF) @@ -1345,31 +1621,29 @@ WHERE time BETWEEN '2018-04-19 00:01' AND '2018-06-01 00:00' GROUP BY 1, 2 HAVING avg(temp) > 4 ORDER BY 1, 2; - QUERY PLAN ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ - Custom Scan (AsyncAppend) - Output: "time", device, (avg(temp)) - -> Merge Append - Sort Key: hyper."time", hyper.device - -> GroupAggregate - Output: hyper."time", hyper.device, avg(hyper.temp) - Group Key: hyper."time", hyper.device - Filter: (avg(hyper.temp) > '4'::double precision) - -> Custom Scan (DataNodeScan) on public.hyper - Output: hyper."time", hyper.device, hyper.temp - Data node: data_node_1 - Chunks: _hyper_1_3_dist_chunk - Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2]) AND (("time" >= '2018-04-19 00:01:00-07'::timestamp with time zone)) AND (("time" <= '2018-06-01 00:00:00-07'::timestamp with time zone)) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST - -> GroupAggregate - Output: hyper_1."time", hyper_1.device, avg(hyper_1.temp) - Group Key: hyper_1."time", hyper_1.device - Filter: (avg(hyper_1.temp) > '4'::double precision) - -> Custom Scan (DataNodeScan) on public.hyper hyper_1 - Output: hyper_1."time", hyper_1.device, hyper_1.temp - Data node: data_node_2 - Chunks: _hyper_1_4_dist_chunk - Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2]) AND (("time" >= '2018-04-19 00:01:00-07'::timestamp with time zone)) AND (("time" <= '2018-06-01 00:00:00-07'::timestamp with time zone)) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST -(22 rows) + QUERY PLAN +----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + Sort + Output: hyper."time", hyper.device, (avg(hyper.temp)) + Sort Key: hyper."time", hyper.device + -> HashAggregate + Output: hyper."time", hyper.device, avg(hyper.temp) + Group Key: hyper."time", hyper.device + Filter: (avg(hyper.temp) > '4'::double precision) + -> Custom Scan (AsyncAppend) + Output: hyper."time", hyper.device, hyper.temp + -> Append + -> Custom Scan (DataNodeScan) on public.hyper hyper_1 + Output: hyper_1."time", hyper_1.device, hyper_1.temp + Data node: data_node_1 + Chunks: _hyper_1_3_dist_chunk + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2]) AND (("time" >= '2018-04-19 00:01:00-07'::timestamp with time zone)) AND (("time" <= '2018-06-01 00:00:00-07'::timestamp with time zone)) + -> Custom Scan (DataNodeScan) on public.hyper hyper_2 + Output: hyper_2."time", hyper_2.device, hyper_2.temp + Data node: data_node_2 + Chunks: _hyper_1_4_dist_chunk + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2]) AND (("time" >= '2018-04-19 00:01:00-07'::timestamp with time zone)) AND (("time" <= '2018-06-01 00:00:00-07'::timestamp with time zone)) +(20 rows) SELECT time, device, avg(temp) AS temp FROM hyper @@ -1427,28 +1701,26 @@ FROM hyper WHERE time < '2018-06-01 00:00' GROUP BY 1, 2 ORDER BY 1, 2; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- - Sort - Output: (date_trunc('month'::text, hyper."time")), hyper.device, (avg(hyper.temp)) - Sort Key: (date_trunc('month'::text, hyper."time")), hyper.device - -> HashAggregate - Output: (date_trunc('month'::text, hyper."time")), hyper.device, avg(hyper.temp) - Group Key: (date_trunc('month'::text, hyper."time")), hyper.device - -> Custom Scan (AsyncAppend) - Output: (date_trunc('month'::text, hyper."time")), hyper.device, hyper.temp - -> Append - -> Custom Scan (DataNodeScan) on public.hyper hyper_1 - Output: date_trunc('month'::text, hyper_1."time"), hyper_1.device, hyper_1.temp - Data node: data_node_1 - Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk - Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) - -> Custom Scan (DataNodeScan) on public.hyper hyper_2 - Output: date_trunc('month'::text, hyper_2."time"), hyper_2.device, hyper_2.temp - Data node: data_node_2 - Chunks: _hyper_1_4_dist_chunk, _hyper_1_2_dist_chunk - Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) -(19 rows) + QUERY PLAN +------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + GroupAggregate + Output: (date_trunc('month'::text, hyper."time")), hyper.device, avg(hyper.temp) + Group Key: (date_trunc('month'::text, hyper."time")), hyper.device + -> Custom Scan (AsyncAppend) + Output: (date_trunc('month'::text, hyper."time")), hyper.device, hyper.temp + -> Merge Append + Sort Key: (date_trunc('month'::text, hyper_1."time")), hyper_1.device + -> Custom Scan (DataNodeScan) on public.hyper hyper_1 + Output: date_trunc('month'::text, hyper_1."time"), hyper_1.device, hyper_1.temp + Data node: data_node_1 + Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) ORDER BY date_trunc('month'::text, "time") ASC NULLS LAST, device ASC NULLS LAST + -> Custom Scan (DataNodeScan) on public.hyper hyper_2 + Output: date_trunc('month'::text, hyper_2."time"), hyper_2.device, hyper_2.temp + Data node: data_node_2 + Chunks: _hyper_1_4_dist_chunk, _hyper_1_2_dist_chunk + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) ORDER BY date_trunc('month'::text, "time") ASC NULLS LAST, device ASC NULLS LAST +(17 rows) SELECT date_trunc('month', time), device, avg(temp) FROM hyper @@ -1686,28 +1958,26 @@ FROM hyper WHERE time < '2018-06-01 00:00' GROUP BY 1, 2 ORDER BY 1, 2; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- - Sort - Output: (time_bucket('@ 1 day'::interval, hyper."time")), hyper.device, (avg(hyper.temp)) - Sort Key: (time_bucket('@ 1 day'::interval, hyper."time")), hyper.device - -> HashAggregate - Output: (time_bucket('@ 1 day'::interval, hyper."time")), hyper.device, avg(hyper.temp) - Group Key: (time_bucket('@ 1 day'::interval, hyper."time")), hyper.device - -> Custom Scan (AsyncAppend) - Output: (time_bucket('@ 1 day'::interval, hyper."time")), hyper.device, hyper.temp - -> Append - -> Custom Scan (DataNodeScan) on public.hyper hyper_1 - Output: time_bucket('@ 1 day'::interval, hyper_1."time"), hyper_1.device, hyper_1.temp - Data node: data_node_1 - Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk - Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) - -> Custom Scan (DataNodeScan) on public.hyper hyper_2 - Output: time_bucket('@ 1 day'::interval, hyper_2."time"), hyper_2.device, hyper_2.temp - Data node: data_node_2 - Chunks: _hyper_1_4_dist_chunk, _hyper_1_2_dist_chunk - Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) -(19 rows) + QUERY PLAN +------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + GroupAggregate + Output: (time_bucket('@ 1 day'::interval, hyper."time")), hyper.device, avg(hyper.temp) + Group Key: (time_bucket('@ 1 day'::interval, hyper."time")), hyper.device + -> Custom Scan (AsyncAppend) + Output: (time_bucket('@ 1 day'::interval, hyper."time")), hyper.device, hyper.temp + -> Merge Append + Sort Key: (time_bucket('@ 1 day'::interval, hyper_1."time")), hyper_1.device + -> Custom Scan (DataNodeScan) on public.hyper hyper_1 + Output: time_bucket('@ 1 day'::interval, hyper_1."time"), hyper_1.device, hyper_1.temp + Data node: data_node_1 + Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) ORDER BY public.time_bucket('1 day'::interval, "time") ASC NULLS LAST, device ASC NULLS LAST + -> Custom Scan (DataNodeScan) on public.hyper hyper_2 + Output: time_bucket('@ 1 day'::interval, hyper_2."time"), hyper_2.device, hyper_2.temp + Data node: data_node_2 + Chunks: _hyper_1_4_dist_chunk, _hyper_1_2_dist_chunk + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) ORDER BY public.time_bucket('1 day'::interval, "time") ASC NULLS LAST, device ASC NULLS LAST +(17 rows) -- Show result SELECT time_bucket('1 day', time), device, avg(temp) @@ -1796,19 +2066,21 @@ WHERE time < '2018-06-01 00:00' AND device = 1 GROUP BY 1, 2 ORDER BY 1, 2; - QUERY PLAN ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ - Sort - Output: (time_bucket('@ 1 day'::interval, hyper."time")), hyper.device, (avg(hyper.temp)) - Sort Key: (time_bucket('@ 1 day'::interval, hyper."time")) - -> Append - -> Custom Scan (DataNodeScan) - Output: (time_bucket('@ 1 day'::interval, hyper."time")), hyper.device, (avg(hyper.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_1 - Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk - Remote SQL: SELECT public.time_bucket('@ 1 day'::interval, "time"), device, avg(temp) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) AND ((device = 1)) GROUP BY 1, 2 -(10 rows) + QUERY PLAN +----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + GroupAggregate + Output: (time_bucket('@ 1 day'::interval, hyper."time")), hyper.device, avg(hyper.temp) + Group Key: (time_bucket('@ 1 day'::interval, hyper."time")), hyper.device + -> Sort + Output: (time_bucket('@ 1 day'::interval, hyper."time")), hyper.device, hyper.temp + Sort Key: (time_bucket('@ 1 day'::interval, hyper."time")) + -> Append + -> Custom Scan (DataNodeScan) on public.hyper + Output: time_bucket('@ 1 day'::interval, hyper."time"), hyper.device, hyper.temp + Data node: data_node_1 + Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) AND ((device = 1)) +(12 rows) SELECT time_bucket('1 day', time), device, avg(temp) FROM hyper @@ -1884,8 +2156,7 @@ LIMIT 1; Output: "time", device, (avg(temp)), (sum((temp * (((random() <= '1'::double precision))::integer)::double precision))) -> Custom Scan (AsyncAppend) Output: "time", device, (avg(temp)), (sum((temp * (((random() <= '1'::double precision))::integer)::double precision))) - -> Merge Append - Sort Key: hyper."time", hyper.device + -> Append -> GroupAggregate Output: hyper."time", hyper.device, avg(hyper.temp), sum((hyper.temp * (((random() <= '1'::double precision))::integer)::double precision)) Group Key: hyper."time", hyper.device @@ -1902,7 +2173,7 @@ LIMIT 1; Data node: data_node_2 Chunks: _hyper_1_4_dist_chunk, _hyper_1_2_dist_chunk Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST -(22 rows) +(21 rows) -- Pushed down with non-pushable expression taken out EXPLAIN (VERBOSE, COSTS OFF) @@ -1944,29 +2215,29 @@ FROM hyper WHERE time < '2018-06-01 00:00' GROUP BY 1, 2 LIMIT 1; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + QUERY PLAN +---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- Limit Output: "time", device, (avg(temp)), ((random() * (device)::double precision)), (custom_sum(device)) -> Custom Scan (AsyncAppend) Output: "time", device, (avg(temp)), ((random() * (device)::double precision)), (custom_sum(device)) -> Append - -> HashAggregate + -> GroupAggregate Output: hyper."time", hyper.device, avg(hyper.temp), (random() * (hyper.device)::double precision), custom_sum(hyper.device) Group Key: hyper."time", hyper.device -> Custom Scan (DataNodeScan) on public.hyper Output: hyper."time", hyper.device, hyper.temp Data node: data_node_1 Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk - Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) - -> HashAggregate + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST + -> GroupAggregate Output: hyper_1."time", hyper_1.device, avg(hyper_1.temp), (random() * (hyper_1.device)::double precision), custom_sum(hyper_1.device) Group Key: hyper_1."time", hyper_1.device -> Custom Scan (DataNodeScan) on public.hyper hyper_1 Output: hyper_1."time", hyper_1.device, hyper_1.temp Data node: data_node_2 Chunks: _hyper_1_4_dist_chunk, _hyper_1_2_dist_chunk - Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST (21 rows) EXPLAIN (VERBOSE, COSTS OFF) @@ -2018,8 +2289,7 @@ LIMIT 1; Output: "time", device, (avg(temp)), (((sum(temp) * random()) * (device)::double precision)) -> Custom Scan (AsyncAppend) Output: "time", device, (avg(temp)), (((sum(temp) * random()) * (device)::double precision)) - -> Merge Append - Sort Key: hyper."time", hyper.device + -> Append -> GroupAggregate Output: hyper."time", hyper.device, avg(hyper.temp), ((sum(hyper.temp) * random()) * (hyper.device)::double precision) Group Key: hyper."time", hyper.device @@ -2038,7 +2308,7 @@ LIMIT 1; Data node: data_node_2 Chunks: _hyper_1_4_dist_chunk, _hyper_1_2_dist_chunk Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST -(24 rows) +(23 rows) -- not pushed down because of non-shippable expression on the -- underlying rel @@ -2087,14 +2357,14 @@ WHERE (hyper.temp * random() <= 20) AND time < '2018-06-01 00:00' GROUP BY 1, 2 LIMIT 1; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + QUERY PLAN +---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- Limit Output: "time", device, (avg(temp)) -> Custom Scan (AsyncAppend) Output: "time", device, (avg(temp)) -> Append - -> HashAggregate + -> GroupAggregate Output: hyper."time", hyper.device, avg(hyper.temp) Group Key: hyper."time", hyper.device -> Custom Scan (DataNodeScan) on public.hyper @@ -2102,8 +2372,8 @@ LIMIT 1; Filter: ((hyper.temp * random()) <= '20'::double precision) Data node: data_node_1 Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk - Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) - -> HashAggregate + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST + -> GroupAggregate Output: hyper_1."time", hyper_1.device, avg(hyper_1.temp) Group Key: hyper_1."time", hyper_1.device -> Custom Scan (DataNodeScan) on public.hyper hyper_1 @@ -2111,13 +2381,13 @@ LIMIT 1; Filter: ((hyper_1.temp * random()) <= '20'::double precision) Data node: data_node_2 Chunks: _hyper_1_4_dist_chunk, _hyper_1_2_dist_chunk - Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST (23 rows) -- contains whitelisted time expressions -SELECT ts_test_override_pushdown_timestamptz('2018-06-01 00:00'::timestamptz); - ts_test_override_pushdown_timestamptz ---------------------------------------- +SELECT ts_test_override_current_timestamptz('2018-06-01 00:00'::timestamptz); + ts_test_override_current_timestamptz +-------------------------------------- (1 row) @@ -2176,26 +2446,31 @@ FROM hyper WHERE time < Now( ) - INTERVAL '3 days' GROUP BY 1, 2 LIMIT 1; - QUERY PLAN ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ + QUERY PLAN +---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- Limit Output: "time", device, (avg(temp)) -> Custom Scan (AsyncAppend) Output: "time", device, (avg(temp)) - -> Append - -> Custom Scan (DataNodeScan) - Output: hyper."time", hyper.device, (avg(hyper.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_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 "time", device, avg(temp) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2, 3, 4]) AND (("time" < (('2018-06-01 00:00:00-07'::timestamptz) - '3 days'::interval))) GROUP BY 1, 2 - -> Custom Scan (DataNodeScan) - Output: hyper_1."time", hyper_1.device, (avg(hyper_1.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_2 - Chunks: _hyper_1_2_dist_chunk, _hyper_1_4_dist_chunk, _hyper_1_6_dist_chunk - Remote SQL: SELECT "time", device, avg(temp) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2, 3]) AND (("time" < (('2018-06-01 00:00:00-07'::timestamptz) - '3 days'::interval))) GROUP BY 1, 2 -(17 rows) + -> Merge Append + Sort Key: hyper."time", hyper.device + -> GroupAggregate + Output: hyper."time", hyper.device, avg(hyper.temp) + Group Key: hyper."time", hyper.device + -> Custom Scan (DataNodeScan) on public.hyper + Output: hyper."time", hyper.device, hyper.temp + Data node: data_node_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 "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2, 3, 4]) AND (("time" < (('2018-06-01 00:00:00-07'::timestamptz) - '3 days'::interval))) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST + -> GroupAggregate + Output: hyper_1."time", hyper_1.device, avg(hyper_1.temp) + Group Key: hyper_1."time", hyper_1.device + -> Custom Scan (DataNodeScan) on public.hyper hyper_1 + Output: hyper_1."time", hyper_1.device, hyper_1.temp + Data node: data_node_2 + Chunks: _hyper_1_2_dist_chunk, _hyper_1_4_dist_chunk, _hyper_1_6_dist_chunk + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2, 3]) AND (("time" < (('2018-06-01 00:00:00-07'::timestamptz) - '3 days'::interval))) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST +(22 rows) -- Verify that repeated runs of the same plan will get different timestamps PREPARE timestamp_pushdown_test AS @@ -2206,55 +2481,65 @@ GROUP BY 1, 2 LIMIT 1; EXPLAIN (VERBOSE, COSTS OFF) EXECUTE timestamp_pushdown_test; - QUERY PLAN ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ + QUERY PLAN +---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- Limit Output: "time", device, (avg(temp)) -> Custom Scan (AsyncAppend) Output: "time", device, (avg(temp)) - -> Append - -> Custom Scan (DataNodeScan) - Output: hyper."time", hyper.device, (avg(hyper.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_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 "time", device, avg(temp) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2, 3, 4]) AND (("time" < (('2018-06-01 00:00:00-07'::timestamptz) - '3 days'::interval))) GROUP BY 1, 2 - -> Custom Scan (DataNodeScan) - Output: hyper_1."time", hyper_1.device, (avg(hyper_1.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_2 - Chunks: _hyper_1_2_dist_chunk, _hyper_1_4_dist_chunk, _hyper_1_6_dist_chunk - Remote SQL: SELECT "time", device, avg(temp) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2, 3]) AND (("time" < (('2018-06-01 00:00:00-07'::timestamptz) - '3 days'::interval))) GROUP BY 1, 2 -(17 rows) + -> Merge Append + Sort Key: hyper."time", hyper.device + -> GroupAggregate + Output: hyper."time", hyper.device, avg(hyper.temp) + Group Key: hyper."time", hyper.device + -> Custom Scan (DataNodeScan) on public.hyper + Output: hyper."time", hyper.device, hyper.temp + Data node: data_node_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 "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2, 3, 4]) AND (("time" < (('2018-06-01 00:00:00-07'::timestamptz) - '3 days'::interval))) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST + -> GroupAggregate + Output: hyper_1."time", hyper_1.device, avg(hyper_1.temp) + Group Key: hyper_1."time", hyper_1.device + -> Custom Scan (DataNodeScan) on public.hyper hyper_1 + Output: hyper_1."time", hyper_1.device, hyper_1.temp + Data node: data_node_2 + Chunks: _hyper_1_2_dist_chunk, _hyper_1_4_dist_chunk, _hyper_1_6_dist_chunk + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2, 3]) AND (("time" < (('2018-06-01 00:00:00-07'::timestamptz) - '3 days'::interval))) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST +(22 rows) -SELECT ts_test_override_pushdown_timestamptz('2019-10-15 00:00'::timestamptz); - ts_test_override_pushdown_timestamptz ---------------------------------------- +SELECT ts_test_override_current_timestamptz('2019-10-15 00:00'::timestamptz); + ts_test_override_current_timestamptz +-------------------------------------- (1 row) EXPLAIN (VERBOSE, COSTS OFF) EXECUTE timestamp_pushdown_test; - QUERY PLAN ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ + QUERY PLAN +---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- Limit Output: "time", device, (avg(temp)) -> Custom Scan (AsyncAppend) Output: "time", device, (avg(temp)) - -> Append - -> Custom Scan (DataNodeScan) - Output: hyper."time", hyper.device, (avg(hyper.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_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 "time", device, avg(temp) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2, 3, 4]) AND (("time" < (('2019-10-15 00:00:00-07'::timestamptz) - '3 days'::interval))) GROUP BY 1, 2 - -> Custom Scan (DataNodeScan) - Output: hyper_1."time", hyper_1.device, (avg(hyper_1.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_2 - Chunks: _hyper_1_2_dist_chunk, _hyper_1_4_dist_chunk, _hyper_1_6_dist_chunk - Remote SQL: SELECT "time", device, avg(temp) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2, 3]) AND (("time" < (('2019-10-15 00:00:00-07'::timestamptz) - '3 days'::interval))) GROUP BY 1, 2 -(17 rows) + -> Merge Append + Sort Key: hyper."time", hyper.device + -> GroupAggregate + Output: hyper."time", hyper.device, avg(hyper.temp) + Group Key: hyper."time", hyper.device + -> Custom Scan (DataNodeScan) on public.hyper + Output: hyper."time", hyper.device, hyper.temp + Data node: data_node_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 "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2, 3, 4]) AND (("time" < (('2019-10-15 00:00:00-07'::timestamptz) - '3 days'::interval))) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST + -> GroupAggregate + Output: hyper_1."time", hyper_1.device, avg(hyper_1.temp) + Group Key: hyper_1."time", hyper_1.device + -> Custom Scan (DataNodeScan) on public.hyper hyper_1 + Output: hyper_1."time", hyper_1.device, hyper_1.temp + Data node: data_node_2 + Chunks: _hyper_1_2_dist_chunk, _hyper_1_4_dist_chunk, _hyper_1_6_dist_chunk + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2, 3]) AND (("time" < (('2019-10-15 00:00:00-07'::timestamptz) - '3 days'::interval))) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST +(22 rows) -- Test one-dimensional push down CREATE TABLE hyper1d (time timestamptz, device int, temp float); @@ -2310,27 +2595,30 @@ EXPLAIN (VERBOSE, COSTS OFF) SELECT time_bucket('1 day', time), avg(temp) FROM hyper1d GROUP BY 1; - QUERY PLAN ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- - Finalize HashAggregate + QUERY PLAN +--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + Finalize GroupAggregate Output: (time_bucket('@ 1 day'::interval, "time")), avg(temp) Group Key: (time_bucket('@ 1 day'::interval, "time")) - -> Custom Scan (AsyncAppend) + -> Sort Output: (time_bucket('@ 1 day'::interval, "time")), (PARTIAL avg(temp)) - -> Append - -> Custom Scan (DataNodeScan) - Output: (time_bucket('@ 1 day'::interval, hyper1d."time")), (PARTIAL avg(hyper1d.temp)) - Relations: Aggregate on (public.hyper1d) - Data node: data_node_1 - Chunks: _hyper_2_8_dist_chunk, _hyper_2_10_dist_chunk - Remote SQL: SELECT public.time_bucket('@ 1 day'::interval, "time"), _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(public.hyper1d.*, ARRAY[5, 6]) GROUP BY 1 - -> Custom Scan (DataNodeScan) - Output: (time_bucket('@ 1 day'::interval, hyper1d_1."time")), (PARTIAL avg(hyper1d_1.temp)) - Relations: Aggregate on (public.hyper1d) - Data node: data_node_2 - Chunks: _hyper_2_9_dist_chunk - Remote SQL: SELECT public.time_bucket('@ 1 day'::interval, "time"), _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(public.hyper1d.*, ARRAY[4]) GROUP BY 1 -(18 rows) + Sort Key: (time_bucket('@ 1 day'::interval, "time")) + -> Custom Scan (AsyncAppend) + Output: (time_bucket('@ 1 day'::interval, "time")), (PARTIAL avg(temp)) + -> Append + -> Custom Scan (DataNodeScan) + Output: (time_bucket('@ 1 day'::interval, hyper1d."time")), (PARTIAL avg(hyper1d.temp)) + Relations: Aggregate on (public.hyper1d) + Data node: data_node_1 + Chunks: _hyper_2_8_dist_chunk, _hyper_2_10_dist_chunk + Remote SQL: SELECT public.time_bucket('@ 1 day'::interval, "time"), _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(public.hyper1d.*, ARRAY[5, 6]) GROUP BY 1 + -> Custom Scan (DataNodeScan) + Output: (time_bucket('@ 1 day'::interval, hyper1d_1."time")), (PARTIAL avg(hyper1d_1.temp)) + Relations: Aggregate on (public.hyper1d) + Data node: data_node_2 + Chunks: _hyper_2_9_dist_chunk + Remote SQL: SELECT public.time_bucket('@ 1 day'::interval, "time"), _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(public.hyper1d.*, ARRAY[4]) GROUP BY 1 +(21 rows) --- Only one chunk in query => safe to fully push down although not on --- a partitioning dimension. @@ -2357,25 +2645,28 @@ SELECT device, avg(temp) FROM hyper1d WHERE time < '2018-06-01 00:00' GROUP BY 1; - QUERY PLAN ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- - Finalize HashAggregate + QUERY PLAN +--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + Finalize GroupAggregate Output: device, avg(temp) Group Key: device - -> Custom Scan (AsyncAppend) + -> Sort Output: device, (PARTIAL avg(temp)) - -> Append - -> Custom Scan (DataNodeScan) - Output: hyper1d.device, (PARTIAL avg(hyper1d.temp)) - Relations: Aggregate on (public.hyper1d) - Data node: data_node_1 - Chunks: _hyper_2_8_dist_chunk - Remote SQL: SELECT device, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(public.hyper1d.*, ARRAY[5]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 - -> Custom Scan (DataNodeScan) - Output: hyper1d_1.device, (PARTIAL avg(hyper1d_1.temp)) - Relations: Aggregate on (public.hyper1d) - Data node: data_node_2 - Chunks: _hyper_2_9_dist_chunk - Remote SQL: SELECT device, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(public.hyper1d.*, ARRAY[4]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 -(18 rows) + Sort Key: device + -> Custom Scan (AsyncAppend) + Output: device, (PARTIAL avg(temp)) + -> Append + -> Custom Scan (DataNodeScan) + Output: hyper1d.device, (PARTIAL avg(hyper1d.temp)) + Relations: Aggregate on (public.hyper1d) + Data node: data_node_1 + Chunks: _hyper_2_8_dist_chunk + Remote SQL: SELECT device, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(public.hyper1d.*, ARRAY[5]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 + -> Custom Scan (DataNodeScan) + Output: hyper1d_1.device, (PARTIAL avg(hyper1d_1.temp)) + Relations: Aggregate on (public.hyper1d) + Data node: data_node_2 + Chunks: _hyper_2_9_dist_chunk + Remote SQL: SELECT device, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(public.hyper1d.*, ARRAY[4]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 +(21 rows) diff --git a/tsl/test/expected/partitionwise_distributed-12.out b/tsl/test/expected/partitionwise_distributed-12.out index 7ef38c506..dc85b4c67 100644 --- a/tsl/test/expected/partitionwise_distributed-12.out +++ b/tsl/test/expected/partitionwise_distributed-12.out @@ -14,9 +14,279 @@ CREATE OR REPLACE FUNCTION test.remote_exec(srv_name name[], command text) RETURNS VOID AS :TSL_MODULE_PATHNAME, 'ts_remote_exec' LANGUAGE C; -CREATE OR REPLACE FUNCTION ts_test_override_pushdown_timestamptz(new_value TIMESTAMPTZ) RETURNS VOID -AS :TSL_MODULE_PATHNAME, 'ts_test_override_pushdown_timestamptz' -LANGUAGE C VOLATILE STRICT; +\ir utils/testsupport.sql +-- This file and its contents are licensed under the Apache License 2.0. +-- Please see the included NOTICE for copyright information and +-- LICENSE-APACHE for a copy of the license. +SELECT _timescaledb_internal.stop_background_workers(); + stop_background_workers +------------------------- + t +(1 row) + +CREATE SCHEMA IF NOT EXISTS test; +psql:utils/testsupport.sql:8: NOTICE: schema "test" already exists, skipping +GRANT USAGE ON SCHEMA test TO PUBLIC; +-- Utility functions to show relation information in tests. These +-- functions generate output which is the same across PostgreSQL +-- versions. Their usage is preferred over psql's '\d ', +-- since that output typically changes across PostgreSQL versions. +-- this function is duplicated in test/isolation/specs/multi_transaction_indexing.spec +-- if it changes, that copy may need to change as well +CREATE OR REPLACE FUNCTION test.show_columns(rel regclass) +RETURNS TABLE("Column" name, + "Type" text, + "NotNull" boolean) LANGUAGE SQL STABLE AS +$BODY$ + SELECT a.attname, + format_type(t.oid, t.typtypmod), + a.attnotnull + FROM pg_attribute a, pg_type t + WHERE a.attrelid = rel + AND a.atttypid = t.oid + AND a.attnum >= 0 + ORDER BY a.attnum; +$BODY$; +CREATE OR REPLACE FUNCTION test.show_columnsp(pattern text) +RETURNS TABLE("Relation" regclass, + "Kind" "char", + "Column" name, + "Column type" text, + "NotNull" boolean) LANGUAGE PLPGSQL STABLE AS +$BODY$ +DECLARE + schema_name name = split_part(pattern, '.', 1); + table_name name = split_part(pattern, '.', 2); +BEGIN + IF schema_name = '' OR table_name = '' THEN + schema_name := current_schema(); + table_name := pattern; + END IF; + + RETURN QUERY + SELECT c.oid::regclass, + c.relkind, + a.attname, + format_type(t.oid, t.typtypmod), + a.attnotnull + FROM pg_class c, pg_attribute a, pg_type t + WHERE format('%I.%I', c.relnamespace::regnamespace::name, c.relname) LIKE format('%I.%s', schema_name, table_name) + AND a.attrelid = c.oid + AND a.atttypid = t.oid + AND a.attnum >= 0 + ORDER BY c.relname, a.attnum; +END +$BODY$; +CREATE OR REPLACE FUNCTION test.show_indexes(rel regclass) +RETURNS TABLE("Index" regclass, + "Columns" name[], + "Expr" text, + "Unique" boolean, + "Primary" boolean, + "Exclusion" boolean, + "Tablespace" name) LANGUAGE SQL STABLE AS +$BODY$ + SELECT c.oid::regclass, + array(SELECT "Column" FROM test.show_columns(i.indexrelid)), + pg_get_expr(i.indexprs, c.oid, true), + i.indisunique, + i.indisprimary, + i.indisexclusion, + (SELECT t.spcname FROM pg_tablespace t WHERE t.oid = c.reltablespace) + FROM pg_class c, pg_index i + WHERE c.oid = i.indexrelid AND i.indrelid = rel + ORDER BY c.relname; +$BODY$; +-- this function is duplicated in test/isolation/specs/multi_transaction_indexing.spec +-- if it changes, that copy may need to change as well +CREATE OR REPLACE FUNCTION test.show_indexesp(pattern text) +RETURNS TABLE("Table" regclass, + "Index" regclass, + "Columns" name[], + "Expr" text, + "Unique" boolean, + "Primary" boolean, + "Exclusion" boolean, + "Tablespace" name) LANGUAGE PLPGSQL STABLE AS +$BODY$ +DECLARE + schema_name name = split_part(pattern, '.', 1); + table_name name = split_part(pattern, '.', 2); +BEGIN + IF schema_name = '' OR table_name = '' THEN + schema_name := current_schema(); + table_name := pattern; + END IF; + + RETURN QUERY + SELECT c.oid::regclass, + i.indexrelid::regclass, + array(SELECT "Column" FROM test.show_columns(i.indexrelid)), + pg_get_expr(i.indexprs, c.oid, true), + i.indisunique, + i.indisprimary, + i.indisexclusion, + (SELECT t.spcname FROM pg_class cc, pg_tablespace t WHERE cc.oid = i.indexrelid AND t.oid = cc.reltablespace) + FROM pg_class c, pg_index i + WHERE format('%I.%I', c.relnamespace::regnamespace::name, c.relname) LIKE format('%I.%s', schema_name, table_name) + AND c.oid = i.indrelid + ORDER BY c.oid, i.indexrelid; +END +$BODY$; +CREATE OR REPLACE FUNCTION test.show_constraints(rel regclass) +RETURNS TABLE("Constraint" name, + "Type" "char", + "Columns" name[], + "Index" regclass, + "Expr" text, + "Deferrable" bool, + "Deferred" bool, + "Validated" bool) LANGUAGE SQL STABLE AS +$BODY$ + SELECT c.conname, + c.contype, + array(SELECT attname FROM pg_attribute a, unnest(conkey) k WHERE a.attrelid = rel AND k = a.attnum), + c.conindid::regclass, + pg_get_expr(c.conbin, c.conrelid), + c.condeferrable, + c.condeferred, + c.convalidated + FROM pg_constraint c + WHERE c.conrelid = rel + ORDER BY c.conname; +$BODY$; +CREATE OR REPLACE FUNCTION test.show_constraintsp(pattern text) +RETURNS TABLE("Table" regclass, + "Constraint" name, + "Type" "char", + "Columns" name[], + "Index" regclass, + "Expr" text, + "Deferrable" bool, + "Deferred" bool, + "Validated" bool) LANGUAGE PLPGSQL STABLE AS +$BODY$ +DECLARE + schema_name name = split_part(pattern, '.', 1); + table_name name = split_part(pattern, '.', 2); +BEGIN + IF schema_name = '' OR table_name = '' THEN + schema_name := current_schema(); + table_name := pattern; + END IF; + + RETURN QUERY + SELECT cl.oid::regclass, + c.conname, + c.contype, + array(SELECT attname FROM pg_attribute a, unnest(conkey) k WHERE a.attrelid = cl.oid AND k = a.attnum), + c.conindid::regclass, + pg_get_expr(c.conbin, c.conrelid), + c.condeferrable, + c.condeferred, + c.convalidated + FROM pg_class cl, pg_constraint c + WHERE format('%I.%I', cl.relnamespace::regnamespace::name, cl.relname) LIKE format('%I.%s', schema_name, table_name) + AND c.conrelid = cl.oid + ORDER BY cl.relname, c.conname; +END +$BODY$; +CREATE OR REPLACE FUNCTION test.show_triggers(rel regclass, show_internal boolean = false) +RETURNS TABLE("Trigger" name, + "Type" smallint, + "Function" regproc) LANGUAGE SQL STABLE AS +$BODY$ + SELECT t.tgname, + t.tgtype, + t.tgfoid::regproc + FROM pg_trigger t + WHERE t.tgrelid = rel + AND t.tgisinternal = show_internal + ORDER BY t.tgname; +$BODY$; +CREATE OR REPLACE FUNCTION test.show_triggersp(pattern text, show_internal boolean = false) +RETURNS TABLE("Table" regclass, + "Trigger" name, + "Type" smallint, + "Function" regproc) LANGUAGE PLPGSQL STABLE AS +$BODY$ +DECLARE + schema_name name = split_part(pattern, '.', 1); + table_name name = split_part(pattern, '.', 2); +BEGIN + IF schema_name = '' OR table_name = '' THEN + schema_name := current_schema(); + table_name := pattern; + END IF; + + RETURN QUERY + SELECT t.tgrelid::regclass, + t.tgname, + t.tgtype, + t.tgfoid::regproc + FROM pg_class cl, pg_trigger t + WHERE format('%I.%I', cl.relnamespace::regnamespace::name, cl.relname) LIKE format('%I.%s', schema_name, table_name) + AND t.tgrelid = cl.oid + AND t.tgisinternal = show_internal + ORDER BY t.tgrelid, t.tgname; +END +$BODY$; +CREATE OR REPLACE FUNCTION test.show_subtables(rel regclass) +RETURNS TABLE("Child" regclass, + "Tablespace" name) LANGUAGE SQL STABLE AS +$BODY$ + SELECT objid::regclass, (SELECT t.spcname FROM pg_tablespace t WHERE t.oid = c.reltablespace) + FROM pg_depend d, pg_class c + WHERE d.refobjid = rel + AND d.deptype = 'n' + AND d.classid = 'pg_class'::regclass + AND d.objid = c.oid + ORDER BY d.refobjid, d.objid; +$BODY$; +CREATE OR REPLACE FUNCTION test.show_subtablesp(pattern text) +RETURNS TABLE("Parent" regclass, + "Child" regclass, + "Tablespace" name) LANGUAGE PLPGSQL STABLE AS +$BODY$ +DECLARE + schema_name name = split_part(pattern, '.', 1); + table_name name = split_part(pattern, '.', 2); +BEGIN + IF schema_name = '' OR table_name = '' THEN + schema_name := current_schema(); + table_name := pattern; + END IF; + + RETURN QUERY + SELECT refobjid::regclass, + objid::regclass, + (SELECT t.spcname FROM pg_class cc, pg_tablespace t WHERE cc.oid = d.objid AND t.oid = cc.reltablespace) + FROM pg_class c, pg_depend d + WHERE format('%I.%I', c.relnamespace::regnamespace::name, c.relname) LIKE format('%I.%s', schema_name, table_name) + AND d.refobjid = c.oid + AND d.deptype = 'n' + AND d.classid = 'pg_class'::regclass + ORDER BY d.refobjid, d.objid; +END +$BODY$; +CREATE OR REPLACE FUNCTION test.execute_sql(cmd TEXT) +RETURNS TEXT LANGUAGE PLPGSQL AS $BODY$ +BEGIN + EXECUTE cmd; + RETURN cmd; +END +$BODY$; +-- Used to set a deterministic memory setting during tests +CREATE OR REPLACE FUNCTION test.set_memory_cache_size(memory_amount text) +RETURNS BIGINT AS :MODULE_PATHNAME, 'ts_set_memory_cache_size' LANGUAGE C VOLATILE STRICT; +CREATE OR REPLACE FUNCTION test.empty_trigger_func() + RETURNS TRIGGER LANGUAGE PLPGSQL AS +$BODY$ +BEGIN +END +$BODY$; +CREATE OR REPLACE FUNCTION ts_test_override_current_timestamptz(new_value TIMESTAMPTZ) +RETURNS VOID AS :TSL_MODULE_PATHNAME, 'ts_test_override_current_timestamptz' LANGUAGE C VOLATILE STRICT; -- Cleanup from other potential tests that created these databases SET client_min_messages TO ERROR; DROP DATABASE IF EXISTS data_node_1; @@ -401,28 +671,30 @@ FROM hyper WHERE time < '2018-06-01 00:00' GROUP BY 1 ORDER BY 1; - QUERY PLAN ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + QUERY PLAN +-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- Finalize GroupAggregate Output: "time", avg(temp) Group Key: "time" - -> Custom Scan (AsyncAppend) + -> Sort Output: "time", (PARTIAL avg(temp)) - -> Merge Append - Sort Key: hyper."time" - -> Custom Scan (DataNodeScan) - Output: hyper."time", (PARTIAL avg(hyper.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_1 - Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk - Remote SQL: SELECT "time", _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 ORDER BY "time" ASC NULLS LAST - -> Custom Scan (DataNodeScan) - Output: hyper_1."time", (PARTIAL avg(hyper_1.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_2 - Chunks: _hyper_1_2_dist_chunk, _hyper_1_4_dist_chunk - Remote SQL: SELECT "time", _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 ORDER BY "time" ASC NULLS LAST -(19 rows) + Sort Key: "time" + -> Custom Scan (AsyncAppend) + Output: "time", (PARTIAL avg(temp)) + -> Append + -> Custom Scan (DataNodeScan) + Output: hyper."time", (PARTIAL avg(hyper.temp)) + Relations: Aggregate on (public.hyper) + Data node: data_node_1 + Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk + Remote SQL: SELECT "time", _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 + -> Custom Scan (DataNodeScan) + Output: hyper_1."time", (PARTIAL avg(hyper_1.temp)) + Relations: Aggregate on (public.hyper) + Data node: data_node_2 + Chunks: _hyper_1_2_dist_chunk, _hyper_1_4_dist_chunk + Remote SQL: SELECT "time", _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 +(21 rows) -- Show result SELECT time, avg(temp) @@ -614,25 +886,27 @@ FROM hyper WHERE time < '2018-06-01 00:00' GROUP BY 1 ORDER BY 1; - QUERY PLAN -------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- - Custom Scan (AsyncAppend) + QUERY PLAN +------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ + Sort Output: device, (avg(temp)) - -> Merge Append - Sort Key: hyper.device - -> Custom Scan (DataNodeScan) - Output: hyper.device, (avg(hyper.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_1 - Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk - Remote SQL: SELECT device, avg(temp) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 ORDER BY device ASC NULLS LAST - -> Custom Scan (DataNodeScan) - Output: hyper_1.device, (avg(hyper_1.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_2 - Chunks: _hyper_1_4_dist_chunk, _hyper_1_2_dist_chunk - Remote SQL: SELECT device, avg(temp) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 ORDER BY device ASC NULLS LAST -(16 rows) + Sort Key: device + -> Custom Scan (AsyncAppend) + Output: device, (avg(temp)) + -> Append + -> Custom Scan (DataNodeScan) + Output: hyper.device, (avg(hyper.temp)) + Relations: Aggregate on (public.hyper) + Data node: data_node_1 + Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk + Remote SQL: SELECT device, avg(temp) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 + -> Custom Scan (DataNodeScan) + Output: hyper_1.device, (avg(hyper_1.temp)) + Relations: Aggregate on (public.hyper) + Data node: data_node_2 + Chunks: _hyper_1_4_dist_chunk, _hyper_1_2_dist_chunk + Remote SQL: SELECT device, avg(temp) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 +(18 rows) -- Show result SELECT device, avg(temp) @@ -658,27 +932,30 @@ SELECT location, avg(temp) FROM hyper WHERE time < '2018-06-01 00:00' GROUP BY 1; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- - Finalize HashAggregate + QUERY PLAN +---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + Finalize GroupAggregate Output: location, avg(temp) Group Key: location - -> Custom Scan (AsyncAppend) + -> Sort Output: location, (PARTIAL avg(temp)) - -> Append - -> Custom Scan (DataNodeScan) - Output: hyper.location, (PARTIAL avg(hyper.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_1 - Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk - Remote SQL: SELECT location, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 - -> Custom Scan (DataNodeScan) - Output: hyper_1.location, (PARTIAL avg(hyper_1.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_2 - Chunks: _hyper_1_4_dist_chunk, _hyper_1_2_dist_chunk - Remote SQL: SELECT location, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 -(18 rows) + Sort Key: location + -> Custom Scan (AsyncAppend) + Output: location, (PARTIAL avg(temp)) + -> Append + -> Custom Scan (DataNodeScan) + Output: hyper.location, (PARTIAL avg(hyper.temp)) + Relations: Aggregate on (public.hyper) + Data node: data_node_1 + Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk + Remote SQL: SELECT location, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 + -> Custom Scan (DataNodeScan) + Output: hyper_1.location, (PARTIAL avg(hyper_1.temp)) + Relations: Aggregate on (public.hyper) + Data node: data_node_2 + Chunks: _hyper_1_4_dist_chunk, _hyper_1_2_dist_chunk + Remote SQL: SELECT location, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 +(21 rows) -- Expand query across repartition boundary. This makes it unsafe to -- push down the FULL agg, so should expect a PARTIAL agg on @@ -749,12 +1026,12 @@ GROUP BY 1 ORDER BY 1; QUERY PLAN ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ - Sort - Output: device, (avg(temp)) - Sort Key: device - -> Finalize HashAggregate - Output: device, avg(temp) - Group Key: device + Finalize GroupAggregate + Output: device, avg(temp) + Group Key: device + -> Sort + Output: device, (PARTIAL avg(temp)) + Sort Key: device -> Custom Scan (AsyncAppend) Output: device, (PARTIAL avg(temp)) -> Append @@ -844,12 +1121,12 @@ GROUP BY 1 ORDER BY 1; QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- - Sort - Output: device, (avg(temp)) - Sort Key: device - -> Finalize HashAggregate - Output: device, avg(temp) - Group Key: device + Finalize GroupAggregate + Output: device, avg(temp) + Group Key: device + -> Sort + Output: device, (PARTIAL avg(temp)) + Sort Key: device -> Custom Scan (AsyncAppend) Output: device, (PARTIAL avg(temp)) -> Append @@ -945,28 +1222,26 @@ FROM hyper WHERE time < '2018-06-01 00:00' GROUP BY 1, 2 ORDER BY 1, 2; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- - Sort - Output: hyper."time", hyper.device, (avg(hyper.temp)) - Sort Key: hyper."time", hyper.device - -> HashAggregate - Output: hyper."time", hyper.device, avg(hyper.temp) - Group Key: hyper."time", hyper.device - -> Custom Scan (AsyncAppend) - Output: hyper."time", hyper.device, hyper.temp - -> Append - -> Custom Scan (DataNodeScan) on public.hyper hyper_1 - Output: hyper_1."time", hyper_1.device, hyper_1.temp - Data node: data_node_1 - Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk - Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) - -> Custom Scan (DataNodeScan) on public.hyper hyper_2 - Output: hyper_2."time", hyper_2.device, hyper_2.temp - Data node: data_node_2 - Chunks: _hyper_1_2_dist_chunk, _hyper_1_4_dist_chunk - Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) -(19 rows) + QUERY PLAN +---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + GroupAggregate + Output: hyper."time", hyper.device, avg(hyper.temp) + Group Key: hyper."time", hyper.device + -> Custom Scan (AsyncAppend) + Output: hyper."time", hyper.device, hyper.temp + -> Merge Append + Sort Key: hyper_1."time", hyper_1.device + -> Custom Scan (DataNodeScan) on public.hyper hyper_1 + Output: hyper_1."time", hyper_1.device, hyper_1.temp + Data node: data_node_1 + Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST + -> Custom Scan (DataNodeScan) on public.hyper hyper_2 + Output: hyper_2."time", hyper_2.device, hyper_2.temp + Data node: data_node_2 + Chunks: _hyper_1_2_dist_chunk, _hyper_1_4_dist_chunk + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST +(17 rows) SELECT time, device, avg(temp) FROM hyper @@ -1274,12 +1549,12 @@ GROUP BY 1, 2 ORDER BY 1, 2; QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- - GroupAggregate - Output: hyper."time", hyper.device, avg(hyper.temp) - Group Key: hyper."time", hyper.device - -> Sort - Output: hyper."time", hyper.device, hyper.temp - Sort Key: hyper."time", hyper.device + Sort + Output: hyper."time", hyper.device, (avg(hyper.temp)) + Sort Key: hyper."time", hyper.device + -> HashAggregate + Output: hyper."time", hyper.device, avg(hyper.temp) + Group Key: hyper."time", hyper.device -> Custom Scan (AsyncAppend) Output: hyper."time", hyper.device, hyper.temp -> Append @@ -1345,35 +1620,29 @@ WHERE time BETWEEN '2018-04-19 00:01' AND '2018-06-01 00:00' GROUP BY 1, 2 HAVING avg(temp) > 4 ORDER BY 1, 2; - QUERY PLAN ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ - Merge Append + QUERY PLAN +----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + Sort + Output: hyper."time", hyper.device, (avg(hyper.temp)) Sort Key: hyper."time", hyper.device - -> GroupAggregate + -> HashAggregate Output: hyper."time", hyper.device, avg(hyper.temp) Group Key: hyper."time", hyper.device Filter: (avg(hyper.temp) > '4'::double precision) - -> Sort + -> Custom Scan (AsyncAppend) Output: hyper."time", hyper.device, hyper.temp - Sort Key: hyper."time", hyper.device - -> Custom Scan (DataNodeScan) on public.hyper - Output: hyper."time", hyper.device, hyper.temp - Data node: data_node_1 - Chunks: _hyper_1_3_dist_chunk - Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2]) AND (("time" >= '2018-04-19 00:01:00-07'::timestamp with time zone)) AND (("time" <= '2018-06-01 00:00:00-07'::timestamp with time zone)) - -> GroupAggregate - Output: hyper_1."time", hyper_1.device, avg(hyper_1.temp) - Group Key: hyper_1."time", hyper_1.device - Filter: (avg(hyper_1.temp) > '4'::double precision) - -> Sort - Output: hyper_1."time", hyper_1.device, hyper_1.temp - Sort Key: hyper_1."time", hyper_1.device - -> Custom Scan (DataNodeScan) on public.hyper hyper_1 - Output: hyper_1."time", hyper_1.device, hyper_1.temp - Data node: data_node_2 - Chunks: _hyper_1_4_dist_chunk - Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2]) AND (("time" >= '2018-04-19 00:01:00-07'::timestamp with time zone)) AND (("time" <= '2018-06-01 00:00:00-07'::timestamp with time zone)) -(26 rows) + -> Append + -> Custom Scan (DataNodeScan) on public.hyper hyper_1 + Output: hyper_1."time", hyper_1.device, hyper_1.temp + Data node: data_node_1 + Chunks: _hyper_1_3_dist_chunk + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2]) AND (("time" >= '2018-04-19 00:01:00-07'::timestamp with time zone)) AND (("time" <= '2018-06-01 00:00:00-07'::timestamp with time zone)) + -> Custom Scan (DataNodeScan) on public.hyper hyper_2 + Output: hyper_2."time", hyper_2.device, hyper_2.temp + Data node: data_node_2 + Chunks: _hyper_1_4_dist_chunk + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2]) AND (("time" >= '2018-04-19 00:01:00-07'::timestamp with time zone)) AND (("time" <= '2018-06-01 00:00:00-07'::timestamp with time zone)) +(20 rows) SELECT time, device, avg(temp) AS temp FROM hyper @@ -1431,28 +1700,26 @@ FROM hyper WHERE time < '2018-06-01 00:00' GROUP BY 1, 2 ORDER BY 1, 2; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- - Sort - Output: (date_trunc('month'::text, hyper."time")), hyper.device, (avg(hyper.temp)) - Sort Key: (date_trunc('month'::text, hyper."time")), hyper.device - -> HashAggregate - Output: (date_trunc('month'::text, hyper."time")), hyper.device, avg(hyper.temp) - Group Key: (date_trunc('month'::text, hyper."time")), hyper.device - -> Custom Scan (AsyncAppend) - Output: (date_trunc('month'::text, hyper."time")), hyper.device, hyper.temp - -> Append - -> Custom Scan (DataNodeScan) on public.hyper hyper_1 - Output: date_trunc('month'::text, hyper_1."time"), hyper_1.device, hyper_1.temp - Data node: data_node_1 - Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk - Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) - -> Custom Scan (DataNodeScan) on public.hyper hyper_2 - Output: date_trunc('month'::text, hyper_2."time"), hyper_2.device, hyper_2.temp - Data node: data_node_2 - Chunks: _hyper_1_4_dist_chunk, _hyper_1_2_dist_chunk - Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) -(19 rows) + QUERY PLAN +------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + GroupAggregate + Output: (date_trunc('month'::text, hyper."time")), hyper.device, avg(hyper.temp) + Group Key: (date_trunc('month'::text, hyper."time")), hyper.device + -> Custom Scan (AsyncAppend) + Output: (date_trunc('month'::text, hyper."time")), hyper.device, hyper.temp + -> Merge Append + Sort Key: (date_trunc('month'::text, hyper_1."time")), hyper_1.device + -> Custom Scan (DataNodeScan) on public.hyper hyper_1 + Output: date_trunc('month'::text, hyper_1."time"), hyper_1.device, hyper_1.temp + Data node: data_node_1 + Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) ORDER BY date_trunc('month'::text, "time") ASC NULLS LAST, device ASC NULLS LAST + -> Custom Scan (DataNodeScan) on public.hyper hyper_2 + Output: date_trunc('month'::text, hyper_2."time"), hyper_2.device, hyper_2.temp + Data node: data_node_2 + Chunks: _hyper_1_4_dist_chunk, _hyper_1_2_dist_chunk + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) ORDER BY date_trunc('month'::text, "time") ASC NULLS LAST, device ASC NULLS LAST +(17 rows) SELECT date_trunc('month', time), device, avg(temp) FROM hyper @@ -1690,28 +1957,26 @@ FROM hyper WHERE time < '2018-06-01 00:00' GROUP BY 1, 2 ORDER BY 1, 2; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- - Sort - Output: (time_bucket('@ 1 day'::interval, hyper."time")), hyper.device, (avg(hyper.temp)) - Sort Key: (time_bucket('@ 1 day'::interval, hyper."time")), hyper.device - -> HashAggregate - Output: (time_bucket('@ 1 day'::interval, hyper."time")), hyper.device, avg(hyper.temp) - Group Key: (time_bucket('@ 1 day'::interval, hyper."time")), hyper.device - -> Custom Scan (AsyncAppend) - Output: (time_bucket('@ 1 day'::interval, hyper."time")), hyper.device, hyper.temp - -> Append - -> Custom Scan (DataNodeScan) on public.hyper hyper_1 - Output: time_bucket('@ 1 day'::interval, hyper_1."time"), hyper_1.device, hyper_1.temp - Data node: data_node_1 - Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk - Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) - -> Custom Scan (DataNodeScan) on public.hyper hyper_2 - Output: time_bucket('@ 1 day'::interval, hyper_2."time"), hyper_2.device, hyper_2.temp - Data node: data_node_2 - Chunks: _hyper_1_4_dist_chunk, _hyper_1_2_dist_chunk - Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) -(19 rows) + QUERY PLAN +------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + GroupAggregate + Output: (time_bucket('@ 1 day'::interval, hyper."time")), hyper.device, avg(hyper.temp) + Group Key: (time_bucket('@ 1 day'::interval, hyper."time")), hyper.device + -> Custom Scan (AsyncAppend) + Output: (time_bucket('@ 1 day'::interval, hyper."time")), hyper.device, hyper.temp + -> Merge Append + Sort Key: (time_bucket('@ 1 day'::interval, hyper_1."time")), hyper_1.device + -> Custom Scan (DataNodeScan) on public.hyper hyper_1 + Output: time_bucket('@ 1 day'::interval, hyper_1."time"), hyper_1.device, hyper_1.temp + Data node: data_node_1 + Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) ORDER BY public.time_bucket('1 day'::interval, "time") ASC NULLS LAST, device ASC NULLS LAST + -> Custom Scan (DataNodeScan) on public.hyper hyper_2 + Output: time_bucket('@ 1 day'::interval, hyper_2."time"), hyper_2.device, hyper_2.temp + Data node: data_node_2 + Chunks: _hyper_1_4_dist_chunk, _hyper_1_2_dist_chunk + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) ORDER BY public.time_bucket('1 day'::interval, "time") ASC NULLS LAST, device ASC NULLS LAST +(17 rows) -- Show result SELECT time_bucket('1 day', time), device, avg(temp) @@ -1800,18 +2065,20 @@ WHERE time < '2018-06-01 00:00' AND device = 1 GROUP BY 1, 2 ORDER BY 1, 2; - QUERY PLAN ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ - Sort - Output: (time_bucket('@ 1 day'::interval, hyper."time")), hyper.device, (avg(hyper.temp)) - Sort Key: (time_bucket('@ 1 day'::interval, hyper."time")) - -> Custom Scan (DataNodeScan) - Output: (time_bucket('@ 1 day'::interval, hyper."time")), hyper.device, (avg(hyper.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_1 - Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk - Remote SQL: SELECT public.time_bucket('@ 1 day'::interval, "time"), device, avg(temp) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) AND ((device = 1)) GROUP BY 1, 2 -(9 rows) + QUERY PLAN +----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + GroupAggregate + Output: (time_bucket('@ 1 day'::interval, hyper."time")), hyper.device, avg(hyper.temp) + Group Key: (time_bucket('@ 1 day'::interval, hyper."time")), hyper.device + -> Sort + Output: (time_bucket('@ 1 day'::interval, hyper."time")), hyper.device, hyper.temp + Sort Key: (time_bucket('@ 1 day'::interval, hyper."time")) + -> Custom Scan (DataNodeScan) on public.hyper + Output: time_bucket('@ 1 day'::interval, hyper."time"), hyper.device, hyper.temp + Data node: data_node_1 + Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) AND ((device = 1)) +(11 rows) SELECT time_bucket('1 day', time), device, avg(temp) FROM hyper @@ -1887,8 +2154,7 @@ LIMIT 1; Output: "time", device, (avg(temp)), (sum((temp * (((random() <= '1'::double precision))::integer)::double precision))) -> Custom Scan (AsyncAppend) Output: "time", device, (avg(temp)), (sum((temp * (((random() <= '1'::double precision))::integer)::double precision))) - -> Merge Append - Sort Key: hyper."time", hyper.device + -> Append -> GroupAggregate Output: hyper."time", hyper.device, avg(hyper.temp), sum((hyper.temp * (((random() <= '1'::double precision))::integer)::double precision)) Group Key: hyper."time", hyper.device @@ -1905,7 +2171,7 @@ LIMIT 1; Data node: data_node_2 Chunks: _hyper_1_4_dist_chunk, _hyper_1_2_dist_chunk Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST -(22 rows) +(21 rows) -- Pushed down with non-pushable expression taken out EXPLAIN (VERBOSE, COSTS OFF) @@ -1953,8 +2219,7 @@ LIMIT 1; Output: "time", device, (avg(temp)), ((random() * (device)::double precision)), (custom_sum(device)) -> Custom Scan (AsyncAppend) Output: "time", device, (avg(temp)), ((random() * (device)::double precision)), (custom_sum(device)) - -> Merge Append - Sort Key: hyper."time", hyper.device + -> Append -> GroupAggregate Output: hyper."time", hyper.device, avg(hyper.temp), (random() * (hyper.device)::double precision), custom_sum(hyper.device) Group Key: hyper."time", hyper.device @@ -1971,7 +2236,7 @@ LIMIT 1; Data node: data_node_2 Chunks: _hyper_1_4_dist_chunk, _hyper_1_2_dist_chunk Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST -(22 rows) +(21 rows) EXPLAIN (VERBOSE, COSTS OFF) SELECT time, device, avg(temp), sum(temp) * random() * device as sum_temp @@ -2022,8 +2287,7 @@ LIMIT 1; Output: "time", device, (avg(temp)), (((sum(temp) * random()) * (device)::double precision)) -> Custom Scan (AsyncAppend) Output: "time", device, (avg(temp)), (((sum(temp) * random()) * (device)::double precision)) - -> Merge Append - Sort Key: hyper."time", hyper.device + -> Append -> GroupAggregate Output: hyper."time", hyper.device, avg(hyper.temp), ((sum(hyper.temp) * random()) * (hyper.device)::double precision) Group Key: hyper."time", hyper.device @@ -2042,7 +2306,7 @@ LIMIT 1; Data node: data_node_2 Chunks: _hyper_1_4_dist_chunk, _hyper_1_2_dist_chunk Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST -(24 rows) +(23 rows) -- not pushed down because of non-shippable expression on the -- underlying rel @@ -2094,32 +2358,34 @@ LIMIT 1; QUERY PLAN ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- Limit - Output: hyper."time", hyper.device, (avg(hyper.temp)) - -> GroupAggregate - Output: hyper."time", hyper.device, avg(hyper.temp) - Group Key: hyper."time", hyper.device - -> Custom Scan (AsyncAppend) - Output: hyper."time", hyper.device, hyper.temp - -> Merge Append - Sort Key: hyper_1."time", hyper_1.device - -> Custom Scan (DataNodeScan) on public.hyper hyper_1 - Output: hyper_1."time", hyper_1.device, hyper_1.temp - Filter: ((hyper_1.temp * random()) <= '20'::double precision) + Output: "time", device, (avg(temp)) + -> Custom Scan (AsyncAppend) + Output: "time", device, (avg(temp)) + -> Append + -> GroupAggregate + Output: hyper."time", hyper.device, avg(hyper.temp) + Group Key: hyper."time", hyper.device + -> Custom Scan (DataNodeScan) on public.hyper + Output: hyper."time", hyper.device, hyper.temp + Filter: ((hyper.temp * random()) <= '20'::double precision) Data node: data_node_1 Chunks: _hyper_1_1_dist_chunk, _hyper_1_3_dist_chunk Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST - -> Custom Scan (DataNodeScan) on public.hyper hyper_2 - Output: hyper_2."time", hyper_2.device, hyper_2.temp - Filter: ((hyper_2.temp * random()) <= '20'::double precision) + -> GroupAggregate + Output: hyper_1."time", hyper_1.device, avg(hyper_1.temp) + Group Key: hyper_1."time", hyper_1.device + -> Custom Scan (DataNodeScan) on public.hyper hyper_1 + Output: hyper_1."time", hyper_1.device, hyper_1.temp + Filter: ((hyper_1.temp * random()) <= '20'::double precision) Data node: data_node_2 Chunks: _hyper_1_4_dist_chunk, _hyper_1_2_dist_chunk Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[2, 1]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST -(21 rows) +(23 rows) -- contains whitelisted time expressions -SELECT ts_test_override_pushdown_timestamptz('2018-06-01 00:00'::timestamptz); - ts_test_override_pushdown_timestamptz ---------------------------------------- +SELECT ts_test_override_current_timestamptz('2018-06-01 00:00'::timestamptz); + ts_test_override_current_timestamptz +-------------------------------------- (1 row) @@ -2178,26 +2444,31 @@ FROM hyper WHERE time < Now( ) - INTERVAL '3 days' GROUP BY 1, 2 LIMIT 1; - QUERY PLAN ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ + QUERY PLAN +---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- Limit Output: "time", device, (avg(temp)) -> Custom Scan (AsyncAppend) Output: "time", device, (avg(temp)) - -> Append - -> Custom Scan (DataNodeScan) - Output: hyper."time", hyper.device, (avg(hyper.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_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 "time", device, avg(temp) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2, 3, 4]) AND (("time" < (('2018-06-01 00:00:00-07'::timestamptz) - '3 days'::interval))) GROUP BY 1, 2 - -> Custom Scan (DataNodeScan) - Output: hyper_1."time", hyper_1.device, (avg(hyper_1.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_2 - Chunks: _hyper_1_2_dist_chunk, _hyper_1_4_dist_chunk, _hyper_1_6_dist_chunk - Remote SQL: SELECT "time", device, avg(temp) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2, 3]) AND (("time" < (('2018-06-01 00:00:00-07'::timestamptz) - '3 days'::interval))) GROUP BY 1, 2 -(17 rows) + -> Merge Append + Sort Key: hyper."time", hyper.device + -> GroupAggregate + Output: hyper."time", hyper.device, avg(hyper.temp) + Group Key: hyper."time", hyper.device + -> Custom Scan (DataNodeScan) on public.hyper + Output: hyper."time", hyper.device, hyper.temp + Data node: data_node_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 "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2, 3, 4]) AND (("time" < (('2018-06-01 00:00:00-07'::timestamptz) - '3 days'::interval))) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST + -> GroupAggregate + Output: hyper_1."time", hyper_1.device, avg(hyper_1.temp) + Group Key: hyper_1."time", hyper_1.device + -> Custom Scan (DataNodeScan) on public.hyper hyper_1 + Output: hyper_1."time", hyper_1.device, hyper_1.temp + Data node: data_node_2 + Chunks: _hyper_1_2_dist_chunk, _hyper_1_4_dist_chunk, _hyper_1_6_dist_chunk + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2, 3]) AND (("time" < (('2018-06-01 00:00:00-07'::timestamptz) - '3 days'::interval))) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST +(22 rows) -- Verify that repeated runs of the same plan will get different timestamps PREPARE timestamp_pushdown_test AS @@ -2208,55 +2479,65 @@ GROUP BY 1, 2 LIMIT 1; EXPLAIN (VERBOSE, COSTS OFF) EXECUTE timestamp_pushdown_test; - QUERY PLAN ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ + QUERY PLAN +---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- Limit Output: "time", device, (avg(temp)) -> Custom Scan (AsyncAppend) Output: "time", device, (avg(temp)) - -> Append - -> Custom Scan (DataNodeScan) - Output: hyper."time", hyper.device, (avg(hyper.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_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 "time", device, avg(temp) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2, 3, 4]) AND (("time" < (('2018-06-01 00:00:00-07'::timestamptz) - '3 days'::interval))) GROUP BY 1, 2 - -> Custom Scan (DataNodeScan) - Output: hyper_1."time", hyper_1.device, (avg(hyper_1.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_2 - Chunks: _hyper_1_2_dist_chunk, _hyper_1_4_dist_chunk, _hyper_1_6_dist_chunk - Remote SQL: SELECT "time", device, avg(temp) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2, 3]) AND (("time" < (('2018-06-01 00:00:00-07'::timestamptz) - '3 days'::interval))) GROUP BY 1, 2 -(17 rows) + -> Merge Append + Sort Key: hyper."time", hyper.device + -> GroupAggregate + Output: hyper."time", hyper.device, avg(hyper.temp) + Group Key: hyper."time", hyper.device + -> Custom Scan (DataNodeScan) on public.hyper + Output: hyper."time", hyper.device, hyper.temp + Data node: data_node_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 "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2, 3, 4]) AND (("time" < (('2018-06-01 00:00:00-07'::timestamptz) - '3 days'::interval))) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST + -> GroupAggregate + Output: hyper_1."time", hyper_1.device, avg(hyper_1.temp) + Group Key: hyper_1."time", hyper_1.device + -> Custom Scan (DataNodeScan) on public.hyper hyper_1 + Output: hyper_1."time", hyper_1.device, hyper_1.temp + Data node: data_node_2 + Chunks: _hyper_1_2_dist_chunk, _hyper_1_4_dist_chunk, _hyper_1_6_dist_chunk + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2, 3]) AND (("time" < (('2018-06-01 00:00:00-07'::timestamptz) - '3 days'::interval))) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST +(22 rows) -SELECT ts_test_override_pushdown_timestamptz('2019-10-15 00:00'::timestamptz); - ts_test_override_pushdown_timestamptz ---------------------------------------- +SELECT ts_test_override_current_timestamptz('2019-10-15 00:00'::timestamptz); + ts_test_override_current_timestamptz +-------------------------------------- (1 row) EXPLAIN (VERBOSE, COSTS OFF) EXECUTE timestamp_pushdown_test; - QUERY PLAN ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ + QUERY PLAN +---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- Limit Output: "time", device, (avg(temp)) -> Custom Scan (AsyncAppend) Output: "time", device, (avg(temp)) - -> Append - -> Custom Scan (DataNodeScan) - Output: hyper."time", hyper.device, (avg(hyper.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_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 "time", device, avg(temp) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2, 3, 4]) AND (("time" < (('2019-10-15 00:00:00-07'::timestamptz) - '3 days'::interval))) GROUP BY 1, 2 - -> Custom Scan (DataNodeScan) - Output: hyper_1."time", hyper_1.device, (avg(hyper_1.temp)) - Relations: Aggregate on (public.hyper) - Data node: data_node_2 - Chunks: _hyper_1_2_dist_chunk, _hyper_1_4_dist_chunk, _hyper_1_6_dist_chunk - Remote SQL: SELECT "time", device, avg(temp) FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2, 3]) AND (("time" < (('2019-10-15 00:00:00-07'::timestamptz) - '3 days'::interval))) GROUP BY 1, 2 -(17 rows) + -> Merge Append + Sort Key: hyper."time", hyper.device + -> GroupAggregate + Output: hyper."time", hyper.device, avg(hyper.temp) + Group Key: hyper."time", hyper.device + -> Custom Scan (DataNodeScan) on public.hyper + Output: hyper."time", hyper.device, hyper.temp + Data node: data_node_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 "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2, 3, 4]) AND (("time" < (('2019-10-15 00:00:00-07'::timestamptz) - '3 days'::interval))) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST + -> GroupAggregate + Output: hyper_1."time", hyper_1.device, avg(hyper_1.temp) + Group Key: hyper_1."time", hyper_1.device + -> Custom Scan (DataNodeScan) on public.hyper hyper_1 + Output: hyper_1."time", hyper_1.device, hyper_1.temp + Data node: data_node_2 + Chunks: _hyper_1_2_dist_chunk, _hyper_1_4_dist_chunk, _hyper_1_6_dist_chunk + Remote SQL: SELECT "time", device, temp FROM public.hyper WHERE _timescaledb_internal.chunks_in(public.hyper.*, ARRAY[1, 2, 3]) AND (("time" < (('2019-10-15 00:00:00-07'::timestamptz) - '3 days'::interval))) ORDER BY "time" ASC NULLS LAST, device ASC NULLS LAST +(22 rows) -- Test one-dimensional push down CREATE TABLE hyper1d (time timestamptz, device int, temp float); @@ -2312,27 +2593,30 @@ EXPLAIN (VERBOSE, COSTS OFF) SELECT time_bucket('1 day', time), avg(temp) FROM hyper1d GROUP BY 1; - QUERY PLAN ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- - Finalize HashAggregate + QUERY PLAN +--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + Finalize GroupAggregate Output: (time_bucket('@ 1 day'::interval, "time")), avg(temp) Group Key: (time_bucket('@ 1 day'::interval, "time")) - -> Custom Scan (AsyncAppend) + -> Sort Output: (time_bucket('@ 1 day'::interval, "time")), (PARTIAL avg(temp)) - -> Append - -> Custom Scan (DataNodeScan) - Output: (time_bucket('@ 1 day'::interval, hyper1d."time")), (PARTIAL avg(hyper1d.temp)) - Relations: Aggregate on (public.hyper1d) - Data node: data_node_1 - Chunks: _hyper_2_8_dist_chunk, _hyper_2_10_dist_chunk - Remote SQL: SELECT public.time_bucket('@ 1 day'::interval, "time"), _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(public.hyper1d.*, ARRAY[5, 6]) GROUP BY 1 - -> Custom Scan (DataNodeScan) - Output: (time_bucket('@ 1 day'::interval, hyper1d_1."time")), (PARTIAL avg(hyper1d_1.temp)) - Relations: Aggregate on (public.hyper1d) - Data node: data_node_2 - Chunks: _hyper_2_9_dist_chunk - Remote SQL: SELECT public.time_bucket('@ 1 day'::interval, "time"), _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(public.hyper1d.*, ARRAY[4]) GROUP BY 1 -(18 rows) + Sort Key: (time_bucket('@ 1 day'::interval, "time")) + -> Custom Scan (AsyncAppend) + Output: (time_bucket('@ 1 day'::interval, "time")), (PARTIAL avg(temp)) + -> Append + -> Custom Scan (DataNodeScan) + Output: (time_bucket('@ 1 day'::interval, hyper1d."time")), (PARTIAL avg(hyper1d.temp)) + Relations: Aggregate on (public.hyper1d) + Data node: data_node_1 + Chunks: _hyper_2_8_dist_chunk, _hyper_2_10_dist_chunk + Remote SQL: SELECT public.time_bucket('@ 1 day'::interval, "time"), _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(public.hyper1d.*, ARRAY[5, 6]) GROUP BY 1 + -> Custom Scan (DataNodeScan) + Output: (time_bucket('@ 1 day'::interval, hyper1d_1."time")), (PARTIAL avg(hyper1d_1.temp)) + Relations: Aggregate on (public.hyper1d) + Data node: data_node_2 + Chunks: _hyper_2_9_dist_chunk + Remote SQL: SELECT public.time_bucket('@ 1 day'::interval, "time"), _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(public.hyper1d.*, ARRAY[4]) GROUP BY 1 +(21 rows) --- Only one chunk in query => safe to fully push down although not on --- a partitioning dimension. @@ -2358,25 +2642,28 @@ SELECT device, avg(temp) FROM hyper1d WHERE time < '2018-06-01 00:00' GROUP BY 1; - QUERY PLAN ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- - Finalize HashAggregate + QUERY PLAN +--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + Finalize GroupAggregate Output: device, avg(temp) Group Key: device - -> Custom Scan (AsyncAppend) + -> Sort Output: device, (PARTIAL avg(temp)) - -> Append - -> Custom Scan (DataNodeScan) - Output: hyper1d.device, (PARTIAL avg(hyper1d.temp)) - Relations: Aggregate on (public.hyper1d) - Data node: data_node_1 - Chunks: _hyper_2_8_dist_chunk - Remote SQL: SELECT device, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(public.hyper1d.*, ARRAY[5]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 - -> Custom Scan (DataNodeScan) - Output: hyper1d_1.device, (PARTIAL avg(hyper1d_1.temp)) - Relations: Aggregate on (public.hyper1d) - Data node: data_node_2 - Chunks: _hyper_2_9_dist_chunk - Remote SQL: SELECT device, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(public.hyper1d.*, ARRAY[4]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 -(18 rows) + Sort Key: device + -> Custom Scan (AsyncAppend) + Output: device, (PARTIAL avg(temp)) + -> Append + -> Custom Scan (DataNodeScan) + Output: hyper1d.device, (PARTIAL avg(hyper1d.temp)) + Relations: Aggregate on (public.hyper1d) + Data node: data_node_1 + Chunks: _hyper_2_8_dist_chunk + Remote SQL: SELECT device, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(public.hyper1d.*, ARRAY[5]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 + -> Custom Scan (DataNodeScan) + Output: hyper1d_1.device, (PARTIAL avg(hyper1d_1.temp)) + Relations: Aggregate on (public.hyper1d) + Data node: data_node_2 + Chunks: _hyper_2_9_dist_chunk + Remote SQL: SELECT device, _timescaledb_internal.partialize_agg(avg(temp)) FROM public.hyper1d WHERE _timescaledb_internal.chunks_in(public.hyper1d.*, ARRAY[4]) AND (("time" < '2018-06-01 00:00:00-07'::timestamp with time zone)) GROUP BY 1 +(21 rows) diff --git a/tsl/test/sql/debug_notice.sql b/tsl/test/sql/debug_notice.sql index b95c5453d..1d006c8a8 100644 --- a/tsl/test/sql/debug_notice.sql +++ b/tsl/test/sql/debug_notice.sql @@ -48,6 +48,7 @@ INSERT INTO hyper VALUES ('2018-05-20 15:08', 5, 1, 9.4), ('2018-05-30 13:02', 3, 2, 9.0); +-- Update table stats ANALYZE hyper; -- Optimizer debug messages shown at debug level 2 diff --git a/tsl/test/sql/hypertable_distributed.sql.in b/tsl/test/sql/hypertable_distributed.sql.in index 5b3d2f908..0312c5d28 100644 --- a/tsl/test/sql/hypertable_distributed.sql.in +++ b/tsl/test/sql/hypertable_distributed.sql.in @@ -4,12 +4,12 @@ -- Need to be super user to create extension and add data nodes \c :TEST_DBNAME :ROLE_CLUSTER_SUPERUSER; -\ir include/remote_exec.sql --- Support for execute_sql_and_filter_data_node_name_on_error() \unset ECHO \o /dev/null \ir include/filter_exec.sql +\ir include/remote_exec.sql +\ir utils/testsupport.sql \o \set ECHO all @@ -950,7 +950,56 @@ INSERT INTO dist_device VALUES ('2017-01-01 09:11', 3, 2.1), ('2017-01-01 08:01', 1, 1.2); -EXPLAIN VERBOSE +EXPLAIN (VERBOSE, COSTS OFF) SELECT * FROM dist_device; SELECT * FROM dist_device; +-- Test estimating relation size without stats +CREATE TABLE hyper_estimate(time timestamptz, device int, temp float); +SELECT * FROM create_distributed_hypertable('hyper_estimate', 'time', 'device', number_partitions => 3, replication_factor => 1, chunk_time_interval => INTERVAL '7 days'); + +-- This will enable us to more easily see estimates per chunk +SET timescaledb.enable_per_data_node_queries = false; + +-- Estimating chunk progress uses current timestamp so we override it for test purposes +SELECT ts_test_override_current_timestamptz('2019-11-11 00:00'::timestamptz); + +-- Test estimates when backfilling. 3 chunks should be historical and 3 should be considered current when estimating. +-- Note that estimate numbers are way off since we are using shared buffer size as starting point. This will not be +-- an issue in 'production' like env since chunk size should be similar to shared buffer size. +INSERT INTO hyper_estimate VALUES + ('2017-01-01 06:01', 1, 1.1), + ('2017-01-01 09:11', 1, 2.1), + ('2017-01-01 08:01', 1, 1.2), + ('2017-01-02 08:01', 1, 1.3), + ('2017-01-02 08:01', 2, 1.6), + ('2017-01-02 06:01', 2, 1.4), + ('2017-01-03 01:01', 3, 2), + ('2017-01-03 01:16', 3, 3), + ('2017-01-03 01:17', 3, 4), + ('2018-01-13 01:01', 1, 2), + ('2018-01-13 01:10', 1, 0.4), + ('2018-01-13 02:10', 2, 1.4), + ('2018-01-13 05:01', 2, 2), + ('2018-01-13 05:50', 2, 4), + ('2018-01-13 16:01', 3, 2); + +-- Since there are no stats we use shared buffers size to estimate number of rows +EXPLAIN (COSTS ON) +SELECT * +FROM hyper_estimate; + +-- This will calculate the stats +ANALYZE hyper_estimate; + +EXPLAIN (COSTS ON) +SELECT * +FROM hyper_estimate; + +-- Let's insert data into a new chunk. This will result in chunk creation. +INSERT INTO hyper_estimate VALUES ('2019-11-11 06:01', 1, 1.1); + +-- We have stats for previous chunks so we can interpolate number of rows for the new chunk +EXPLAIN (COSTS ON) +SELECT * +FROM hyper_estimate; diff --git a/tsl/test/sql/partitionwise_distributed.sql.in b/tsl/test/sql/partitionwise_distributed.sql.in index fd6b664d2..556ffed41 100644 --- a/tsl/test/sql/partitionwise_distributed.sql.in +++ b/tsl/test/sql/partitionwise_distributed.sql.in @@ -5,10 +5,7 @@ -- Need to be super user to create extension and add data nodes \c :TEST_DBNAME :ROLE_CLUSTER_SUPERUSER; \ir include/remote_exec.sql - -CREATE OR REPLACE FUNCTION ts_test_override_pushdown_timestamptz(new_value TIMESTAMPTZ) RETURNS VOID -AS :TSL_MODULE_PATHNAME, 'ts_test_override_pushdown_timestamptz' -LANGUAGE C VOLATILE STRICT; +\ir utils/testsupport.sql -- Cleanup from other potential tests that created these databases SET client_min_messages TO ERROR; @@ -658,7 +655,7 @@ GROUP BY 1, 2 LIMIT 1; -- contains whitelisted time expressions -SELECT ts_test_override_pushdown_timestamptz('2018-06-01 00:00'::timestamptz); +SELECT ts_test_override_current_timestamptz('2018-06-01 00:00'::timestamptz); EXPLAIN (VERBOSE, COSTS OFF) SELECT time, device, avg(temp) @@ -685,7 +682,7 @@ LIMIT 1; EXPLAIN (VERBOSE, COSTS OFF) EXECUTE timestamp_pushdown_test; -SELECT ts_test_override_pushdown_timestamptz('2019-10-15 00:00'::timestamptz); +SELECT ts_test_override_current_timestamptz('2019-10-15 00:00'::timestamptz); EXPLAIN (VERBOSE, COSTS OFF) EXECUTE timestamp_pushdown_test; diff --git a/tsl/test/src/remote/scan_exec_debug.c b/tsl/test/src/remote/scan_exec_debug.c index 0ac4e5ff7..58c0c21fc 100644 --- a/tsl/test/src/remote/scan_exec_debug.c +++ b/tsl/test/src/remote/scan_exec_debug.c @@ -8,13 +8,13 @@ #include "export.h" #include "fdw/scan_exec.h" -TS_FUNCTION_INFO_V1(ts_test_override_pushdown_timestamptz); +TS_FUNCTION_INFO_V1(ts_test_override_current_timestamptz); Datum -ts_test_override_pushdown_timestamptz(PG_FUNCTION_ARGS) +ts_test_override_current_timestamptz(PG_FUNCTION_ARGS) { #ifdef TS_DEBUG - fdw_scan_debug_override_pushdown_timestamp(PG_GETARG_INT64(0)); + fdw_scan_debug_override_current_timestamp(PG_GETARG_INT64(0)); PG_RETURN_VOID(); #else elog(ERROR, "unable to handle ts_test_is_frontend_session without TS_DEBUG flag set");