diff --git a/sql/cagg_utils.sql b/sql/cagg_utils.sql index 166d27ac9a6..935ac9b9d60 100644 --- a/sql/cagg_utils.sql +++ b/sql/cagg_utils.sql @@ -34,11 +34,6 @@ CREATE OR REPLACE FUNCTION _timescaledb_functions.materialization_invalidation_l mat_hypertable_id INTEGER ) RETURNS VOID AS '@MODULE_PATHNAME@', 'ts_materialization_invalidation_log_delete' LANGUAGE C STRICT VOLATILE; --- raw_hypertable_id - The hypertable ID of the original distributed hypertable in the Access Node -CREATE OR REPLACE FUNCTION _timescaledb_functions.drop_dist_ht_invalidation_trigger( - raw_hypertable_id INTEGER -) RETURNS VOID AS '@MODULE_PATHNAME@', 'ts_drop_dist_ht_invalidation_trigger' LANGUAGE C STRICT VOLATILE; - -- Processes the hypertable invalidation log in a data node for all the CAGGs that belong to the -- distributed hypertable with hypertable ID 'raw_hypertable_id' in the Access Node. The -- invalidations are cut, merged and moved to the materialization invalidation log. diff --git a/sql/compat.sql b/sql/compat.sql index a874a948da6..713c461601f 100644 --- a/sql/compat.sql +++ b/sql/compat.sql @@ -224,16 +224,6 @@ END$$ SET search_path TO pg_catalog,pg_temp; -CREATE OR REPLACE FUNCTION _timescaledb_internal.drop_dist_ht_invalidation_trigger(raw_hypertable_id integer) RETURNS void LANGUAGE PLPGSQL AS $$ -BEGIN - IF current_setting('timescaledb.enable_deprecation_warnings', true)::bool THEN - RAISE WARNING 'function _timescaledb_internal.drop_dist_ht_invalidation_trigger(integer) is deprecated and has been moved to _timescaledb_functions schema. this compatibility function will be removed in a future version.'; - END IF; - PERFORM _timescaledb_functions.drop_dist_ht_invalidation_trigger($1); -END$$ -SET search_path TO pg_catalog,pg_temp; - - -- We cannot create a wrapper function in plpgsql for the aggregate transition -- functions because plpgsql cannot deal with datatype internal but since these -- are used in an aggregation context and cannot be called directly and will diff --git a/sql/updates/latest-dev.sql b/sql/updates/latest-dev.sql index 27e92f9d1bc..f3c4190a4d2 100644 --- a/sql/updates/latest-dev.sql +++ b/sql/updates/latest-dev.sql @@ -116,3 +116,5 @@ DROP PROCEDURE IF EXISTS timescaledb_experimental.cleanup_copy_chunk_operation; DROP FUNCTION IF EXISTS _timescaledb_functions.set_chunk_default_data_node; DROP FUNCTION IF EXISTS _timescaledb_internal.set_chunk_default_data_node; +DROP FUNCTION IF EXISTS _timescaledb_functions.drop_dist_ht_invalidation_trigger; +DROP FUNCTION IF EXISTS _timescaledb_internal.drop_dist_ht_invalidation_trigger; diff --git a/sql/updates/reverse-dev.sql b/sql/updates/reverse-dev.sql index 8724e7a591a..c466d51b242 100644 --- a/sql/updates/reverse-dev.sql +++ b/sql/updates/reverse-dev.sql @@ -340,4 +340,7 @@ AS '@MODULE_PATHNAME@', 'ts_copy_chunk_cleanup_proc' LANGUAGE C; CREATE FUNCTION _timescaledb_functions.set_chunk_default_data_node(chunk REGCLASS, node_name NAME) RETURNS BOOLEAN AS '@MODULE_PATHNAME@', 'ts_chunk_set_default_data_node' LANGUAGE C VOLATILE; +CREATE FUNCTION _timescaledb_functions.drop_dist_ht_invalidation_trigger( + raw_hypertable_id INTEGER +) RETURNS VOID AS '@MODULE_PATHNAME@', 'ts_drop_dist_ht_invalidation_trigger' LANGUAGE C STRICT VOLATILE; diff --git a/src/cache.c b/src/cache.c index c82ee72c1d3..b982dd1fed6 100644 --- a/src/cache.c +++ b/src/cache.c @@ -204,27 +204,6 @@ ts_cache_fetch(Cache *cache, CacheQuery *query) return query->result; } -bool -ts_cache_remove(Cache *cache, void *key) -{ - bool found; - - if (cache->remove_entry != NULL) - { - /* In case we want to free the removing entry we must do it beforehand - * because HASH_REMOVE call returns dangling pointer, which cannot be used */ - void *entry = hash_search(cache->htab, key, HASH_FIND, &found); - if (found) - cache->remove_entry(entry); - } - - hash_search(cache->htab, key, HASH_REMOVE, &found); - if (found) - cache->stats.numelements--; - - return found; -} - static void release_all_pinned_caches() { diff --git a/src/cache.h b/src/cache.h index a723efe7117..d2e341e7605 100644 --- a/src/cache.h +++ b/src/cache.h @@ -63,7 +63,6 @@ typedef struct Cache extern TSDLLEXPORT void ts_cache_init(Cache *cache); extern TSDLLEXPORT void ts_cache_invalidate(Cache *cache); extern TSDLLEXPORT void *ts_cache_fetch(Cache *cache, CacheQuery *query); -extern TSDLLEXPORT bool ts_cache_remove(Cache *cache, void *key); extern TSDLLEXPORT MemoryContext ts_cache_memory_ctx(Cache *cache); extern TSDLLEXPORT Cache *ts_cache_pin(Cache *cache); extern TSDLLEXPORT int ts_cache_release(Cache *cache); diff --git a/src/cache_invalidate.c b/src/cache_invalidate.c index 52fe04a5838..4655d85005a 100644 --- a/src/cache_invalidate.c +++ b/src/cache_invalidate.c @@ -101,23 +101,6 @@ cache_invalidate_relcache_callback(Datum arg, Oid relid) } } -/* Registration for given cache ids happens in non-TSL code when the extension - * is created. - * - * Cache entries get invalidated when either the foreign server entry or the - * role entry in the PostgreSQL catalog changes. - * - * When the foreign server entry changes, connection parameters might have - * changed. When the role entry changes, the certificate used for client - * authentication with backend data nodes might no longer be valid. - */ -static void -cache_invalidate_syscache_callback(Datum arg, int cacheid, uint32 hashvalue) -{ - Assert(cacheid == FOREIGNSERVEROID || cacheid == AUTHOID); - ts_cm_functions->cache_syscache_invalidate(arg, cacheid, hashvalue); -} - TS_FUNCTION_INFO_V1(ts_timescaledb_invalidate_cache); /* @@ -183,14 +166,6 @@ _cache_invalidate_init(void) RegisterXactCallback(cache_invalidate_xact_end, NULL); RegisterSubXactCallback(cache_invalidate_subxact_end, NULL); CacheRegisterRelcacheCallback(cache_invalidate_relcache_callback, PointerGetDatum(NULL)); - - /* Specific syscache callbacks */ - CacheRegisterSyscacheCallback(FOREIGNSERVEROID, - cache_invalidate_syscache_callback, - PointerGetDatum(NULL)); - CacheRegisterSyscacheCallback(AUTHOID, - cache_invalidate_syscache_callback, - PointerGetDatum(NULL)); } void diff --git a/src/chunk.c b/src/chunk.c index b0b8ad89f91..027a508bf8d 100644 --- a/src/chunk.c +++ b/src/chunk.c @@ -66,11 +66,9 @@ #include "time_utils.h" #include "trigger.h" #include "ts_catalog/catalog.h" -#include "ts_catalog/chunk_data_node.h" #include "ts_catalog/compression_chunk_size.h" #include "ts_catalog/continuous_agg.h" #include "ts_catalog/continuous_aggs_watermark.h" -#include "ts_catalog/hypertable_data_node.h" #include "utils.h" TS_FUNCTION_INFO_V1(ts_chunk_show_chunks); @@ -814,88 +812,6 @@ ts_chunk_create_table(const Chunk *chunk, const Hypertable *ht, const char *tabl return objaddr.objectId; } -static List * -chunk_assign_data_nodes(const Chunk *chunk, const Hypertable *ht) -{ - List *htnodes; - List *chunk_data_nodes = NIL; - ListCell *lc; - - if (chunk->relkind != RELKIND_FOREIGN_TABLE) - return NIL; - - if (ht->data_nodes == NIL) - ereport(ERROR, - (errcode(ERRCODE_TS_INSUFFICIENT_NUM_DATA_NODES), - (errmsg("no data nodes associated with hypertable \"%s\"", - get_rel_name(ht->main_table_relid))))); - - Assert(chunk->cube != NULL); - - htnodes = ts_hypertable_assign_chunk_data_nodes(ht, chunk->cube); - Assert(htnodes != NIL); - - foreach (lc, htnodes) - { - const char *dn = lfirst(lc); - ForeignServer *foreign_server = GetForeignServerByName(dn, false); - ChunkDataNode *chunk_data_node = palloc0(sizeof(ChunkDataNode)); - - /* - * Create a stub data node (partially filled in entry). This will be - * fully filled in and persisted to metadata tables once we create the - * remote tables during insert - */ - chunk_data_node->fd.chunk_id = chunk->fd.id; - chunk_data_node->fd.node_chunk_id = -1; - namestrcpy(&chunk_data_node->fd.node_name, foreign_server->servername); - chunk_data_node->foreign_server_oid = foreign_server->serverid; - chunk_data_nodes = lappend(chunk_data_nodes, chunk_data_node); - } - - return chunk_data_nodes; -} - -List * -ts_chunk_get_data_node_name_list(const Chunk *chunk) -{ - List *datanodes = NULL; - ListCell *lc; - - foreach (lc, chunk->data_nodes) - { - ChunkDataNode *cdn = lfirst(lc); - - datanodes = lappend(datanodes, NameStr(cdn->fd.node_name)); - } - - return datanodes; -} - -bool -ts_chunk_has_data_node(const Chunk *chunk, const char *node_name) -{ - ListCell *lc; - ChunkDataNode *cdn; - bool found = false; - - if (chunk == NULL || node_name == NULL) - return false; - - /* check that the chunk is indeed present on the specified data node */ - foreach (lc, chunk->data_nodes) - { - cdn = lfirst(lc); - if (namestrcmp(&cdn->fd.node_name, node_name) == 0) - { - found = true; - break; - } - } - - return found; -} - static int32 get_next_chunk_id() { @@ -960,9 +876,6 @@ chunk_create_object(const Hypertable *ht, Hypercube *cube, const char *schema_na else namestrcpy(&chunk->fd.table_name, table_name); - if (chunk->relkind == RELKIND_FOREIGN_TABLE) - chunk->data_nodes = chunk_assign_data_nodes(chunk, ht); - return chunk; } @@ -1473,11 +1386,6 @@ ts_chunk_create_for_point(const Hypertable *ht, const Point *p, bool *found, con /* Create the chunk normally. */ if (found) *found = false; - if (hypertable_is_distributed_member(ht)) - ereport(ERROR, - (errcode(ERRCODE_TS_INTERNAL_ERROR), - errmsg("distributed hypertable member cannot create chunk on its own"), - errhint("Chunk creation should only happen through an access node."))); Chunk *chunk = chunk_create_from_point_after_lock(ht, p, schema, NULL, prefix); @@ -1710,9 +1618,6 @@ chunk_tuple_found(TupleInfo *ti, void *arg) NameStr(chunk->fd.schema_name), NameStr(chunk->fd.table_name)); - if (chunk->relkind == RELKIND_FOREIGN_TABLE && !IS_OSM_CHUNK(chunk)) - chunk->data_nodes = ts_chunk_data_node_scan_by_chunk_id(chunk->fd.id, ti->mctx); - return SCAN_DONE; } @@ -1954,13 +1859,6 @@ chunk_resurrect(const Hypertable *ht, int chunk_id) /* Create data table and related objects */ chunk->hypertable_relid = ht->main_table_relid; chunk->relkind = hypertable_chunk_relkind(ht); - if (chunk->relkind == RELKIND_FOREIGN_TABLE) - { - chunk->data_nodes = ts_chunk_data_node_scan_by_chunk_id(chunk->fd.id, ti->mctx); - /* If the Data-Node replica list information has been deleted reassign them */ - if (!chunk->data_nodes) - chunk->data_nodes = chunk_assign_data_nodes(chunk, ht); - } chunk->table_id = chunk_create_table(chunk, ht); chunk_create_table_constraints(ht, chunk); @@ -2342,25 +2240,6 @@ get_chunks_in_time_range(Hypertable *ht, int64 older_than, int64 newer_than, Mem return chunks; } -List * -ts_chunk_data_nodes_copy(const Chunk *chunk) -{ - List *lcopy = NIL; - ListCell *lc; - - foreach (lc, chunk->data_nodes) - { - ChunkDataNode *node = lfirst(lc); - ChunkDataNode *copy = palloc(sizeof(ChunkDataNode)); - - memcpy(copy, node, sizeof(ChunkDataNode)); - - lcopy = lappend(lcopy, copy); - } - - return lcopy; -} - Chunk * ts_chunk_copy(const Chunk *chunk) { @@ -2376,8 +2255,6 @@ ts_chunk_copy(const Chunk *chunk) if (NULL != chunk->cube) copy->cube = ts_hypercube_copy(chunk->cube); - copy->data_nodes = ts_chunk_data_nodes_copy(chunk); - return copy; } @@ -2641,8 +2518,6 @@ ts_chunk_free(Chunk *chunk) pfree(c); } - list_free(chunk->data_nodes); - pfree(chunk); } @@ -3069,7 +2944,6 @@ chunk_tuple_delete(TupleInfo *ti, DropBehavior behavior, bool preserve_chunk_cat ts_chunk_index_delete_by_chunk_id(form.id, true); ts_compression_chunk_size_delete(form.id); - ts_chunk_data_node_delete_by_chunk_id(form.id); /* Delete any row in bgw_policy_chunk-stats corresponding to this chunk */ ts_bgw_policy_chunk_stats_delete_by_chunk_id(form.id); @@ -3942,7 +3816,7 @@ lock_referenced_tables(Oid table_relid) List * ts_chunk_do_drop_chunks(Hypertable *ht, int64 older_than, int64 newer_than, int32 log_level, - List **affected_data_nodes, Oid time_type, Oid arg_type, bool older_newer) + Oid time_type, Oid arg_type, bool older_newer) { uint64 num_chunks = 0; @@ -4083,12 +3957,10 @@ ts_chunk_do_drop_chunks(Hypertable *ht, int64 older_than, int64 newer_than, int3 } } - List *data_nodes = NIL; List *dropped_chunk_names = NIL; for (uint64 i = 0; i < num_chunks; i++) { char *chunk_name; - ListCell *lc; ASSERT_IS_VALID_CHUNK(&chunks[i]); @@ -4111,14 +3983,6 @@ ts_chunk_do_drop_chunks(Hypertable *ht, int64 older_than, int64 newer_than, int3 ts_chunk_drop_preserve_catalog_row(chunks + i, DROP_RESTRICT, log_level); else ts_chunk_drop(chunks + i, DROP_RESTRICT, log_level); - - /* Collect a list of affected data nodes so that we know which data - * nodes we need to drop chunks on */ - foreach (lc, chunks[i].data_nodes) - { - ChunkDataNode *cdn = lfirst(lc); - data_nodes = list_append_unique_oid(data_nodes, cdn->foreign_server_oid); - } } // if we have tiered chunks cascade drop to tiering layer as well #if PG14_GE @@ -4156,9 +4020,6 @@ ts_chunk_do_drop_chunks(Hypertable *ht, int64 older_than, int64 newer_than, int3 ts_cagg_watermark_update(ht, watermark, isnull, true); } - if (affected_data_nodes) - *affected_data_nodes = data_nodes; - DEBUG_WAITPOINT("drop_chunks_end"); return dropped_chunk_names; @@ -4249,7 +4110,6 @@ ts_chunk_drop_chunks(PG_FUNCTION_ARGS) bool verbose; int elevel; - List *data_node_oids = NIL; Cache *hcache; const Dimension *time_dim; Oid time_type; @@ -4386,7 +4246,6 @@ ts_chunk_drop_chunks(PG_FUNCTION_ARGS) older_than, newer_than, elevel, - &data_node_oids, time_type, arg_type, older_newer); @@ -4511,12 +4370,6 @@ ts_chunk_is_compressed(const Chunk *chunk) return ts_flags_are_set_32(chunk->fd.status, CHUNK_STATUS_COMPRESSED); } -bool -ts_chunk_is_distributed(const Chunk *chunk) -{ - return chunk->data_nodes != NIL; -} - /* Note that only a compressed chunk can have partial flag set */ bool ts_chunk_is_partial(const Chunk *chunk) @@ -5164,7 +5017,6 @@ ts_chunk_vec_add_from_tuple(ChunkVec **chunks, TupleInfo *ti) true); chunkptr->hypertable_relid = ts_hypertable_id_to_relid(chunkptr->fd.hypertable_id, false); chunkptr->relkind = get_rel_relkind(chunkptr->table_id); - chunkptr->data_nodes = NIL; return vec; } diff --git a/src/chunk.h b/src/chunk.h index bc77f0118ca..34cca8d91b4 100644 --- a/src/chunk.h +++ b/src/chunk.h @@ -76,11 +76,6 @@ typedef struct Chunk Hypercube *cube; ChunkConstraints *constraints; - /* - * The data nodes that hold a copy of the chunk. NIL for non-distributed - * hypertables. - */ - List *data_nodes; } Chunk; /* This structure is used during the join of the chunk constraints to find @@ -216,8 +211,8 @@ extern TSDLLEXPORT void ts_chunk_drop(const Chunk *chunk, DropBehavior behavior, extern TSDLLEXPORT void ts_chunk_drop_preserve_catalog_row(const Chunk *chunk, DropBehavior behavior, int32 log_level); extern TSDLLEXPORT List *ts_chunk_do_drop_chunks(Hypertable *ht, int64 older_than, int64 newer_than, - int32 log_level, List **affected_data_nodes, - Oid time_type, Oid arg_type, bool older_newer); + int32 log_level, Oid time_type, Oid arg_type, + bool older_newer); extern TSDLLEXPORT Chunk * ts_chunk_find_or_create_without_cuts(const Hypertable *ht, Hypercube *hc, const char *schema_name, const char *table_name, Oid chunk_table_relid, bool *created); @@ -235,10 +230,7 @@ extern TSDLLEXPORT ChunkCompressionStatus ts_chunk_get_compression_status(int32 extern TSDLLEXPORT Datum ts_chunk_id_from_relid(PG_FUNCTION_ARGS); extern TSDLLEXPORT List *ts_chunk_get_chunk_ids_by_hypertable_id(int32 hypertable_id); extern TSDLLEXPORT List *ts_chunk_get_all_chunk_ids(LOCKMODE lockmode); -extern TSDLLEXPORT List *ts_chunk_get_data_node_name_list(const Chunk *chunk); -extern bool TSDLLEXPORT ts_chunk_has_data_node(const Chunk *chunk, const char *node_name); -extern List *ts_chunk_data_nodes_copy(const Chunk *chunk); extern TSDLLEXPORT Chunk *ts_chunk_create_only_table(Hypertable *ht, Hypercube *cube, const char *schema_name, const char *table_name); diff --git a/src/chunk_adaptive.c b/src/chunk_adaptive.c index 6846b1b601d..0b9c1d843f0 100644 --- a/src/chunk_adaptive.c +++ b/src/chunk_adaptive.c @@ -441,11 +441,6 @@ ts_calculate_chunk_interval(PG_FUNCTION_ARGS) (errcode(ERRCODE_INSUFFICIENT_PRIVILEGE), errmsg("permission denied for table %s", NameStr(ht->fd.table_name)))); - if (hypertable_is_distributed(ht)) - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("adaptive chunking not supported on distributed hypertables"))); - dim = ts_hyperspace_get_dimension_by_id(ht->space, dimension_id); Assert(dim != NULL); diff --git a/src/chunk_scan.c b/src/chunk_scan.c index 10c576dd464..134a858a6b8 100644 --- a/src/chunk_scan.c +++ b/src/chunk_scan.c @@ -18,7 +18,6 @@ #include "chunk_scan.h" #include "chunk.h" #include "chunk_constraint.h" -#include "ts_catalog/chunk_data_node.h" #include "utils.h" /* @@ -207,56 +206,6 @@ ts_chunk_scan_by_chunk_ids(const Hyperspace *hs, const List *chunk_ids, unsigned ts_scan_iterator_close(&slice_iterator); Assert(CurrentMemoryContext == work_mcxt); - - /* - * Fill in data nodes for remote chunks. - * - * Avoid the loop if there are no remote chunks. (Typically, either all - * chunks are remote chunks or none are.) - */ - if (remote_chunk_count > 0) - { - ScanIterator data_node_it = ts_chunk_data_nodes_scan_iterator_create(orig_mcxt); - - for (int i = 0; i < locked_chunk_count; i++) - { - Chunk *chunk = locked_chunks[i]; - - if (chunk->relkind == RELKIND_FOREIGN_TABLE) - { - /* Must start or restart the scan on the longer-lived context */ - ts_chunk_data_nodes_scan_iterator_set_chunk_id(&data_node_it, chunk->fd.id); - ts_scan_iterator_start_or_restart_scan(&data_node_it); - - while (ts_scan_iterator_next(&data_node_it) != NULL) - { - bool should_free; - TupleInfo *ti = ts_scan_iterator_tuple_info(&data_node_it); - ChunkDataNode *chunk_data_node; - Form_chunk_data_node form; - MemoryContext old_mcxt; - HeapTuple tuple; - - tuple = ts_scanner_fetch_heap_tuple(ti, false, &should_free); - form = (Form_chunk_data_node) GETSTRUCT(tuple); - old_mcxt = MemoryContextSwitchTo(ti->mctx); - chunk_data_node = palloc(sizeof(ChunkDataNode)); - memcpy(&chunk_data_node->fd, form, sizeof(FormData_chunk_data_node)); - chunk_data_node->foreign_server_oid = - get_foreign_server_oid(NameStr(form->node_name), - /* missing_ok = */ false); - chunk->data_nodes = lappend(chunk->data_nodes, chunk_data_node); - MemoryContextSwitchTo(old_mcxt); - - if (should_free) - heap_freetuple(tuple); - } - } - } - - ts_scan_iterator_close(&data_node_it); - } - MemoryContextSwitchTo(orig_mcxt); MemoryContextDelete(work_mcxt); diff --git a/src/cross_module_fn.c b/src/cross_module_fn.c index d33d74a7e61..78fac363f49 100644 --- a/src/cross_module_fn.c +++ b/src/cross_module_fn.c @@ -84,7 +84,6 @@ CROSSMODULE_WRAPPER(continuous_agg_refresh); CROSSMODULE_WRAPPER(continuous_agg_validate_query); CROSSMODULE_WRAPPER(invalidation_cagg_log_add_entry); CROSSMODULE_WRAPPER(invalidation_hyper_log_add_entry); -CROSSMODULE_WRAPPER(drop_dist_ht_invalidation_trigger); CROSSMODULE_WRAPPER(invalidation_process_hypertable_log); CROSSMODULE_WRAPPER(invalidation_process_cagg_log); CROSSMODULE_WRAPPER(cagg_try_repair); @@ -219,12 +218,6 @@ process_cagg_try_repair(PG_FUNCTION_ARGS) pg_unreachable(); } -static void -cache_syscache_invalidate_default(Datum arg, int cacheid, uint32 hashvalue) -{ - /* The default is a no-op */ -} - static DDLResult process_cagg_viewstmt_default(Node *stmt, const char *query_string, void *pstmt, WithClauseResult *with_clause_options) @@ -345,7 +338,6 @@ TSDLLEXPORT CrossModuleFunctions ts_cm_functions_default = { .continuous_agg_validate_query = error_no_default_fn_pg_community, .invalidation_cagg_log_add_entry = error_no_default_fn_pg_community, .invalidation_hyper_log_add_entry = error_no_default_fn_pg_community, - .drop_dist_ht_invalidation_trigger = error_no_default_fn_pg_community, .invalidation_process_hypertable_log = error_no_default_fn_pg_community, .invalidation_process_cagg_log = error_no_default_fn_pg_community, .cagg_try_repair = process_cagg_try_repair, @@ -374,7 +366,6 @@ TSDLLEXPORT CrossModuleFunctions ts_cm_functions_default = { .create_chunk = error_no_default_fn_pg_community, .chunk_freeze_chunk = error_no_default_fn_pg_community, .chunk_unfreeze_chunk = error_no_default_fn_pg_community, - .cache_syscache_invalidate = cache_syscache_invalidate_default, .chunk_get_relstats = error_no_default_fn_pg_community, .chunk_get_colstats = error_no_default_fn_pg_community, .chunk_create_empty_table = error_no_default_fn_pg_community, diff --git a/src/cross_module_fn.h b/src/cross_module_fn.h index 7d7eb376aa4..04f6d8f209e 100644 --- a/src/cross_module_fn.h +++ b/src/cross_module_fn.h @@ -113,10 +113,6 @@ typedef struct CrossModuleFunctions PGFunction continuous_agg_validate_query; PGFunction invalidation_cagg_log_add_entry; PGFunction invalidation_hyper_log_add_entry; - void (*remote_invalidation_log_delete)(int32 raw_hypertable_id, - ContinuousAggHypertableStatus caggstatus); - PGFunction drop_dist_ht_invalidation_trigger; - void (*remote_drop_dist_ht_invalidation_trigger)(int32 raw_hypertable_id); PGFunction invalidation_process_hypertable_log; PGFunction invalidation_process_cagg_log; PGFunction cagg_try_repair; @@ -154,7 +150,6 @@ typedef struct CrossModuleFunctions PGFunction create_chunk; PGFunction show_chunk; - void (*cache_syscache_invalidate)(Datum arg, int cacheid, uint32 hashvalue); PGFunction chunk_get_relstats; PGFunction chunk_get_colstats; PGFunction chunk_create_empty_table; diff --git a/src/dimension.c b/src/dimension.c index f2d1a1b3a21..a053998c532 100644 --- a/src/dimension.c +++ b/src/dimension.c @@ -1241,7 +1241,6 @@ ts_dimension_update(const Hypertable *ht, const NameData *dimname, DimensionType } dimension_scan_update(dim->fd.id, dimension_tuple_update, dim, RowExclusiveLock); - ts_hypertable_check_partitioning(ht, dim->fd.id); } TS_FUNCTION_INFO_V1(ts_dimension_set_num_slices); @@ -1640,10 +1639,9 @@ ts_dimension_add_internal(FunctionCallInfo fcinfo, DimensionInfo *info, bool is_ if (space_dim != NULL) { - List *data_nodes = ts_hypertable_get_available_data_nodes(info->ht, false); ts_dimension_partition_info_recreate(dimension_id, info->num_slices, - data_nodes, + NIL, info->ht->fd.replication_factor); } } @@ -1658,9 +1656,6 @@ ts_dimension_add_internal(FunctionCallInfo fcinfo, DimensionInfo *info, bool is_ info->ht = ts_hypertable_get_by_id(info->ht->fd.id); ts_indexing_verify_indexes(info->ht); - /* Check that partitioning is sane */ - ts_hypertable_check_partitioning(info->ht, dimension_id); - /* * If the hypertable has chunks, to make it compatible * we add artificial dimension slice which will cover -inf / inf diff --git a/src/guc.c b/src/guc.c index f681eb8bada..8b0f2b8605d 100644 --- a/src/guc.c +++ b/src/guc.c @@ -208,16 +208,6 @@ ts_feature_flag_check(FeatureFlagType type) */ static bool gucs_are_initialized = false; -/* Hook for plugins to allow additional SSL options */ -set_ssl_options_hook_type ts_set_ssl_options_hook = NULL; - -/* Assign the hook to the passed in function argument */ -void -ts_assign_ssl_options_hook(void *fn) -{ - ts_set_ssl_options_hook = (set_ssl_options_hook_type) fn; -} - /* * Warn about the mismatched cache sizes that can lead to cache thrashing. */ diff --git a/src/guc.h b/src/guc.h index 8dc0bcbac95..fcff5e54cb0 100644 --- a/src/guc.h +++ b/src/guc.h @@ -95,10 +95,6 @@ typedef enum DistCopyTransferFormat extern TSDLLEXPORT DistCopyTransferFormat ts_guc_dist_copy_transfer_format; -/* Hook for plugins to allow additional SSL options */ -typedef void (*set_ssl_options_hook_type)(const char *user_name); -extern TSDLLEXPORT set_ssl_options_hook_type ts_set_ssl_options_hook; - #ifdef TS_DEBUG extern bool ts_shutdown_bgw; extern char *ts_current_timestamp_mock; diff --git a/src/hypertable.c b/src/hypertable.c index 236a9372ea8..dc66fe7551c 100644 --- a/src/hypertable.c +++ b/src/hypertable.c @@ -40,7 +40,6 @@ #include "hypertable.h" #include "ts_catalog/compression_settings.h" -#include "ts_catalog/hypertable_data_node.h" #include "ts_catalog/catalog.h" #include "ts_catalog/metadata.h" #include "hypercube.h" @@ -260,7 +259,6 @@ ts_hypertable_from_tupleinfo(const TupleInfo *ti) h->chunk_cache = ts_subspace_store_init(h->space, ti->mctx, ts_guc_max_cached_chunks_per_hypertable); h->chunk_sizing_func = get_chunk_sizing_func_oid(&h->fd); - h->data_nodes = ts_hypertable_data_node_scan(h->fd.id, ti->mctx); return h; } @@ -619,7 +617,6 @@ hypertable_tuple_delete(TupleInfo *ti, void *data) ts_tablespace_delete(hypertable_id, NULL, InvalidOid); ts_chunk_delete_by_hypertable_id(hypertable_id); ts_dimension_delete_by_hypertable_id(hypertable_id, true); - ts_hypertable_data_node_delete_by_hypertable_id(hypertable_id); /* Also remove any policy argument / job that uses this hypertable */ ts_bgw_policy_delete_by_hypertable_id(hypertable_id); @@ -1506,69 +1503,6 @@ create_hypertable_datum(FunctionCallInfo fcinfo, const Hypertable *ht, bool crea return HeapTupleGetDatum(tuple); } -/* - * Check that the partitioning is reasonable and raise warnings if - * not. Typically called after applying updates to a partitioning dimension. - */ -void -ts_hypertable_check_partitioning(const Hypertable *ht, int32 id_of_updated_dimension) -{ - const Dimension *dim; - - Assert(OidIsValid(id_of_updated_dimension)); - - dim = ts_hyperspace_get_dimension_by_id(ht->space, id_of_updated_dimension); - - Assert(dim); - - if (hypertable_is_distributed(ht)) - { - const Dimension *first_closed_dim = hyperspace_get_closed_dimension(ht->space, 0); - int num_nodes = list_length(ht->data_nodes); - - /* Warn the user that there aren't enough slices to make use of all - * servers. Only do this if this is the first closed (space) dimension. */ - if (first_closed_dim != NULL && dim->fd.id == first_closed_dim->fd.id && - num_nodes > first_closed_dim->fd.num_slices) - ereport(WARNING, - (errcode(ERRCODE_WARNING), - errmsg("insufficient number of partitions for dimension \"%s\"", - NameStr(dim->fd.column_name)), - errdetail("There are not enough partitions to make" - " use of all data nodes."), - errhint("Increase the number of partitions in dimension \"%s\" to match or" - " exceed the number of attached data nodes.", - NameStr(dim->fd.column_name)))); - } -} - -extern int16 -ts_validate_replication_factor(const char *hypertable_name, int32 replication_factor, - int num_data_nodes) -{ - bool valid = replication_factor >= 1 && replication_factor <= PG_INT16_MAX; - - if (num_data_nodes < replication_factor) - ereport(ERROR, - (errcode(ERRCODE_TS_INSUFFICIENT_NUM_DATA_NODES), - errmsg("replication factor too large for hypertable \"%s\"", hypertable_name), - errdetail("The hypertable has %d data nodes attached, while " - "the replication factor is %d.", - num_data_nodes, - replication_factor), - errhint("Decrease the replication factor or attach more data " - "nodes to the hypertable."))); - - if (!valid) - ereport(ERROR, - (errcode(ERRCODE_INVALID_PARAMETER_VALUE), - errmsg("invalid replication factor"), - errhint("A hypertable's replication factor must be between 1 and %d.", - PG_INT16_MAX))); - - return (int16) (replication_factor & 0xFFFF); -} - TS_FUNCTION_INFO_V1(ts_hypertable_create); TS_FUNCTION_INFO_V1(ts_hypertable_distributed_create); TS_FUNCTION_INFO_V1(ts_hypertable_create_general); @@ -1681,8 +1615,6 @@ ts_hypertable_create_internal(FunctionCallInfo fcinfo, Oid table_relid, Assert(created); ht = ts_hypertable_cache_get_cache_and_entry(table_relid, CACHE_FLAG_NONE, &hcache); - if (NULL != closed_dim_info) - ts_hypertable_check_partitioning(ht, closed_dim_info->dimension_id); } retval = create_hypertable_datum(fcinfo, ht, created, is_generic); @@ -2141,7 +2073,7 @@ ts_hypertable_create_from_info(Oid table_relid, int32 hypertable_id, uint32 flag ts_indexing_verify_indexes(ht); /* Attach tablespace, if any */ - if (OidIsValid(tspc_oid) && !hypertable_is_distributed(ht)) + if (OidIsValid(tspc_oid)) { NameData tspc_name; @@ -2376,8 +2308,7 @@ ts_hypertable_set_compressed(Hypertable *ht, int32 compressed_hypertable_id) /* distr. hypertables do not have a internal compression table * on the access node */ - if (!hypertable_is_distributed(ht)) - ht->fd.compressed_hypertable_id = compressed_hypertable_id; + ht->fd.compressed_hypertable_id = compressed_hypertable_id; return ts_hypertable_update(ht) > 0; } @@ -2521,215 +2452,6 @@ ts_hypertable_clone_constraints_to_compressed(const Hypertable *user_ht, List *c ts_catalog_restore_user(&sec_ctx); } -#if defined(USE_ASSERT_CHECKING) -static void -assert_chunk_data_nodes_is_a_set(const List *chunk_data_nodes) -{ - Bitmapset *chunk_data_node_oids = NULL; - ListCell *lc; - - foreach (lc, chunk_data_nodes) - { - const char *nodename = lfirst(lc); - const ForeignServer *server = GetForeignServerByName(nodename, false); - chunk_data_node_oids = bms_add_member(chunk_data_node_oids, server->serverid); - } - - Assert(list_length(chunk_data_nodes) == bms_num_members(chunk_data_node_oids)); -} -#endif - -/* - * Assign data nodes to a chunk. - * - * For space-partitioned tables, the first "closed" space dimension should - * have explicit partitioning information to map the chunk to a set of data - * nodes (dimension partition metadata). Otherwise compute the set of data - * nodes dynamically. - * - * A chunk should be assigned up to replication_factor number of data - * nodes. Dynamic assignment happens similar to tablespaces, i.e., based on - * dimension type. - * - * Returns a list of data node name strings (char *). - */ -List * -ts_hypertable_assign_chunk_data_nodes(const Hypertable *ht, const Hypercube *cube) -{ - List *chunk_data_nodes = NIL; - const Dimension *closed_dim = hyperspace_get_closed_dimension(ht->space, 0); - - if (NULL != closed_dim && NULL != closed_dim->dimension_partitions) - { - const DimensionSlice *slice = - ts_hypercube_get_slice_by_dimension_id(cube, closed_dim->fd.id); - const DimensionPartition *dp = - ts_dimension_partition_find(closed_dim->dimension_partitions, slice->fd.range_start); - ListCell *lc; - - /* Filter out data nodes that aren't available */ - foreach (lc, dp->data_nodes) - { - char *node_name = lfirst(lc); - - if (ts_data_node_is_available(node_name)) - chunk_data_nodes = lappend(chunk_data_nodes, node_name); - } - } - else - { - List *available_nodes = ts_hypertable_get_available_data_nodes(ht, false); - int num_assigned = MIN(ht->fd.replication_factor, list_length(available_nodes)); - int n, i; - - n = hypertable_get_chunk_round_robin_index(ht, cube); - - for (i = 0; i < num_assigned; i++) - { - int j = (n + i) % list_length(available_nodes); - HypertableDataNode *hdn = list_nth(available_nodes, j); - chunk_data_nodes = lappend(chunk_data_nodes, NameStr(hdn->fd.node_name)); - } - } - - if (chunk_data_nodes == NIL) - ereport(ERROR, - (errcode(ERRCODE_TS_INSUFFICIENT_NUM_DATA_NODES), - (errmsg("insufficient number of data nodes"), - errhint("Increase the number of available data nodes on hypertable \"%s\".", - get_rel_name(ht->main_table_relid))))); - - if (list_length(chunk_data_nodes) < ht->fd.replication_factor) - ereport(WARNING, - (errcode(ERRCODE_TS_INSUFFICIENT_NUM_DATA_NODES), - errmsg("insufficient number of data nodes"), - errdetail("There are not enough data nodes to replicate chunks according to the" - " configured replication factor."), - errhint("Attach %d or more data nodes to hypertable \"%s\".", - ht->fd.replication_factor - list_length(chunk_data_nodes), - NameStr(ht->fd.table_name)))); - -#if defined(USE_ASSERT_CHECKING) - assert_chunk_data_nodes_is_a_set(chunk_data_nodes); -#endif - - return chunk_data_nodes; -} - -typedef bool (*hypertable_data_node_filter)(const HypertableDataNode *hdn); - -static bool -filter_non_blocked_data_nodes(const HypertableDataNode *node) -{ - if (!ts_data_node_is_available(NameStr(node->fd.node_name))) - return false; - - return !node->fd.block_chunks; -} - -typedef void *(*get_value)(const HypertableDataNode *hdn); - -static void * -get_hypertable_data_node_name(const HypertableDataNode *node) -{ - return pstrdup(NameStr(node->fd.node_name)); -} - -static void * -get_hypertable_data_node(const HypertableDataNode *node) -{ - HypertableDataNode *copy = palloc(sizeof(HypertableDataNode)); - - memcpy(copy, node, sizeof(HypertableDataNode)); - return copy; -} - -static List * -get_hypertable_data_node_values(const Hypertable *ht, hypertable_data_node_filter filter, - get_value valuefunc) -{ - List *list = NULL; - ListCell *cell; - - foreach (cell, ht->data_nodes) - { - HypertableDataNode *node = lfirst(cell); - if (filter == NULL || filter(node)) - list = lappend(list, valuefunc(node)); - } - - return list; -} - -List * -ts_hypertable_get_data_node_name_list(const Hypertable *ht) -{ - return get_hypertable_data_node_values(ht, NULL, get_hypertable_data_node_name); -} - -static List * -get_available_data_nodes(const Hypertable *ht, get_value valuefunc, bool error_if_missing) -{ - List *available_nodes = - get_hypertable_data_node_values(ht, filter_non_blocked_data_nodes, valuefunc); - if (available_nodes == NIL && error_if_missing) - ereport(ERROR, - (errcode(ERRCODE_TS_INSUFFICIENT_NUM_DATA_NODES), - (errmsg("insufficient number of data nodes"), - errhint("Increase the number of available data nodes on hypertable \"%s\".", - get_rel_name(ht->main_table_relid))))); - return available_nodes; -} - -List * -ts_hypertable_get_available_data_nodes(const Hypertable *ht, bool error_if_missing) -{ - return get_available_data_nodes(ht, get_hypertable_data_node, error_if_missing); -} - -List * -ts_hypertable_get_available_data_node_names(const Hypertable *ht, bool error_if_missing) -{ - return get_available_data_nodes(ht, get_hypertable_data_node_name, error_if_missing); -} - -static List * -get_hypertable_data_node_ids(const Hypertable *ht, hypertable_data_node_filter filter) -{ - List *nodeids = NIL; - ListCell *lc; - - foreach (lc, ht->data_nodes) - { - HypertableDataNode *node = lfirst(lc); - if (filter == NULL || filter(node)) - nodeids = lappend_oid(nodeids, node->foreign_server_oid); - } - - return nodeids; -} - -List * -ts_hypertable_get_data_node_serverids_list(const Hypertable *ht) -{ - return get_hypertable_data_node_ids(ht, NULL); -} - -List * -ts_hypertable_get_available_data_node_server_oids(const Hypertable *ht) -{ - return get_hypertable_data_node_ids(ht, filter_non_blocked_data_nodes); -} - -HypertableType -ts_hypertable_get_type(const Hypertable *ht) -{ - Assert(ht->fd.replication_factor >= -1); - if (ht->fd.replication_factor < 1) - return (HypertableType) ht->fd.replication_factor; - return HYPERTABLE_DISTRIBUTED; -} - /* * Get the max value of an open dimension. */ @@ -2813,9 +2535,6 @@ ts_hypertable_update_dimension_partitions(const Hypertable *ht) { List *data_node_names = NIL; - if (hypertable_is_distributed(ht)) - data_node_names = ts_hypertable_get_available_data_node_names(ht, false); - ts_dimension_partition_info_recreate(closed_dim->fd.id, closed_dim->fd.num_slices, data_node_names, diff --git a/src/hypertable.h b/src/hypertable.h index 95481e227ec..67c4d58efbc 100644 --- a/src/hypertable.h +++ b/src/hypertable.h @@ -57,7 +57,6 @@ typedef struct Hypertable * Allows restricting the data nodes to use for the hypertable. Default is to * use all available data nodes. */ - List *data_nodes; } Hypertable; /* create_hypertable record attribute numbers */ @@ -135,8 +134,6 @@ extern TSDLLEXPORT ObjectAddress ts_hypertable_create_trigger(const Hypertable * extern TSDLLEXPORT void ts_hypertable_drop_trigger(Oid relid, const char *trigger_name); extern TSDLLEXPORT void ts_hypertable_drop(Hypertable *hypertable, DropBehavior behavior); -extern TSDLLEXPORT void ts_hypertable_check_partitioning(const Hypertable *ht, - int32 id_of_updated_dimension); extern int ts_hypertable_reset_associated_schema_name(const char *associated_schema); extern TSDLLEXPORT Oid ts_hypertable_id_to_relid(int32 hypertable_id, bool return_invalid); extern TSDLLEXPORT int32 ts_hypertable_relid_to_id(Oid relid); @@ -161,16 +158,6 @@ extern TSDLLEXPORT bool ts_hypertable_set_compress_interval(Hypertable *ht, int64 compress_interval); extern TSDLLEXPORT void ts_hypertable_clone_constraints_to_compressed(const Hypertable *ht, List *constraint_list); -extern TSDLLEXPORT List *ts_hypertable_assign_chunk_data_nodes(const Hypertable *ht, - const Hypercube *cube); -extern TSDLLEXPORT List *ts_hypertable_get_data_node_name_list(const Hypertable *ht); -extern TSDLLEXPORT List *ts_hypertable_get_data_node_serverids_list(const Hypertable *ht); -extern TSDLLEXPORT List *ts_hypertable_get_available_data_nodes(const Hypertable *ht, - bool error_if_missing); -extern TSDLLEXPORT List *ts_hypertable_get_available_data_node_names(const Hypertable *ht, - bool error_if_missing); -extern TSDLLEXPORT List *ts_hypertable_get_available_data_node_server_oids(const Hypertable *ht); -extern TSDLLEXPORT HypertableType ts_hypertable_get_type(const Hypertable *ht); extern TSDLLEXPORT int64 ts_hypertable_get_open_dim_max_value(const Hypertable *ht, int dimension_index, bool *isnull); @@ -179,9 +166,6 @@ extern TSDLLEXPORT void ts_hypertable_formdata_fill(FormData_hypertable *fd, con extern TSDLLEXPORT void ts_hypertable_scan_by_name(ScanIterator *iterator, const char *schema, const char *name); extern TSDLLEXPORT bool ts_hypertable_update_dimension_partitions(const Hypertable *ht); -extern TSDLLEXPORT int16 ts_validate_replication_factor(const char *hypertable_name, - int32 replication_factor, - int num_data_nodes); #define hypertable_scan(schema, table, tuple_found, data, lockmode) \ ts_hypertable_scan_with_memory_context(schema, \ diff --git a/src/indexing.c b/src/indexing.c index d151f1d08c1..81bd7332112 100644 --- a/src/indexing.c +++ b/src/indexing.c @@ -289,7 +289,7 @@ ts_indexing_relation_has_primary_or_unique_index(Relation htrel) */ extern ObjectAddress ts_indexing_root_table_create_index(IndexStmt *stmt, const char *queryString, - bool is_multitransaction, bool is_distributed) + bool is_multitransaction) { Oid relid; LOCKMODE lockmode; @@ -324,7 +324,7 @@ ts_indexing_root_table_create_index(IndexStmt *stmt, const char *queryString, * table, i.e., we do not recurse to chunks. Therefore, there is no need to * take locks on the chunks here. */ - if (!is_multitransaction && !is_distributed) + if (!is_multitransaction) { ListCell *lc; List *inheritors = NIL; diff --git a/src/indexing.h b/src/indexing.h index d2fc41170cd..ab009488e28 100644 --- a/src/indexing.h +++ b/src/indexing.h @@ -17,8 +17,7 @@ extern void ts_indexing_verify_index(const Hyperspace *hs, const IndexStmt *stmt extern void ts_indexing_verify_indexes(const Hypertable *ht); extern void ts_indexing_create_default_indexes(const Hypertable *ht); extern ObjectAddress ts_indexing_root_table_create_index(IndexStmt *stmt, const char *queryString, - bool is_multitransaction, - bool is_distributed); + bool is_multitransaction); extern TSDLLEXPORT Oid ts_indexing_find_clustered_index(Oid table_relid); extern void ts_indexing_mark_as_valid(Oid index_id); diff --git a/src/nodes/chunk_dispatch/chunk_dispatch.c b/src/nodes/chunk_dispatch/chunk_dispatch.c index 915774585f9..cd4e0914e0d 100644 --- a/src/nodes/chunk_dispatch/chunk_dispatch.c +++ b/src/nodes/chunk_dispatch/chunk_dispatch.c @@ -22,7 +22,6 @@ #include "dimension.h" #include "guc.h" #include "nodes/hypertable_modify.h" -#include "ts_catalog/chunk_data_node.h" #include "hypercube.h" static Node *chunk_dispatch_state_create(CustomScan *cscan); @@ -164,7 +163,7 @@ ts_chunk_dispatch_get_chunk_insert_state(ChunkDispatch *dispatch, Point *point, if (found) { - if (cis->chunk_compressed && cis->chunk_data_nodes == NIL) + if (cis->chunk_compressed) { /* * If this is an INSERT into a compressed chunk with UNIQUE or diff --git a/src/nodes/chunk_dispatch/chunk_insert_state.c b/src/nodes/chunk_dispatch/chunk_insert_state.c index a65096b0a50..4418222e3c7 100644 --- a/src/nodes/chunk_dispatch/chunk_insert_state.c +++ b/src/nodes/chunk_dispatch/chunk_insert_state.c @@ -28,7 +28,6 @@ #include "errors.h" #include "chunk_dispatch.h" #include "chunk_insert_state.h" -#include "ts_catalog/chunk_data_node.h" #include "ts_catalog/continuous_agg.h" #include "chunk_index.h" #include "indexing.h" @@ -509,12 +508,7 @@ set_arbiter_indexes(ChunkInsertState *state, ChunkDispatch *dispatch) errmsg("could not find arbiter index for hypertable index \"%s\" on chunk " "\"%s\"", get_rel_name(hypertable_index), - get_rel_name(RelationGetRelid(state->rel))), - hypertable_is_distributed(dispatch->hypertable) ? - errhint( - "Omit the index inference specification for the distributed hypertable" - " in the ON CONFLICT clause.") : - 0)); + get_rel_name(RelationGetRelid(state->rel))))); } state->arbiter_indexes = lappend_oid(state->arbiter_indexes, cim.indexoid); @@ -661,7 +655,6 @@ ts_chunk_insert_state_create(const Chunk *chunk, ChunkDispatch *dispatch) #else state->user_id = ExecGetResultRelCheckAsUser(relinfo, state->estate); #endif - state->chunk_data_nodes = ts_chunk_data_nodes_copy(chunk); } if (dispatch->hypertable_result_rel_info->ri_usesFdwDirectModify) diff --git a/src/nodes/chunk_dispatch/chunk_insert_state.h b/src/nodes/chunk_dispatch/chunk_insert_state.h index 7a6832bcf24..8b969b1c637 100644 --- a/src/nodes/chunk_dispatch/chunk_insert_state.h +++ b/src/nodes/chunk_dispatch/chunk_insert_state.h @@ -47,7 +47,6 @@ typedef struct ChunkInsertState TupleConversionMap *hyper_to_chunk_map; MemoryContext mctx; EState *estate; - List *chunk_data_nodes; /* List of data nodes for the chunk (ChunkDataNode objects) */ int32 chunk_id; Oid user_id; diff --git a/src/nodes/hypertable_modify.c b/src/nodes/hypertable_modify.c index 356d5fb66b7..654d2de28fa 100644 --- a/src/nodes/hypertable_modify.c +++ b/src/nodes/hypertable_modify.c @@ -28,7 +28,6 @@ #include "hypertable_cache.h" #include "hypertable_modify.h" #include "nodes/chunk_append/chunk_append.h" -#include "ts_catalog/hypertable_data_node.h" #if PG14_GE static void fireASTriggers(ModifyTableState *node); @@ -199,12 +198,7 @@ static void hypertable_modify_explain(CustomScanState *node, List *ancestors, ExplainState *es) { HypertableModifyState *state = (HypertableModifyState *) node; - List *fdw_private = linitial_node(List, state->mt->fdwPrivLists); ModifyTableState *mtstate = linitial_node(ModifyTableState, node->custom_ps); - Index rti = state->mt->nominalRelation; - RangeTblEntry *rte = rt_fetch(rti, es->rtable); - const char *relname = get_rel_name(rte->relid); - const char *namespace = get_namespace_name(get_rel_namespace(rte->relid)); #if PG14_GE /* @@ -267,42 +261,6 @@ hypertable_modify_explain(CustomScanState *node, List *ancestors, ExplainState * ExplainPropertyInteger("Batches decompressed", NULL, state->batches_decompressed, es); if (state->tuples_decompressed > 0) ExplainPropertyInteger("Tuples decompressed", NULL, state->tuples_decompressed, es); - - if (NULL != state->fdwroutine) - { - appendStringInfo(es->str, "Insert on distributed hypertable"); - - if (es->verbose) - { - List *node_names = NIL; - ListCell *lc; - - appendStringInfo(es->str, - " %s.%s\n", - quote_identifier(namespace), - quote_identifier(relname)); - - foreach (lc, state->serveroids) - { - ForeignServer *server = GetForeignServer(lfirst_oid(lc)); - - node_names = lappend(node_names, server->servername); - } - - ExplainPropertyList("Data nodes", node_names, es); - } - else - appendStringInfo(es->str, " %s\n", quote_identifier(relname)); - - /* Let the foreign data wrapper add its part of the explain, but only - * if this was using the non-direct API. */ - if (NIL != fdw_private && state->fdwroutine->ExplainForeignModify) - state->fdwroutine->ExplainForeignModify(mtstate, - mtstate->resultRelInfo, - fdw_private, - 0, - es); - } } static CustomExecMethods hypertable_modify_state_methods = { @@ -319,31 +277,12 @@ hypertable_modify_state_create(CustomScan *cscan) { HypertableModifyState *state; ModifyTable *mt = castNode(ModifyTable, linitial(cscan->custom_plans)); - Oid serverid; state = (HypertableModifyState *) newNode(sizeof(HypertableModifyState), T_CustomScanState); state->cscan_state.methods = &hypertable_modify_state_methods; state->mt = mt; state->mt->arbiterIndexes = linitial(cscan->custom_private); - /* - * Get the list of data nodes to insert on. - */ - state->serveroids = lsecond(cscan->custom_private); - - /* - * Get the FDW routine for the first data node. It should be the same for - * all of them - */ - if (NIL != state->serveroids) - { - serverid = linitial_oid(state->serveroids); - state->fdwroutine = GetFdwRoutineByServerId(serverid); - Assert(state->fdwroutine != NULL); - } - else - state->fdwroutine = NULL; - return (Node *) state; } @@ -681,7 +620,7 @@ ts_hypertable_modify_path_create(PlannerInfo *root, ModifyTablePath *mtpath, Hyp hmpath->cpath.custom_paths = list_make1(mtpath); hmpath->cpath.methods = &hypertable_modify_path_methods; hmpath->distributed_insert_plans = distributed_insert_plans; - hmpath->serveroids = ts_hypertable_get_available_data_node_server_oids(ht); + hmpath->serveroids = NIL; path = &hmpath->cpath.path; #if PG14_LT mtpath->subpaths = list_make1(subpath); diff --git a/src/nodes/hypertable_modify.h b/src/nodes/hypertable_modify.h index b8d06ec4957..2674c182e49 100644 --- a/src/nodes/hypertable_modify.h +++ b/src/nodes/hypertable_modify.h @@ -25,10 +25,8 @@ typedef struct HypertableModifyState { CustomScanState cscan_state; ModifyTable *mt; - List *serveroids; bool comp_chunks_processed; Snapshot snapshot; - FdwRoutine *fdwroutine; int64 tuples_decompressed; int64 batches_decompressed; } HypertableModifyState; diff --git a/src/planner/agg_bookend.c b/src/planner/agg_bookend.c index 88f178b37ae..ce43d3757a4 100644 --- a/src/planner/agg_bookend.c +++ b/src/planner/agg_bookend.c @@ -672,7 +672,7 @@ build_first_last_path(PlannerInfo *root, FirstLastAggInfo *fl_info, Oid eqop, Oi { RangeTblEntry *rte = lfirst_node(RangeTblEntry, lc); - if (ts_rte_is_hypertable(rte, NULL)) + if (ts_rte_is_hypertable(rte)) { ListCell *prev = NULL; ListCell *next = list_head(subroot->append_rel_list); @@ -715,7 +715,7 @@ build_first_last_path(PlannerInfo *root, FirstLastAggInfo *fl_info, Oid eqop, Oi { RangeTblEntry *rte = lfirst_node(RangeTblEntry, lc); - if (ts_rte_is_hypertable(rte, NULL)) + if (ts_rte_is_hypertable(rte)) rte->inh = true; } diff --git a/src/planner/expand_hypertable.c b/src/planner/expand_hypertable.c index dcfc698870b..b7ef2e1e87e 100644 --- a/src/planner/expand_hypertable.c +++ b/src/planner/expand_hypertable.c @@ -1157,106 +1157,6 @@ get_chunks(CollectQualCtx *ctx, PlannerInfo *root, RelOptInfo *rel, Hypertable * return find_children_chunks(hri, ht, num_chunks); } -/* - * Create partition expressions for a hypertable. - * - * Build an array of partition expressions where each element represents valid - * expressions on a particular partitioning key. - * - * The partition expressions are used by, e.g., group_by_has_partkey() to check - * whether a GROUP BY clause covers all partitioning dimensions. - * - * For dimensions with a partitioning function, we can support either - * expressions on the plain key (column) or the partitioning function applied - * to the key. For instance, the queries - * - * SELECT time, device, avg(temp) - * FROM hypertable - * GROUP BY 1, 2; - * - * and - * - * SELECT time_func(time), device, avg(temp) - * FROM hypertable - * GROUP BY 1, 2; - * - * are both amenable to aggregate push down if "time" is supported by the - * partitioning function "time_func" and "device" is also a partitioning - * dimension. - */ -static List ** -get_hypertable_partexprs(Hypertable *ht, Query *parse, Index varno) -{ - int i; - List **partexprs; - - Assert(NULL != ht->space); - - partexprs = palloc0(sizeof(List *) * ht->space->num_dimensions); - - for (i = 0; i < ht->space->num_dimensions; i++) - { - Dimension *dim = &ht->space->dimensions[i]; - - partexprs[i] = ts_dimension_get_partexprs(dim, varno); - } - - return partexprs; -} - -#define PARTITION_STRATEGY_MULTIDIM 'm' - -/* - * Partition info for hypertables. - * - * Build a "fake" partition scheme for a hypertable that makes the planner - * believe this is a PostgreSQL partitioned table for planning purposes. In - * particular, this will make the planner consider partitionwise aggregations - * when applicable. - * - * Partitionwise aggregation can either be FULL or PARTIAL. The former means - * that the aggregation can be performed independently on each partition - * (chunk) without a finalize step which is needed in PARTIAL. FULL requires - * that the GROUP BY clause contains all hypertable partitioning - * dimensions. This requirement is enforced by creating a partitioning scheme - * that covers multiple attributes, i.e., one per dimension. This works well - * since the "shallow" (one-level hierarchy) of a multi-dimensional hypertable - * is similar to a one-level partitioned PostgreSQL table where the - * partitioning key covers multiple attributes. - * - * Note that we use a partition scheme with a strategy that does not exist in - * PostgreSQL. This makes PostgreSQL raise errors when this partition scheme is - * used in places that require a valid partition scheme with a supported - * strategy. - */ -static void -build_hypertable_partition_info(Hypertable *ht, PlannerInfo *root, RelOptInfo *hyper_rel, - int nparts) -{ - PartitionScheme part_scheme = palloc0(sizeof(PartitionSchemeData)); - PartitionBoundInfo boundinfo = palloc0(sizeof(PartitionBoundInfoData)); - - /* We only set the info needed for planning */ - part_scheme->partnatts = ht->space->num_dimensions; - part_scheme->strategy = PARTITION_STRATEGY_MULTIDIM; - hyper_rel->nparts = nparts; - part_scheme->partopfamily = palloc0(part_scheme->partnatts * sizeof(Oid)); - part_scheme->partopcintype = palloc0(part_scheme->partnatts * sizeof(Oid)); - part_scheme->partcollation = palloc0(part_scheme->partnatts * sizeof(Oid)); - hyper_rel->part_scheme = part_scheme; - hyper_rel->partexprs = get_hypertable_partexprs(ht, root->parse, hyper_rel->relid); - hyper_rel->nullable_partexprs = (List **) palloc0(sizeof(List *) * part_scheme->partnatts); - - /* PartitionBoundInfo is used for ordered append. We use a strategy that - * will avoid triggering an ordered append. */ - boundinfo->strategy = PARTITION_STRATEGY_MULTIDIM; - boundinfo->default_index = -1; - boundinfo->null_index = -1; - - hyper_rel->boundinfo = boundinfo; - hyper_rel->part_rels = palloc0(sizeof(*hyper_rel->part_rels) * nparts); -} - static bool timebucket_annotate_walker(Node *node, CollectQualCtx *ctx) { @@ -1308,7 +1208,6 @@ ts_plan_expand_timebucket_annotate(PlannerInfo *root, RelOptInfo *rel) void ts_plan_expand_hypertable_chunks(Hypertable *ht, PlannerInfo *root, RelOptInfo *rel) { - TimescaleDBPrivate *priv = rel->fdw_private; RangeTblEntry *rte = rt_fetch(rel->relid, root->parse->rtable); Oid parent_oid = rte->relid; List *inh_oids = NIL; @@ -1368,7 +1267,7 @@ ts_plan_expand_hypertable_chunks(Hypertable *ht, PlannerInfo *root, RelOptInfo * } /* nothing to do here if we have no chunks and no data nodes */ - if (list_length(inh_oids) + list_length(ht->data_nodes) == 0) + if (list_length(inh_oids) == 0) return; oldrelation = table_open(parent_oid, NoLock); @@ -1378,15 +1277,7 @@ ts_plan_expand_hypertable_chunks(Hypertable *ht, PlannerInfo *root, RelOptInfo * * children to them. We include potential data node rels we might need to * create in case of a distributed hypertable. */ - expand_planner_arrays(root, list_length(inh_oids) + list_length(ht->data_nodes)); - - /* Adding partition info will make PostgreSQL consider the inheritance - * children as part of a partitioned relation. This will enable - * partitionwise aggregation for distributed queries. */ - if (hypertable_is_distributed(ht)) - { - build_hypertable_partition_info(ht, root, rel, list_length(inh_oids)); - } + expand_planner_arrays(root, list_length(inh_oids)); foreach (l, inh_oids) { @@ -1460,44 +1351,6 @@ ts_plan_expand_hypertable_chunks(Hypertable *ht, PlannerInfo *root, RelOptInfo * table_close(oldrelation, NoLock); - priv->serverids = ts_hypertable_get_data_node_serverids_list(ht); - - /* For distributed hypertables, we'd like to turn per-chunk plans into - * per-data_node plans. We proactively add RTEs for the per-data_node rels here - * because the PostgreSQL planning code that we call to replan the - * per-data_node queries assumes there are RTEs for each rel that is considered - * a "partition." - * - * Note that each per-data_node RTE reuses the relid (OID) of the parent - * hypertable relation. This makes sense since each data node's - * hypertable is an identical (albeit partial) version of the access node's - * hypertable. The upside of this is that the planner can plan remote - * queries to take into account the indexes on the hypertable to produce - * more efficient remote queries. In contrast, chunks are foreign tables so - * they do not have indexes. - */ - foreach (l, priv->serverids) - { - RangeTblEntry *data_node_rte = copyObject(rte); - - data_node_rte->inh = false; - data_node_rte->ctename = NULL; - data_node_rte->securityQuals = NIL; -#if PG16_LT - data_node_rte->requiredPerms = 0; -#else - /* Since PG16, the permission info is maintained separetely. Unlink - * the old perminfo from the RTE to disable permission checking. - */ - data_node_rte->perminfoindex = 0; -#endif - parse->rtable = lappend(parse->rtable, data_node_rte); - rti = list_length(parse->rtable); - root->simple_rte_array[rti] = data_node_rte; - root->simple_rel_array[rti] = NULL; - priv->server_relids = bms_add_member(priv->server_relids, rti); - } - ts_add_append_rel_infos(root, appinfos); /* PostgreSQL will not set up the child rels for use, due to the games diff --git a/src/planner/partialize.c b/src/planner/partialize.c index 4552e6aeec2..ca7eb180bb0 100644 --- a/src/planner/partialize.c +++ b/src/planner/partialize.c @@ -760,7 +760,7 @@ ts_pushdown_partial_agg(PlannerInfo *root, Hypertable *ht, RelOptInfo *input_rel Query *parse = root->parse; /* We are only interested in hypertables */ - if (ht == NULL || hypertable_is_distributed(ht)) + if (!ht) return; /* Perform partial aggregation planning only if there is an aggregation is requested */ diff --git a/src/planner/planner.c b/src/planner/planner.c index e72c6f2d61e..56e262dc43e 100644 --- a/src/planner/planner.c +++ b/src/planner/planner.c @@ -274,13 +274,10 @@ ts_planner_get_hypertable(const Oid relid, const unsigned int flags) } bool -ts_rte_is_hypertable(const RangeTblEntry *rte, bool *isdistributed) +ts_rte_is_hypertable(const RangeTblEntry *rte) { Hypertable *ht = ts_planner_get_hypertable(rte->relid, CACHE_FLAG_CHECK); - if (isdistributed && ht != NULL) - *isdistributed = hypertable_is_distributed(ht); - return ht != NULL; } @@ -292,15 +289,6 @@ typedef struct Query *rootquery; Query *current_query; PlannerInfo *root; - /* - * The number of distributed hypertables in the query and its subqueries. - * Specifically, we count range table entries here, so using the same - * distributed table twice counts as two tables. No matter whether it's the - * same physical table or not, the range table entries can be scanned - * concurrently, and more than one of them being distributed means we have - * to use the cursor fetcher so that these scans can be interleaved. - */ - int num_distributed_tables; } PreprocessQueryContext; /* @@ -388,11 +376,6 @@ preprocess_query(Node *node, PreprocessQueryContext *context) query->rowMarks == NIL && rte->inh) rte_mark_for_expansion(rte); - if (hypertable_is_distributed(ht)) - { - context->num_distributed_tables++; - } - if (TS_HYPERTABLE_HAS_COMPRESSION_TABLE(ht)) { int compr_htid = ht->fd.compressed_hypertable_id; @@ -504,62 +487,6 @@ timescaledb_planner(Query *parse, const char *query_string, int cursor_opts, * Preprocess the hypertables in the query and warm up the caches. */ preprocess_query((Node *) parse, &context); - - /* - * Determine which type of fetcher to use. If set by GUC, use what - * is set. If the GUC says 'auto', use the COPY fetcher if we - * have at most one distributed table in the query. This enables - * parallel plans on data nodes, which speeds up the query. - * We can't use parallel plans with the cursor fetcher, because the - * cursors don't support parallel execution. This is because a - * cursor can be suspended at any time, then some arbitrary user - * code can be executed, and then the cursor is resumed. The - * parallel infrastructure doesn't have enough reentrability to - * survive this. - * We have to use a cursor fetcher when we have multiple distributed - * tables, because we might first have to get some rows from one - * table and then from another, without running either of them to - * completion first. This happens e.g. when doing a join. If we had - * a connection per table, we could avoid this requirement. - * - * Note that this function can be called recursively, e.g. when - * trying to evaluate an SQL function at the planning stage. We must - * only set/reset the fetcher type at the topmost level, that's why - * we check it's not already set. - */ - if (ts_data_node_fetcher_scan_type == AutoFetcherType) - { - reset_fetcher_type = true; - - if (context.num_distributed_tables >= 2) - { - if (ts_guc_remote_data_fetcher != CursorFetcherType && - ts_guc_remote_data_fetcher != AutoFetcherType) - { - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("only cursor fetcher is supported for this query"), - errhint("COPY or prepared statement fetching of data is not " - "supported in " - "queries with multiple distributed hypertables." - " Use cursor fetcher instead."))); - } - ts_data_node_fetcher_scan_type = CursorFetcherType; - } - else - { - if (ts_guc_remote_data_fetcher == AutoFetcherType) - { - ts_data_node_fetcher_scan_type = CopyFetcherType; - } - else - { - ts_data_node_fetcher_scan_type = ts_guc_remote_data_fetcher; - } - } - } - - Assert(ts_data_node_fetcher_scan_type != AutoFetcherType); } if (prev_planner_hook != NULL) @@ -832,17 +759,12 @@ ts_classify_relation(const PlannerInfo *root, const RelOptInfo *rel, Hypertable *ht = entry->ht; if (*ht) { - if (rte->relkind == RELKIND_FOREIGN_TABLE && !hypertable_is_distributed(*ht)) + if (rte->relkind == RELKIND_FOREIGN_TABLE) { /* * OSM chunk or other foreign chunk. We can't even access the * fdw_private for it, because it's a foreign chunk managed by a * different extension. Try to ignore it as much as possible. - * - * Note that we also have to disambiguate them from distributed - * hypertable chunks, which are also foreign. We can't use the - * fdwroutine here because it is set later, in - * tsl_set_rel_pathlist(). */ return TS_REL_OTHER; } @@ -869,7 +791,7 @@ should_chunk_append(Hypertable *ht, PlannerInfo *root, RelOptInfo *rel, Path *pa ((root->parse->commandType == CMD_DELETE || root->parse->commandType == CMD_UPDATE) && bms_num_members(root->all_baserels) > 1) || #endif - !ts_guc_enable_chunk_append || hypertable_is_distributed(ht)) + !ts_guc_enable_chunk_append) return false; switch (nodeTag(path)) @@ -993,7 +915,7 @@ should_constraint_aware_append(PlannerInfo *root, Hypertable *ht, Path *path) * per-server relations without a corresponding "real" table in the * system. Further, per-server appends shouldn't need runtime pruning in any * case. */ - if (root->parse->commandType != CMD_SELECT || hypertable_is_distributed(ht)) + if (root->parse->commandType != CMD_SELECT) return false; return ts_constraint_aware_append_possible(path); @@ -1002,7 +924,7 @@ should_constraint_aware_append(PlannerInfo *root, Hypertable *ht, Path *path) static bool rte_should_expand(const RangeTblEntry *rte) { - bool is_hypertable = ts_rte_is_hypertable(rte, NULL); + bool is_hypertable = ts_rte_is_hypertable(rte); return is_hypertable && !rte->inh && ts_rte_is_marked_for_expansion(rte); } @@ -1081,43 +1003,10 @@ reenable_inheritance(PlannerInfo *root, RelOptInfo *rel, Index rti, RangeTblEntr if (set_pathlist_for_current_rel) { - bool do_distributed; - - Hypertable *ht = ts_planner_get_hypertable(rte->relid, CACHE_FLAG_NOCREATE); - Assert(ht != NULL); - - /* the hypertable will have been planned as if it was a regular table - * with no data. Since such a plan would be cheaper than any real plan, - * it would always be used, and we need to remove these plans before - * adding ours. - * - * Also, if it's a distributed hypertable and per data node queries are - * enabled then we will be throwing this below append path away. So only - * build it otherwise - */ - do_distributed = !IS_DUMMY_REL(rel) && hypertable_is_distributed(ht) && - ts_guc_enable_per_data_node_queries; - rel->pathlist = NIL; rel->partial_pathlist = NIL; - /* allow a session parameter to override the use of this datanode only path */ -#ifdef TS_DEBUG - if (do_distributed) - { - const char *allow_dn_path = - GetConfigOption("timescaledb.debug_allow_datanode_only_path", true, false); - if (allow_dn_path && pg_strcasecmp(allow_dn_path, "on") != 0) - { - do_distributed = false; - elog(DEBUG2, "creating per chunk append paths"); - } - else - elog(DEBUG2, "avoiding per chunk append paths"); - } -#endif - if (!do_distributed) - ts_set_append_rel_pathlist(root, rel, rti, rte); + ts_set_append_rel_pathlist(root, rel, rti, rte); } } @@ -1404,7 +1293,7 @@ timescaledb_get_relation_info_hook(PlannerInfo *root, Oid relation_objectid, boo * the trigger behaviour on access nodes, which would otherwise * no longer fire. */ - if (IS_UPDL_CMD(root->parse) && !hypertable_is_distributed(ht)) + if (IS_UPDL_CMD(root->parse)) mark_dummy_rel(rel); break; case TS_REL_OTHER: @@ -1517,7 +1406,7 @@ replace_hypertable_modify_paths(PlannerInfo *root, List *pathlist, RelOptInfo *i #endif mt->operation == CMD_INSERT) { - if (ht && (mt->operation == CMD_INSERT || !hypertable_is_distributed(ht))) + if (ht) { path = ts_hypertable_modify_path_create(root, mt, ht, input_rel); } diff --git a/src/planner/planner.h b/src/planner/planner.h index d3d8099ddab..c238253db02 100644 --- a/src/planner/planner.h +++ b/src/planner/planner.h @@ -25,7 +25,6 @@ typedef struct Chunk Chunk; typedef struct Hypertable Hypertable; -typedef struct TsFdwRelInfo TsFdwRelInfo; typedef struct TimescaleDBPrivate { bool appends_ordered; @@ -33,9 +32,6 @@ typedef struct TimescaleDBPrivate int order_attno; List *nested_oids; List *chunk_oids; - List *serverids; - Relids server_relids; - TsFdwRelInfo *fdw_relation_info; /* Cached chunk data for the chunk relinfo. */ Chunk *cached_chunk_struct; @@ -44,7 +40,7 @@ typedef struct TimescaleDBPrivate List *compressed_ec_em_pairs; } TimescaleDBPrivate; -extern TSDLLEXPORT bool ts_rte_is_hypertable(const RangeTblEntry *rte, bool *isdistributed); +extern TSDLLEXPORT bool ts_rte_is_hypertable(const RangeTblEntry *rte); extern TSDLLEXPORT bool ts_rte_is_marked_for_expansion(const RangeTblEntry *rte); extern TSDLLEXPORT bool ts_contain_param(Node *node); diff --git a/src/process_utility.c b/src/process_utility.c index efb183d216a..4e7a4b9bb2c 100644 --- a/src/process_utility.c +++ b/src/process_utility.c @@ -50,7 +50,6 @@ #include "ts_catalog/catalog.h" #include "chunk.h" #include "chunk_index.h" -#include "ts_catalog/chunk_data_node.h" #include "compat/compat.h" #include "copy.h" #include "errors.h" @@ -60,7 +59,6 @@ #include "hypertable.h" #include "hypertable_cache.h" #include "ts_catalog/compression_settings.h" -#include "ts_catalog/hypertable_data_node.h" #include "ts_catalog/array_utils.h" #include "dimension_vector.h" #include "indexing.h" @@ -695,8 +693,7 @@ add_chunk_to_vacuum(Hypertable *ht, Oid chunk_relid, void *arg) /* * Construct a list of VacuumRelations for all vacuumable rels in * the current database. This is similar to the PostgresQL get_all_vacuum_rels - * from vacuum.c, only it filters out distributed hypertables and chunks - * that have been compressed. + * from vacuum.c. */ static List * ts_get_all_vacuum_rels(bool is_vacuumcmd) @@ -714,7 +711,6 @@ ts_get_all_vacuum_rels(bool is_vacuumcmd) while ((tuple = heap_getnext(scan, ForwardScanDirection)) != NULL) { Form_pg_class classform = (Form_pg_class) GETSTRUCT(tuple); - Hypertable *ht; Oid relid; relid = classform->oid; @@ -734,11 +730,6 @@ ts_get_all_vacuum_rels(bool is_vacuumcmd) classform->relkind != RELKIND_PARTITIONED_TABLE) continue; - ht = ts_hypertable_cache_get_entry(hcache, relid, CACHE_FLAG_MISSING_OK); - if (ht) - if (hypertable_is_distributed(ht)) - continue; - /* * Build VacuumRelation(s) specifying the table OIDs to be processed. * We omit a RangeVar since it wouldn't be appropriate to complain @@ -811,15 +802,6 @@ process_vacuum(ProcessUtilityArgs *args) { add_hypertable_to_process_args(args, ht); - /* Exclude distributed hypertables from the list of relations - * to vacuum and analyze since they contain no local tuples. - * - * Support for VACUUM/ANALYZE operations on a distributed hypertable - * is implemented as a part of distributed ddl and remote - * statistics import functions. - */ - if (hypertable_is_distributed(ht)) - continue; ctx.ht_vacuum_rel = vacuum_rel; foreach_chunk(ht, add_chunk_to_vacuum, &ctx); } @@ -1009,12 +991,7 @@ process_truncate(ProcessUtilityArgs *args) hypertables = lappend(hypertables, ht); - if (!hypertable_is_distributed(ht)) - list_append = true; - else - /* mark list as changed because we'll not add the distributed hypertable - */ - list_changed = true; + list_append = true; } else if ((chunk = ts_chunk_get_by_relid(relid, false)) != NULL) { /* this is a chunk */ @@ -1777,15 +1754,8 @@ process_reindex(ProcessUtilityArgs *args) (errmsg("concurrent index creation on hypertables is not supported"))); /* Do not process remote chunks in case of distributed hypertable */ - if (hypertable_is_distributed(ht)) - { + if (foreach_chunk(ht, reindex_chunk, args) >= 0) result = DDL_DONE; - } - else - { - if (foreach_chunk(ht, reindex_chunk, args) >= 0) - result = DDL_DONE; - } add_hypertable_to_process_args(args, ht); } @@ -2062,7 +2032,7 @@ process_rename_constraint_or_trigger(ProcessUtilityArgs *args, Cache *hcache, Oi if (stmt->renameType == OBJECT_TABCONSTRAINT) foreach_chunk(ht, rename_hypertable_constraint, stmt); - else if (stmt->renameType == OBJECT_TRIGGER && !hypertable_is_distributed(ht)) + else if (stmt->renameType == OBJECT_TRIGGER) foreach_chunk(ht, rename_hypertable_trigger, stmt); } else if (stmt->renameType == OBJECT_TABCONSTRAINT) @@ -2392,8 +2362,6 @@ process_index_chunk(Hypertable *ht, Oid chunk_relid, void *arg) IndexInfo *indexinfo; Chunk *chunk; - Assert(!hypertable_is_distributed(ht)); - chunk = ts_chunk_get_by_relid(chunk_relid, true); if (IS_OSM_CHUNK(chunk)) /*cannot create index on foreign OSM chunk */ { @@ -2672,12 +2640,6 @@ process_index_start(ProcessUtilityArgs *args) errmsg( "cannot use timescaledb.transaction_per_chunk with UNIQUE or PRIMARY KEY"))); - if (info.extended_options.multitransaction && hypertable_is_distributed(ht)) - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg( - "cannot use timescaledb.transaction_per_chunk with distributed hypertable"))); - ts_indexing_verify_index(ht->space, stmt); if (info.extended_options.multitransaction) @@ -2699,8 +2661,7 @@ process_index_start(ProcessUtilityArgs *args) /* CREATE INDEX on the root table of the hypertable */ root_table_index = ts_indexing_root_table_create_index(stmt, args->query_string, - info.extended_options.multitransaction, - hypertable_is_distributed(ht)); + info.extended_options.multitransaction); if (cagg) RESTORE_USER(uid, saved_uid, sec_ctx); @@ -2722,13 +2683,6 @@ process_index_start(ProcessUtilityArgs *args) return DDL_DONE; } - /* CREATE INDEX on the chunks, unless this is a distributed hypertable */ - if (hypertable_is_distributed(ht)) - { - ts_cache_release(hcache); - return DDL_DONE; - } - info.obj.objectId = root_table_index.objectId; /* collect information required for per chunk index creation */ main_table_relation = table_open(ht->main_table_relid, AccessShareLock); @@ -3730,8 +3684,7 @@ process_altertable_end_subcmd(Hypertable *ht, Node *parsetree, ObjectAddress *ob /* Avoid running this command for distributed hypertable chunks * since PostgreSQL currently does not allow to alter * storage options for a foreign table. */ - if (!hypertable_is_distributed(ht)) - foreach_chunk(ht, process_altertable_chunk, cmd); + foreach_chunk(ht, process_altertable_chunk, cmd); break; case AT_SetTableSpace: process_altertable_set_tablespace_end(ht, cmd); @@ -4337,16 +4290,6 @@ process_drop_view(EventTriggerDropView *dropped_view) ts_continuous_agg_drop(dropped_view->schema, dropped_view->view_name); } -static void -process_drop_foreign_server(EventTriggerDropObject *obj) -{ - EventTriggerDropForeignServer *server = (EventTriggerDropForeignServer *) obj; - - Assert(obj->type == EVENT_TRIGGER_DROP_FOREIGN_SERVER); - ts_hypertable_data_node_delete_by_node_name(server->servername); - ts_chunk_data_node_delete_by_node_name(server->servername); -} - static void process_ddl_sql_drop(EventTriggerDropObject *obj) { @@ -4359,7 +4302,6 @@ process_ddl_sql_drop(EventTriggerDropObject *obj) process_drop_index(obj); break; case EVENT_TRIGGER_DROP_TABLE: - case EVENT_TRIGGER_DROP_FOREIGN_TABLE: process_drop_table(obj); break; case EVENT_TRIGGER_DROP_SCHEMA: @@ -4371,8 +4313,8 @@ process_ddl_sql_drop(EventTriggerDropObject *obj) case EVENT_TRIGGER_DROP_VIEW: process_drop_view((EventTriggerDropView *) obj); break; + case EVENT_TRIGGER_DROP_FOREIGN_TABLE: case EVENT_TRIGGER_DROP_FOREIGN_SERVER: - process_drop_foreign_server(obj); break; } } diff --git a/src/telemetry/stats.c b/src/telemetry/stats.c index f0de5694e59..202bafb2669 100644 --- a/src/telemetry/stats.c +++ b/src/telemetry/stats.c @@ -277,10 +277,6 @@ add_chunk_stats(HyperStats *stats, Form_pg_class class, const Chunk *chunk, if (ts_chunk_is_compressed(chunk)) stats->compressed_chunk_count++; - /* Add replica chunks, if any. Only count the extra replicas */ - if (list_length(chunk->data_nodes) > 1) - stats->replica_chunk_count += (list_length(chunk->data_nodes) - 1); - /* * A chunk on a distributed hypertable can be marked as compressed but * have no compression stats (the stats exists on the data node and might diff --git a/src/ts_catalog/CMakeLists.txt b/src/ts_catalog/CMakeLists.txt index 6f619c832d2..d20d065ce3d 100644 --- a/src/ts_catalog/CMakeLists.txt +++ b/src/ts_catalog/CMakeLists.txt @@ -1,13 +1,11 @@ set(SOURCES ${CMAKE_CURRENT_SOURCE_DIR}/array_utils.c ${CMAKE_CURRENT_SOURCE_DIR}/catalog.c - ${CMAKE_CURRENT_SOURCE_DIR}/chunk_data_node.c ${CMAKE_CURRENT_SOURCE_DIR}/compression_chunk_size.c ${CMAKE_CURRENT_SOURCE_DIR}/compression_settings.c ${CMAKE_CURRENT_SOURCE_DIR}/continuous_agg.c ${CMAKE_CURRENT_SOURCE_DIR}/continuous_aggs_watermark.c ${CMAKE_CURRENT_SOURCE_DIR}/dimension_partition.c - ${CMAKE_CURRENT_SOURCE_DIR}/hypertable_data_node.c ${CMAKE_CURRENT_SOURCE_DIR}/metadata.c ${CMAKE_CURRENT_SOURCE_DIR}/tablespace.c) target_sources(${PROJECT_NAME} PRIVATE ${SOURCES}) diff --git a/src/ts_catalog/chunk_data_node.c b/src/ts_catalog/chunk_data_node.c deleted file mode 100644 index d9ea0ddddbb..00000000000 --- a/src/ts_catalog/chunk_data_node.c +++ /dev/null @@ -1,444 +0,0 @@ -/* - * 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. - */ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include "ts_catalog/chunk_data_node.h" -#include "cache.h" -#include "hypercube.h" -#include "hypertable_cache.h" -#include "scanner.h" -#include "chunk.h" -#include "debug_point.h" - -static void -chunk_data_node_insert_relation(const Relation rel, int32 chunk_id, int32 node_chunk_id, - const NameData *node_name) -{ - TupleDesc desc = RelationGetDescr(rel); - Datum values[Natts_chunk_data_node]; - bool nulls[Natts_chunk_data_node] = { false }; - CatalogSecurityContext sec_ctx; - - values[AttrNumberGetAttrOffset(Anum_chunk_data_node_chunk_id)] = Int32GetDatum(chunk_id); - values[AttrNumberGetAttrOffset(Anum_chunk_data_node_node_chunk_id)] = - Int32GetDatum(node_chunk_id); - values[AttrNumberGetAttrOffset(Anum_chunk_data_node_node_name)] = NameGetDatum(node_name); - - ts_catalog_database_info_become_owner(ts_catalog_database_info_get(), &sec_ctx); - ts_catalog_insert_values(rel, desc, values, nulls); - ts_catalog_restore_user(&sec_ctx); -} - -static void -chunk_data_node_insert_internal(int32 chunk_id, int32 node_chunk_id, const NameData *node_name) -{ - Catalog *catalog = ts_catalog_get(); - Relation rel; - - rel = table_open(catalog->tables[CHUNK_DATA_NODE].id, RowExclusiveLock); - - chunk_data_node_insert_relation(rel, chunk_id, node_chunk_id, node_name); - - table_close(rel, RowExclusiveLock); -} - -void -ts_chunk_data_node_insert(const ChunkDataNode *node) -{ - chunk_data_node_insert_internal(node->fd.chunk_id, node->fd.node_chunk_id, &node->fd.node_name); -} - -void -ts_chunk_data_node_insert_multi(List *chunk_data_nodes) -{ - Catalog *catalog = ts_catalog_get(); - Relation rel; - ListCell *lc; - - rel = table_open(catalog->tables[CHUNK_DATA_NODE].id, RowExclusiveLock); - - foreach (lc, chunk_data_nodes) - { - ChunkDataNode *node = lfirst(lc); - - chunk_data_node_insert_relation(rel, - node->fd.chunk_id, - node->fd.node_chunk_id, - &node->fd.node_name); - } - - table_close(rel, RowExclusiveLock); -} - -static int -chunk_data_node_scan_limit_internal(ScanKeyData *scankey, int num_scankeys, int indexid, - tuple_found_func on_tuple_found, void *scandata, int limit, - LOCKMODE lock, ScanTupLock *tuplock, MemoryContext mctx) -{ - Catalog *catalog = ts_catalog_get(); - ScannerCtx scanctx = { - .table = catalog->tables[CHUNK_DATA_NODE].id, - .index = catalog_get_index(catalog, CHUNK_DATA_NODE, indexid), - .nkeys = num_scankeys, - .scankey = scankey, - .data = scandata, - .limit = limit, - .tuple_found = on_tuple_found, - .tuplock = tuplock, - .lockmode = lock, - .scandirection = ForwardScanDirection, - .result_mctx = mctx, - }; - - return ts_scanner_scan(&scanctx); -} - -static ScanTupleResult -chunk_data_node_tuple_found(TupleInfo *ti, void *data) -{ - List **nodes = data; - bool should_free; - HeapTuple tuple = ts_scanner_fetch_heap_tuple(ti, false, &should_free); - Form_chunk_data_node form = (Form_chunk_data_node) GETSTRUCT(tuple); - ForeignServer *server; - ChunkDataNode *chunk_data_node; - MemoryContext old; - - server = GetForeignServerByName(NameStr(form->node_name), false); - old = MemoryContextSwitchTo(ti->mctx); - chunk_data_node = palloc(sizeof(ChunkDataNode)); - memcpy(&chunk_data_node->fd, form, sizeof(FormData_chunk_data_node)); - chunk_data_node->foreign_server_oid = server->serverid; - *nodes = lappend(*nodes, chunk_data_node); - MemoryContextSwitchTo(old); - - if (should_free) - heap_freetuple(tuple); - - return SCAN_CONTINUE; -} - -/* return a filtered list of "available" ChunkDataNode entries */ -static ScanTupleResult -chunk_data_node_tuple_found_filter(TupleInfo *ti, void *data) -{ - List **nodes = data; - bool should_free; - HeapTuple tuple = ts_scanner_fetch_heap_tuple(ti, false, &should_free); - Form_chunk_data_node form = (Form_chunk_data_node) GETSTRUCT(tuple); - ForeignServer *server; - - server = GetForeignServerByName(NameStr(form->node_name), false); - - if (ts_data_node_is_available_by_server(server)) - { - ChunkDataNode *chunk_data_node; - MemoryContext old; - - old = MemoryContextSwitchTo(ti->mctx); - chunk_data_node = palloc(sizeof(ChunkDataNode)); - memcpy(&chunk_data_node->fd, form, sizeof(FormData_chunk_data_node)); - chunk_data_node->foreign_server_oid = server->serverid; - *nodes = lappend(*nodes, chunk_data_node); - MemoryContextSwitchTo(old); - } - - if (should_free) - heap_freetuple(tuple); - - return SCAN_CONTINUE; -} - -static int -ts_chunk_data_node_scan_by_chunk_id_and_node_internal(int32 chunk_id, const char *node_name, - bool scan_by_remote_chunk_id, - tuple_found_func tuple_found, void *data, - LOCKMODE lockmode, ScanTupLock *tuplock, - MemoryContext mctx) -{ - ScanKeyData scankey[2]; - int nkeys = 0; - int attrnum_chunk_id; - int attrnum_node_name; - int indexid; - - if (scan_by_remote_chunk_id) - { - attrnum_chunk_id = Anum_chunk_data_node_node_chunk_id_node_name_idx_chunk_id; - attrnum_node_name = Anum_chunk_data_node_node_chunk_id_node_name_idx_node_name; - indexid = CHUNK_DATA_NODE_NODE_CHUNK_ID_NODE_NAME_IDX; - } - else - { - attrnum_chunk_id = Anum_chunk_data_node_chunk_id_node_name_idx_chunk_id; - attrnum_node_name = Anum_chunk_data_node_chunk_id_node_name_idx_node_name; - indexid = CHUNK_DATA_NODE_CHUNK_ID_NODE_NAME_IDX; - } - - ScanKeyInit(&scankey[nkeys++], - attrnum_chunk_id, - BTEqualStrategyNumber, - F_INT4EQ, - Int32GetDatum(chunk_id)); - - if (NULL != node_name) - ScanKeyInit(&scankey[nkeys++], - attrnum_node_name, - BTEqualStrategyNumber, - F_NAMEEQ, - CStringGetDatum(node_name)); - - return chunk_data_node_scan_limit_internal(scankey, - nkeys, - indexid, - tuple_found, - data, - 0, - lockmode, - tuplock, - mctx); -} - -static int -ts_chunk_data_node_scan_by_node_internal(const char *node_name, tuple_found_func tuple_found, - void *data, LOCKMODE lockmode, ScanTupLock *tuplock, - MemoryContext mctx) -{ - ScanKeyData scankey[1]; - - ScanKeyInit(&scankey[0], - Anum_chunk_data_node_node_name, - BTEqualStrategyNumber, - F_NAMEEQ, - CStringGetDatum(node_name)); - - return chunk_data_node_scan_limit_internal(scankey, - 1, - INVALID_INDEXID, - tuple_found, - data, - 0, - lockmode, - tuplock, - mctx); -} - -/* Returns a List of ChunkDataNode structs. */ -List * -ts_chunk_data_node_scan_by_chunk_id(int32 chunk_id, MemoryContext mctx) -{ - List *chunk_data_nodes = NIL; - ts_chunk_data_node_scan_by_chunk_id_and_node_internal(chunk_id, - NULL, - false, - chunk_data_node_tuple_found, - &chunk_data_nodes, - AccessShareLock, - NULL, - mctx); - return chunk_data_nodes; -} - -/* Returns a filtered List of available ChunkDataNode structs. */ -List * -ts_chunk_data_node_scan_by_chunk_id_filter(int32 chunk_id, MemoryContext mctx) -{ - List *chunk_data_nodes = NIL; - ts_chunk_data_node_scan_by_chunk_id_and_node_internal(chunk_id, - NULL, - false, - chunk_data_node_tuple_found_filter, - &chunk_data_nodes, - AccessShareLock, - NULL, - mctx); - return chunk_data_nodes; -} - -static ChunkDataNode * -chunk_data_node_scan_by_chunk_id_and_node_name(int32 chunk_id, const char *node_name, - bool scan_by_remote_chunk_id, MemoryContext mctx) - -{ - List *chunk_data_nodes = NIL; - ts_chunk_data_node_scan_by_chunk_id_and_node_internal(chunk_id, - node_name, - scan_by_remote_chunk_id, - chunk_data_node_tuple_found, - &chunk_data_nodes, - AccessShareLock, - NULL, - mctx); - Assert(list_length(chunk_data_nodes) <= 1); - - if (chunk_data_nodes == NIL) - return NULL; - return linitial(chunk_data_nodes); -} - -ChunkDataNode * -ts_chunk_data_node_scan_by_chunk_id_and_node_name(int32 chunk_id, const char *node_name, - MemoryContext mctx) - -{ - return chunk_data_node_scan_by_chunk_id_and_node_name(chunk_id, node_name, false, mctx); -} - -ChunkDataNode * -ts_chunk_data_node_scan_by_remote_chunk_id_and_node_name(int32 chunk_id, const char *node_name, - MemoryContext mctx) - -{ - return chunk_data_node_scan_by_chunk_id_and_node_name(chunk_id, node_name, true, mctx); -} - -static ScanTupleResult -chunk_data_node_tuple_delete(TupleInfo *ti, void *data) -{ - CatalogSecurityContext sec_ctx; - - switch (ti->lockresult) - { - case TM_Ok: - ts_catalog_database_info_become_owner(ts_catalog_database_info_get(), &sec_ctx); - ts_catalog_delete_tid(ti->scanrel, ts_scanner_get_tuple_tid(ti)); - ts_catalog_restore_user(&sec_ctx); - break; - case TM_Deleted: - /* Already deleted, do nothing. */ - break; - default: - Assert(false); - break; - } - - return SCAN_CONTINUE; -} - -int -ts_chunk_data_node_delete_by_chunk_id(int32 chunk_id) -{ - ScanTupLock tuplock = { - .lockmode = LockTupleExclusive, - .waitpolicy = LockWaitBlock, - }; - - return ts_chunk_data_node_scan_by_chunk_id_and_node_internal(chunk_id, - NULL, - false, - chunk_data_node_tuple_delete, - NULL, - RowExclusiveLock, - &tuplock, - CurrentMemoryContext); -} - -int -ts_chunk_data_node_delete_by_chunk_id_and_node_name(int32 chunk_id, const char *node_name) -{ - int count; - - ScanTupLock tuplock = { - .lockmode = LockTupleExclusive, - .waitpolicy = LockWaitBlock, - }; - - count = ts_chunk_data_node_scan_by_chunk_id_and_node_internal(chunk_id, - node_name, - false, - chunk_data_node_tuple_delete, - NULL, - RowExclusiveLock, - &tuplock, - CurrentMemoryContext); - DEBUG_WAITPOINT("chunk_data_node_delete"); - return count; -} - -int -ts_chunk_data_node_delete_by_node_name(const char *node_name) -{ - ScanTupLock tuplock = { - .lockmode = LockTupleExclusive, - .waitpolicy = LockWaitBlock, - }; - return ts_chunk_data_node_scan_by_node_internal(node_name, - chunk_data_node_tuple_delete, - NULL, - RowExclusiveLock, - &tuplock, - CurrentMemoryContext); -} - -List * -ts_chunk_data_node_scan_by_node_name_and_hypertable_id(const char *node_name, int32 hypertable_id, - MemoryContext mctx) -{ - List *results = NIL; - ListCell *lc; - MemoryContext old; - List *chunk_ids = NIL; - - old = MemoryContextSwitchTo(mctx); - chunk_ids = ts_chunk_get_chunk_ids_by_hypertable_id(hypertable_id); - - foreach (lc, chunk_ids) - { - int32 chunk_id = lfirst_int(lc); - ChunkDataNode *cdn = - ts_chunk_data_node_scan_by_chunk_id_and_node_name(chunk_id, node_name, mctx); - if (cdn != NULL) - results = lappend(results, cdn); - } - - MemoryContextSwitchTo(old); - return results; -} - -ScanIterator -ts_chunk_data_nodes_scan_iterator_create(MemoryContext result_mcxt) -{ - ScanIterator it = ts_scan_iterator_create(CHUNK_DATA_NODE, AccessShareLock, result_mcxt); - it.ctx.flags |= SCANNER_F_NOEND_AND_NOCLOSE; - - return it; -} - -void -ts_chunk_data_nodes_scan_iterator_set_chunk_id(ScanIterator *it, int32 chunk_id) -{ - it->ctx.index = catalog_get_index(ts_catalog_get(), - CHUNK_DATA_NODE, - CHUNK_DATA_NODE_CHUNK_ID_NODE_NAME_IDX); - ts_scan_iterator_scan_key_reset(it); - ts_scan_iterator_scan_key_init(it, - Anum_chunk_data_node_chunk_id_node_name_idx_chunk_id, - BTEqualStrategyNumber, - F_INT4EQ, - Int32GetDatum(chunk_id)); -} - -void -ts_chunk_data_nodes_scan_iterator_set_node_name(ScanIterator *it, const char *node_name) -{ - it->ctx.index = - catalog_get_index(ts_catalog_get(), CHUNK_DATA_NODE, CHUNK_DATA_NODE_NODE_NAME_IDX); - ts_scan_iterator_scan_key_reset(it); - ts_scan_iterator_scan_key_init(it, - Anum_chunk_data_node_name_idx_node_name, - BTEqualStrategyNumber, - F_NAMEEQ, - CStringGetDatum(node_name)); -} diff --git a/src/ts_catalog/chunk_data_node.h b/src/ts_catalog/chunk_data_node.h deleted file mode 100644 index 9b22dff3a74..00000000000 --- a/src/ts_catalog/chunk_data_node.h +++ /dev/null @@ -1,41 +0,0 @@ -/* - * 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. - */ -#pragma once - -#include "ts_catalog/catalog.h" -#include "chunk.h" -#include "export.h" -#include "scan_iterator.h" - -typedef struct ChunkDataNode -{ - FormData_chunk_data_node fd; - Oid foreign_server_oid; -} ChunkDataNode; - -extern TSDLLEXPORT List *ts_chunk_data_node_scan_by_chunk_id(int32 chunk_id, MemoryContext mctx); -extern TSDLLEXPORT List *ts_chunk_data_node_scan_by_chunk_id_filter(int32 chunk_id, - MemoryContext mctx); -extern TSDLLEXPORT ChunkDataNode * -ts_chunk_data_node_scan_by_chunk_id_and_node_name(int32 chunk_id, const char *node_name, - MemoryContext mctx); -extern TSDLLEXPORT ChunkDataNode * -ts_chunk_data_node_scan_by_remote_chunk_id_and_node_name(int32 chunk_id, const char *node_name, - MemoryContext mctx); -extern TSDLLEXPORT void ts_chunk_data_node_insert(const ChunkDataNode *node); -extern void ts_chunk_data_node_insert_multi(List *chunk_data_nodes); -extern int ts_chunk_data_node_delete_by_chunk_id(int32 chunk_id); -extern TSDLLEXPORT int ts_chunk_data_node_delete_by_chunk_id_and_node_name(int32 chunk_id, - const char *node_name); -extern int ts_chunk_data_node_delete_by_node_name(const char *node_name); -extern TSDLLEXPORT List * -ts_chunk_data_node_scan_by_node_name_and_hypertable_id(const char *node_name, int32 hypertable_id, - MemoryContext mctx); -extern TSDLLEXPORT ScanIterator ts_chunk_data_nodes_scan_iterator_create(MemoryContext result_mcxt); -extern TSDLLEXPORT void ts_chunk_data_nodes_scan_iterator_set_chunk_id(ScanIterator *it, - int32 chunk_id); -extern TSDLLEXPORT void ts_chunk_data_nodes_scan_iterator_set_node_name(ScanIterator *it, - const char *node_name); diff --git a/src/ts_catalog/continuous_agg.c b/src/ts_catalog/continuous_agg.c index 11e0499110a..a5e58fe3206 100644 --- a/src/ts_catalog/continuous_agg.c +++ b/src/ts_catalog/continuous_agg.c @@ -1176,8 +1176,6 @@ drop_continuous_agg(FormData_continuous_agg *cadata, bool drop_user_view) if (OidIsValid(raw_hypertable_trig.objectId)) { ts_hypertable_drop_trigger(raw_hypertable.objectId, CAGGINVAL_TRIGGER_NAME); - if (ts_cm_functions->remote_drop_dist_ht_invalidation_trigger) - ts_cm_functions->remote_drop_dist_ht_invalidation_trigger(cadata->raw_hypertable_id); } if (OidIsValid(mat_hypertable.objectId)) diff --git a/src/ts_catalog/hypertable_data_node.c b/src/ts_catalog/hypertable_data_node.c deleted file mode 100644 index 01201659733..00000000000 --- a/src/ts_catalog/hypertable_data_node.c +++ /dev/null @@ -1,347 +0,0 @@ -/* - * 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. - */ -#include -#include -#include -#include -#include -#include -#include -#include - -#include "ts_catalog/hypertable_data_node.h" -#include "scanner.h" -#include "ts_catalog/catalog.h" -#include "compat/compat.h" - -static void -hypertable_data_node_insert_relation(Relation rel, int32 hypertable_id, int32 node_hypertable_id, - Name node_name, bool block_chunks) -{ - TupleDesc desc = RelationGetDescr(rel); - Datum values[Natts_hypertable_data_node]; - bool nulls[Natts_hypertable_data_node] = { false }; - CatalogSecurityContext sec_ctx; - - values[AttrNumberGetAttrOffset(Anum_hypertable_data_node_hypertable_id)] = - Int32GetDatum(hypertable_id); - values[AttrNumberGetAttrOffset(Anum_hypertable_data_node_node_name)] = NameGetDatum(node_name); - values[AttrNumberGetAttrOffset(Anum_hypertable_data_node_block_chunks)] = block_chunks; - - if (node_hypertable_id > 0) - values[AttrNumberGetAttrOffset(Anum_hypertable_data_node_node_hypertable_id)] = - Int32GetDatum(node_hypertable_id); - else - nulls[AttrNumberGetAttrOffset(Anum_hypertable_data_node_node_hypertable_id)] = true; - - ts_catalog_database_info_become_owner(ts_catalog_database_info_get(), &sec_ctx); - ts_catalog_insert_values(rel, desc, values, nulls); - ts_catalog_restore_user(&sec_ctx); -} - -void -ts_hypertable_data_node_insert_multi(List *hypertable_data_nodes) -{ - Catalog *catalog = ts_catalog_get(); - Relation rel; - ListCell *lc; - Oid curuserid = GetUserId(); - - rel = table_open(catalog->tables[HYPERTABLE_DATA_NODE].id, RowExclusiveLock); - - foreach (lc, hypertable_data_nodes) - { - HypertableDataNode *node = lfirst(lc); - AclResult aclresult; - - /* Must also have usage on the server object */ - aclresult = object_aclcheck(ForeignServerRelationId, - node->foreign_server_oid, - curuserid, - ACL_USAGE); - - if (aclresult != ACLCHECK_OK) - { - aclcheck_error(aclresult, OBJECT_FOREIGN_SERVER, NameStr(node->fd.node_name)); - } - hypertable_data_node_insert_relation(rel, - node->fd.hypertable_id, - node->fd.node_hypertable_id, - &node->fd.node_name, - node->fd.block_chunks); - } - - table_close(rel, RowExclusiveLock); -} - -static int -hypertable_data_node_scan_limit_internal(ScanKeyData *scankey, int num_scankeys, int indexid, - tuple_found_func on_tuple_found, void *scandata, int limit, - LOCKMODE lock, MemoryContext mctx) -{ - Catalog *catalog = ts_catalog_get(); - ScannerCtx scanctx = { - .table = catalog->tables[HYPERTABLE_DATA_NODE].id, - .index = catalog_get_index(catalog, HYPERTABLE_DATA_NODE, indexid), - .nkeys = num_scankeys, - .scankey = scankey, - .data = scandata, - .limit = limit, - .tuple_found = on_tuple_found, - .lockmode = lock, - .scandirection = ForwardScanDirection, - .result_mctx = mctx, - }; - - return ts_scanner_scan(&scanctx); -} - -static ScanTupleResult -hypertable_data_node_tuple_delete(TupleInfo *ti, void *data) -{ - CatalogSecurityContext sec_ctx; - - ts_catalog_database_info_become_owner(ts_catalog_database_info_get(), &sec_ctx); - ts_catalog_delete_tid(ti->scanrel, ts_scanner_get_tuple_tid(ti)); - ts_catalog_restore_user(&sec_ctx); - - return SCAN_CONTINUE; -} - -static ScanTupleResult -hypertable_data_node_tuple_update(TupleInfo *ti, void *data) -{ - CatalogSecurityContext sec_ctx; - const HypertableDataNode *update = data; - bool should_free; - HeapTuple tuple = ts_scanner_fetch_heap_tuple(ti, false, &should_free); - HeapTuple new_tuple = heap_copytuple(tuple); - FormData_hypertable_data_node *form = (FormData_hypertable_data_node *) GETSTRUCT(new_tuple); - - if (should_free) - heap_freetuple(tuple); - - Assert(form->hypertable_id == update->fd.hypertable_id); - Assert(strcmp(NameStr(form->node_name), NameStr(update->fd.node_name)) == 0); - - form->node_hypertable_id = update->fd.node_hypertable_id; - form->block_chunks = update->fd.block_chunks; - - ts_catalog_database_info_become_owner(ts_catalog_database_info_get(), &sec_ctx); - ts_catalog_update(ti->scanrel, new_tuple); - ts_catalog_restore_user(&sec_ctx); - - heap_freetuple(new_tuple); - - return SCAN_DONE; -} - -static HypertableDataNode * -hypertable_data_node_create_from_tuple(TupleInfo *ti) -{ - const char *node_name; - HypertableDataNode *hypertable_data_node; - ForeignServer *foreign_server; - MemoryContext old; - Datum values[Natts_hypertable_data_node]; - bool nulls[Natts_hypertable_data_node] = { false }; - bool should_free; - HeapTuple tuple = ts_scanner_fetch_heap_tuple(ti, false, &should_free); - - /* - * Need to use heap_deform_tuple instead of GETSTRUCT since the tuple can - * contain NULL values - */ - heap_deform_tuple(tuple, ts_scanner_get_tupledesc(ti), values, nulls); - - node_name = - DatumGetCString(values[AttrNumberGetAttrOffset(Anum_hypertable_data_node_node_name)]); - foreign_server = GetForeignServerByName(node_name, false); - - old = MemoryContextSwitchTo(ti->mctx); - hypertable_data_node = palloc0(sizeof(HypertableDataNode)); - - hypertable_data_node->fd.hypertable_id = - DatumGetInt32(values[AttrNumberGetAttrOffset(Anum_hypertable_data_node_hypertable_id)]); - namestrcpy(&hypertable_data_node->fd.node_name, node_name); - - if (nulls[AttrNumberGetAttrOffset(Anum_hypertable_data_node_node_hypertable_id)]) - hypertable_data_node->fd.node_hypertable_id = 0; - else - hypertable_data_node->fd.node_hypertable_id = DatumGetInt32( - values[AttrNumberGetAttrOffset(Anum_hypertable_data_node_node_hypertable_id)]); - - hypertable_data_node->fd.block_chunks = - DatumGetBool(values[AttrNumberGetAttrOffset(Anum_hypertable_data_node_block_chunks)]); - hypertable_data_node->foreign_server_oid = foreign_server->serverid; - MemoryContextSwitchTo(old); - - if (should_free) - heap_freetuple(tuple); - - return hypertable_data_node; -} - -static ScanTupleResult -hypertable_data_node_tuples_found(TupleInfo *ti, void *data) -{ - List **nodes = data; - MemoryContext old; - HypertableDataNode *hypertable_data_node = hypertable_data_node_create_from_tuple(ti); - - old = MemoryContextSwitchTo(ti->mctx); - *nodes = lappend(*nodes, hypertable_data_node); - MemoryContextSwitchTo(old); - - return SCAN_CONTINUE; -} - -static int -hypertable_data_node_scan_by_hypertable_id(int32 hypertable_id, tuple_found_func tuple_found, - void *data, LOCKMODE lockmode, MemoryContext mctx) -{ - ScanKeyData scankey[1]; - - ScanKeyInit(&scankey[0], - Anum_hypertable_data_node_hypertable_id_node_name_idx_hypertable_id, - BTEqualStrategyNumber, - F_INT4EQ, - Int32GetDatum(hypertable_id)); - - return hypertable_data_node_scan_limit_internal( - scankey, - 1, - HYPERTABLE_DATA_NODE_HYPERTABLE_ID_NODE_NAME_IDX, - tuple_found, - data, - 0, - lockmode, - mctx); -} - -static int -hypertable_data_node_scan_by_node_name(const char *node_name, tuple_found_func tuple_found, - void *data, LOCKMODE lockmode, MemoryContext mctx) -{ - ScanKeyData scankey[1]; - - ScanKeyInit(&scankey[0], - Anum_hypertable_data_node_node_name, - BTEqualStrategyNumber, - F_NAMEEQ, - CStringGetDatum(node_name)); - - return hypertable_data_node_scan_limit_internal(scankey, - 1, - INVALID_INDEXID, - tuple_found, - data, - 0, - lockmode, - mctx); -} - -static int -hypertable_data_node_scan_by_hypertable_id_and_node_name(int hypertable_id, const char *node_name, - tuple_found_func tuple_found, void *data, - LOCKMODE lockmode, MemoryContext mctx) -{ - ScanKeyData scankey[2]; - - ScanKeyInit(&scankey[0], - Anum_hypertable_data_node_hypertable_id_node_name_idx_hypertable_id, - BTEqualStrategyNumber, - F_INT4EQ, - Int32GetDatum(hypertable_id)); - - ScanKeyInit(&scankey[1], - Anum_hypertable_data_node_hypertable_id_node_name_idx_node_name, - BTEqualStrategyNumber, - F_NAMEEQ, - CStringGetDatum(node_name)); - - return hypertable_data_node_scan_limit_internal( - scankey, - 2, - HYPERTABLE_DATA_NODE_HYPERTABLE_ID_NODE_NAME_IDX, - tuple_found, - data, - 0, - lockmode, - mctx); -} - -List * -ts_hypertable_data_node_scan(int32 hypertable_id, MemoryContext mctx) -{ - List *hypertable_data_nodes = NIL; - - hypertable_data_node_scan_by_hypertable_id(hypertable_id, - hypertable_data_node_tuples_found, - &hypertable_data_nodes, - AccessShareLock, - mctx); - - return hypertable_data_nodes; -} - -int -ts_hypertable_data_node_delete_by_hypertable_id(int32 hypertable_id) -{ - return hypertable_data_node_scan_by_hypertable_id(hypertable_id, - hypertable_data_node_tuple_delete, - NULL, - RowExclusiveLock, - CurrentMemoryContext); -} - -int -ts_hypertable_data_node_delete_by_node_name(const char *node_name) -{ - return hypertable_data_node_scan_by_node_name(node_name, - hypertable_data_node_tuple_delete, - NULL, - RowExclusiveLock, - CurrentMemoryContext); -} - -int -ts_hypertable_data_node_delete_by_node_name_and_hypertable_id(const char *node_name, - int32 hypertable_id) -{ - return hypertable_data_node_scan_by_hypertable_id_and_node_name( - hypertable_id, - node_name, - hypertable_data_node_tuple_delete, - NULL, - RowExclusiveLock, - CurrentMemoryContext); -} - -List * -ts_hypertable_data_node_scan_by_node_name(const char *node_name, MemoryContext mctx) -{ - List *hypertable_data_nodes = NIL; - - hypertable_data_node_scan_by_node_name(node_name, - hypertable_data_node_tuples_found, - &hypertable_data_nodes, - AccessShareLock, - mctx); - return hypertable_data_nodes; -} - -int -ts_hypertable_data_node_update(const HypertableDataNode *hypertable_data_node) -{ - return hypertable_data_node_scan_by_hypertable_id_and_node_name( - hypertable_data_node->fd.hypertable_id, - NameStr(hypertable_data_node->fd.node_name), - hypertable_data_node_tuple_update, - (void *) hypertable_data_node, - RowExclusiveLock, - CurrentMemoryContext); -} diff --git a/src/ts_catalog/hypertable_data_node.h b/src/ts_catalog/hypertable_data_node.h deleted file mode 100644 index 9273e14fe88..00000000000 --- a/src/ts_catalog/hypertable_data_node.h +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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. - */ -#pragma once - -#include "ts_catalog/catalog.h" -#include "export.h" - -typedef struct HypertableDataNode -{ - FormData_hypertable_data_node fd; - Oid foreign_server_oid; -} HypertableDataNode; - -extern TSDLLEXPORT List *ts_hypertable_data_node_scan(int32 hypertable_id, MemoryContext mctx); -extern TSDLLEXPORT List *ts_hypertable_data_node_scan_by_node_name(const char *node_name, - MemoryContext mctx); -extern TSDLLEXPORT int ts_hypertable_data_node_delete_by_hypertable_id(int32 hypertable_id); -extern TSDLLEXPORT int ts_hypertable_data_node_delete_by_node_name(const char *node_name); -extern TSDLLEXPORT void ts_hypertable_data_node_insert_multi(List *hypertable_data_nodes); -extern TSDLLEXPORT int -ts_hypertable_data_node_delete_by_node_name_and_hypertable_id(const char *node_name, - int32 hypertable_id); -extern TSDLLEXPORT int -ts_hypertable_data_node_update(const HypertableDataNode *hypertable_data_node); diff --git a/src/ts_catalog/tablespace.c b/src/ts_catalog/tablespace.c index e227c35727f..cf44dec9d19 100644 --- a/src/ts_catalog/tablespace.c +++ b/src/ts_catalog/tablespace.c @@ -566,11 +566,6 @@ ts_tablespace_attach_internal(Name tspcname, Oid hypertable_oid, bool if_not_att } ht = ts_hypertable_cache_get_cache_and_entry(hypertable_oid, CACHE_FLAG_NONE, &hcache); - if (hypertable_is_distributed(ht)) - ereport(ERROR, - (errcode(ERRCODE_TS_OPERATION_NOT_SUPPORTED), - errmsg("cannot attach tablespace to distributed hypertable"))); - if (ts_hypertable_has_tablespace(ht, tspc_oid)) { if (if_not_attached) diff --git a/tsl/src/CMakeLists.txt b/tsl/src/CMakeLists.txt index de071b3aefe..887354dc112 100644 --- a/tsl/src/CMakeLists.txt +++ b/tsl/src/CMakeLists.txt @@ -1,11 +1,6 @@ set(SOURCES chunk_api.c chunk.c - data_node.c - deparse.c - dist_util.c - dist_backup.c - hypertable.c init.c partialize_agg.c partialize_finalize.c @@ -58,4 +53,3 @@ add_subdirectory(bgw_policy) add_subdirectory(compression) add_subdirectory(continuous_aggs) add_subdirectory(nodes) -add_subdirectory(remote) diff --git a/tsl/src/bgw_policy/job.c b/tsl/src/bgw_policy/job.c index 6035a653ab9..1c93ea51535 100644 --- a/tsl/src/bgw_policy/job.c +++ b/tsl/src/bgw_policy/job.c @@ -427,62 +427,6 @@ policy_refresh_cagg_read_and_validate_config(Jsonb *config, PolicyContinuousAggD } } -/* - * Invoke recompress_chunk via fmgr so that the call can be deparsed and sent to - * remote data nodes. - */ -static void -policy_invoke_recompress_chunk(Chunk *chunk) -{ - EState *estate; - ExprContext *econtext; - FuncExpr *fexpr; - Oid relid = chunk->table_id; - Oid restype; - Oid func_oid; - List *args = NIL; - bool isnull; - Const *argarr[RECOMPRESS_CHUNK_NARGS] = { - makeConst(REGCLASSOID, - -1, - InvalidOid, - sizeof(relid), - ObjectIdGetDatum(relid), - false, - false), - castNode(Const, makeBoolConst(true, false)), - }; - Oid type_id[RECOMPRESS_CHUNK_NARGS] = { REGCLASSOID, BOOLOID }; - char *schema_name = ts_extension_schema_name(); - List *fqn = list_make2(makeString(schema_name), makeString(RECOMPRESS_CHUNK_FUNCNAME)); - - StaticAssertStmt(lengthof(type_id) == lengthof(argarr), - "argarr and type_id should have matching lengths"); - - func_oid = LookupFuncName(fqn, lengthof(type_id), type_id, false); - Assert(func_oid); /* LookupFuncName should not return an invalid OID */ - - /* Prepare the function expr with argument list */ - get_func_result_type(func_oid, &restype, NULL); - - for (size_t i = 0; i < lengthof(argarr); i++) - args = lappend(args, argarr[i]); - - fexpr = makeFuncExpr(func_oid, restype, args, InvalidOid, InvalidOid, COERCE_EXPLICIT_CALL); - fexpr->funcretset = false; - - estate = CreateExecutorState(); - econtext = CreateExprContext(estate); - - ExprState *exprstate = ExecInitExpr(&fexpr->xpr, NULL); - - ExecEvalExprSwitchContext(exprstate, econtext, &isnull); - - /* Cleanup */ - FreeExprContext(econtext, false); - FreeExecutorState(estate); -} - /* Read configuration for compression job from config object. */ void policy_compression_read_and_validate_config(Jsonb *config, PolicyCompressionData *policy_data) @@ -521,12 +465,11 @@ policy_recompression_execute(int32 job_id, Jsonb *config) ListCell *lc; const Dimension *dim; PolicyCompressionData policy_data; - bool distributed, used_portalcxt = false; + bool used_portalcxt = false; MemoryContext saved_cxt, multitxn_cxt; policy_recompression_read_and_validate_config(config, &policy_data); dim = hyperspace_get_open_dimension(policy_data.hypertable->space, 0); - distributed = hypertable_is_distributed(policy_data.hypertable); /* we want the chunk id list to survive across transactions. So alloc in * a different context */ @@ -570,10 +513,7 @@ policy_recompression_execute(int32 job_id, Jsonb *config) Chunk *chunk = ts_chunk_get_by_id(chunkid, true); if (!chunk || !ts_chunk_is_unordered(chunk)) continue; - if (distributed) - policy_invoke_recompress_chunk(chunk); - else - tsl_recompress_chunk_wrapper(chunk); + tsl_recompress_chunk_wrapper(chunk); elog(LOG, "completed recompressing chunk \"%s.%s\"", diff --git a/tsl/src/bgw_policy/reorder_api.c b/tsl/src/bgw_policy/reorder_api.c index 7b748a480de..65636635d81 100644 --- a/tsl/src/bgw_policy/reorder_api.c +++ b/tsl/src/bgw_policy/reorder_api.c @@ -184,11 +184,6 @@ policy_reorder_add(PG_FUNCTION_ARGS) errhint("Please add the policy to the corresponding uncompressed hypertable " "instead."))); - if (hypertable_is_distributed(ht)) - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("reorder policies not supported on a distributed hypertables"))); - /* Now verify that the index is an actual index on that hypertable */ check_valid_index(ht, index_name); diff --git a/tsl/src/chunk.c b/tsl/src/chunk.c index 15af2efe402..264393c80d3 100644 --- a/tsl/src/chunk.c +++ b/tsl/src/chunk.c @@ -42,11 +42,7 @@ #include "chunk.h" #include "chunk_api.h" -#include "data_node.h" -#include "deparse.h" #include "debug_point.h" -#include "dist_util.h" -#include "ts_catalog/chunk_data_node.h" #include "utils.h" /* Data in a frozen chunk cannot be modified. So any operation diff --git a/tsl/src/chunk_api.c b/tsl/src/chunk_api.c index 246cfd919ad..c291e739cc6 100644 --- a/tsl/src/chunk_api.c +++ b/tsl/src/chunk_api.c @@ -31,17 +31,14 @@ #include "compat/compat.h" #include "chunk.h" #include "chunk_api.h" -#include "data_node.h" #include "errors.h" #include "error_utils.h" #include "hypercube.h" #include "hypertable_cache.h" #include "utils.h" -#include "deparse.h" #include "ts_catalog/array_utils.h" #include "ts_catalog/catalog.h" -#include "ts_catalog/chunk_data_node.h" /* * These values come from the pg_type table. diff --git a/tsl/src/chunk_api.h b/tsl/src/chunk_api.h index 9edb08de979..f1a240317ca 100644 --- a/tsl/src/chunk_api.h +++ b/tsl/src/chunk_api.h @@ -10,18 +10,9 @@ #include #include "chunk.h" -#include "ts_catalog/hypertable_data_node.h" - extern Datum chunk_status(PG_FUNCTION_ARGS); extern Datum chunk_show(PG_FUNCTION_ARGS); extern Datum chunk_create(PG_FUNCTION_ARGS); -extern void chunk_api_create_on_data_nodes(const Chunk *chunk, const Hypertable *ht, - const char *remote_chunk_name, List *data_nodes); extern Datum chunk_api_get_chunk_relstats(PG_FUNCTION_ARGS); extern Datum chunk_api_get_chunk_colstats(PG_FUNCTION_ARGS); -extern void chunk_api_update_distributed_hypertable_stats(Oid table_id); extern Datum chunk_create_empty_table(PG_FUNCTION_ARGS); -extern void chunk_api_call_create_empty_chunk_table(const Hypertable *ht, const Chunk *chunk, - const char *node_name); -extern void chunk_api_call_chunk_drop_replica(const Chunk *chunk, const char *node_name, - Oid serverid); diff --git a/tsl/src/continuous_aggs/create.c b/tsl/src/continuous_aggs/create.c index 93ea8a97236..5cf85493d30 100644 --- a/tsl/src/continuous_aggs/create.c +++ b/tsl/src/continuous_aggs/create.c @@ -62,7 +62,6 @@ #include "ts_catalog/catalog.h" #include "ts_catalog/continuous_agg.h" #include "ts_catalog/continuous_aggs_watermark.h" -#include "ts_catalog/hypertable_data_node.h" #include "dimension.h" #include "extension_constants.h" #include "func_cache.h" @@ -75,8 +74,6 @@ #include "utils.h" #include "errors.h" #include "refresh.h" -#include "remote/dist_commands.h" -#include "deparse.h" #include "timezones.h" #include "guc.h" @@ -348,43 +345,6 @@ cagg_add_trigger_hypertable(Oid relid, int32 hypertable_id) #endif ht = ts_hypertable_cache_get_cache_and_entry(relid, CACHE_FLAG_NONE, &hcache); - if (hypertable_is_distributed(ht)) - { - DistCmdResult *result; - List *data_node_list = ts_hypertable_get_data_node_name_list(ht); - List *cmd_descriptors = NIL; /* same order as ht->data_nodes */ - DistCmdDescr *cmd_descr_data = NULL; - ListCell *cell; - - unsigned i = 0; - cmd_descr_data = palloc(list_length(data_node_list) * sizeof(*cmd_descr_data)); - foreach (cell, ht->data_nodes) - { - HypertableDataNode *node = lfirst(cell); - char node_hypertable_id_str[12]; - CreateTrigStmt remote_stmt = stmt_template; - - pg_ltoa(node->fd.node_hypertable_id, node_hypertable_id_str); - pg_ltoa(node->fd.hypertable_id, hypertable_id_str); - - remote_stmt.args = - list_make2(makeString(node_hypertable_id_str), makeString(hypertable_id_str)); - cmd_descr_data[i].sql = deparse_create_trigger(&remote_stmt); - cmd_descr_data[i].params = NULL; - cmd_descriptors = lappend(cmd_descriptors, &cmd_descr_data[i++]); - } - - result = - ts_dist_multi_cmds_params_invoke_on_data_nodes(cmd_descriptors, data_node_list, true); - if (result) - ts_dist_cmd_close_response(result); - /* - * FALL-THROUGH - * We let the Access Node create a trigger as well, even though it is not used for data - * modifications. We use the Access Node trigger as a check for existence of the remote - * triggers. - */ - } CreateTrigStmt local_stmt = stmt_template; pg_ltoa(hypertable_id, hypertable_id_str); local_stmt.args = list_make1(makeString(hypertable_id_str)); diff --git a/tsl/src/continuous_aggs/insert.c b/tsl/src/continuous_aggs/insert.c index ee89b0bc8e7..ca90b58efbf 100644 --- a/tsl/src/continuous_aggs/insert.c +++ b/tsl/src/continuous_aggs/insert.c @@ -326,11 +326,6 @@ cache_inval_entry_write(ContinuousAggsCacheInvalEntry *entry) if (!entry->value_is_set) return; - Cache *ht_cache = ts_hypertable_cache_pin(); - Hypertable *ht = ts_hypertable_cache_get_entry_by_id(ht_cache, entry->hypertable_id); - bool is_distributed_member = hypertable_is_distributed_member(ht); - ts_cache_release(ht_cache); - /* The materialization worker uses a READ COMMITTED isolation level by default. Therefore, if we * use a stronger isolation level, the isolation thereshold could update without us seeing the * new value. In order to prevent serialization errors, we always append invalidation entries in @@ -338,7 +333,7 @@ cache_inval_entry_write(ContinuousAggsCacheInvalEntry *entry) * threshold. The same applies for distributed member invalidation triggers of hypertables. * The materializer can handle invalidations that are beyond the threshold gracefully. */ - if (IsolationUsesXactSnapshot() || is_distributed_member) + if (IsolationUsesXactSnapshot()) { invalidation_hyper_log_add_entry(entry->entry_id, entry->lowest_modified_value, diff --git a/tsl/src/continuous_aggs/invalidation.c b/tsl/src/continuous_aggs/invalidation.c index b7fe702fbe1..98bc5574e03 100644 --- a/tsl/src/continuous_aggs/invalidation.c +++ b/tsl/src/continuous_aggs/invalidation.c @@ -29,12 +29,9 @@ #include #include "compat/compat.h" -#include "remote/dist_commands.h" #include "ts_catalog/catalog.h" #include "ts_catalog/continuous_agg.h" #include "continuous_aggs/materialize.h" -#include "data_node.h" -#include "deparse.h" #include "invalidation.h" #include "refresh.h" @@ -271,54 +268,6 @@ tsl_invalidation_hyper_log_add_entry(PG_FUNCTION_ARGS) #define INVALIDATION_CAGG_LOG_ADD_ENTRY_FUNCNAME "invalidation_cagg_log_add_entry" #define INVALIDATION_HYPER_LOG_ADD_ENTRY_FUNCNAME "invalidation_hyper_log_add_entry" -void -remote_invalidation_log_add_entry(const Hypertable *raw_ht, - ContinuousAggHypertableStatus caggstatus, int32 entry_id, - int64 start, int64 end) -{ - Oid func_oid; - LOCAL_FCINFO(fcinfo, INVALIDATION_CAGG_ADD_ENTRY_NARGS); - FmgrInfo flinfo; - - Assert(HypertableIsMaterialization == caggstatus || HypertableIsRawTable == caggstatus); - - static const Oid type_id[INVALIDATION_CAGG_ADD_ENTRY_NARGS] = { INT4OID, INT8OID, INT8OID }; - List *const fqn = list_make2(makeString(FUNCTIONS_SCHEMA_NAME), - makeString((caggstatus == HypertableIsMaterialization) ? - INVALIDATION_CAGG_LOG_ADD_ENTRY_FUNCNAME : - INVALIDATION_HYPER_LOG_ADD_ENTRY_FUNCNAME)); - - if (!hypertable_is_distributed(raw_ht)) - elog(ERROR, "function was not provided with a valid distributed hypertable"); - - func_oid = LookupFuncName(fqn, -1 /* lengthof(type_id) */, type_id, false); - Assert(OidIsValid(func_oid)); - - fmgr_info(func_oid, &flinfo); - InitFunctionCallInfoData(*fcinfo, - &flinfo, - INVALIDATION_CAGG_ADD_ENTRY_NARGS, - InvalidOid, - NULL, - NULL); - - FC_NULL(fcinfo, 0) = false; - FC_ARG(fcinfo, 0) = Int32GetDatum(entry_id); - FC_NULL(fcinfo, 1) = false; - FC_ARG(fcinfo, 1) = Int64GetDatum(start); - FC_NULL(fcinfo, 2) = false; - FC_ARG(fcinfo, 2) = Int64GetDatum(end); - /* Check for null result, since caller is clearly not expecting one */ - if (fcinfo->isnull) - elog(ERROR, "function %u returned NULL", flinfo.fn_oid); - - DistCmdResult *result; - List *data_node_list = ts_hypertable_get_data_node_name_list(raw_ht); - result = ts_dist_cmd_invoke_func_call_on_data_nodes(fcinfo, data_node_list); - if (result) - ts_dist_cmd_close_response(result); -} - void invalidation_hyper_log_add_entry(int32 hyper_id, int64 start, int64 end) { @@ -366,14 +315,7 @@ continuous_agg_invalidate_raw_ht(const Hypertable *raw_ht, int64 start, int64 en { Assert(raw_ht != NULL); - if (hypertable_is_distributed(raw_ht)) - { - remote_invalidation_log_add_entry(raw_ht, HypertableIsRawTable, raw_ht->fd.id, start, end); - } - else - { - invalidation_hyper_log_add_entry(raw_ht->fd.id, start, end); - } + invalidation_hyper_log_add_entry(raw_ht->fd.id, start, end); } void @@ -382,18 +324,7 @@ continuous_agg_invalidate_mat_ht(const Hypertable *raw_ht, const Hypertable *mat { Assert((raw_ht != NULL) && (mat_ht != NULL)); - if (hypertable_is_distributed(raw_ht)) - { - remote_invalidation_log_add_entry(raw_ht, - HypertableIsMaterialization, - mat_ht->fd.id, - start, - end); - } - else - { - invalidation_cagg_log_add_entry(mat_ht->fd.id, start, end); - } + invalidation_cagg_log_add_entry(mat_ht->fd.id, start, end); } typedef enum InvalidationResult @@ -1201,70 +1132,6 @@ tsl_invalidation_process_hypertable_log(PG_FUNCTION_ARGS) #define INVALIDATION_PROCESS_HYPERTABLE_LOG_NARGS 7 #define INVALIDATION_PROCESS_HYPERTABLE_LOG_FUNCNAME "invalidation_process_hypertable_log" -void -remote_invalidation_process_hypertable_log(int32 mat_hypertable_id, int32 raw_hypertable_id, - Oid dimtype, const CaggsInfo *all_caggs) -{ - Oid func_oid; - ArrayType *mat_hypertable_ids; - ArrayType *bucket_widths; - ArrayType *bucket_functions; - LOCAL_FCINFO(fcinfo, INVALIDATION_PROCESS_HYPERTABLE_LOG_NARGS); - FmgrInfo flinfo; - unsigned int i; - - ts_create_arrays_from_caggs_info(all_caggs, - &mat_hypertable_ids, - &bucket_widths, - &bucket_functions); - - static const Oid type_id[INVALIDATION_PROCESS_HYPERTABLE_LOG_NARGS] = { - INT4OID, INT4OID, REGTYPEOID, INT4ARRAYOID, INT8ARRAYOID, INT8ARRAYOID, TEXTARRAYOID - }; - List *const fqn = list_make2(makeString(FUNCTIONS_SCHEMA_NAME), - makeString(INVALIDATION_PROCESS_HYPERTABLE_LOG_FUNCNAME)); - - /* - * Note that we have to explicitly specify the amount of arguments in this - * case, because there are several overloaded versions of invalidation_process_hypertable_log(). - */ - func_oid = LookupFuncName(fqn, lengthof(type_id), type_id, false); - Assert(OidIsValid(func_oid)); - - fmgr_info(func_oid, &flinfo); - InitFunctionCallInfoData(*fcinfo, - &flinfo, - INVALIDATION_PROCESS_HYPERTABLE_LOG_NARGS, - InvalidOid, - NULL, - NULL); - - for (i = 0; i < INVALIDATION_PROCESS_HYPERTABLE_LOG_NARGS; ++i) - { - FC_NULL(fcinfo, i) = false; - } - FC_ARG(fcinfo, 0) = Int32GetDatum(mat_hypertable_id); - FC_ARG(fcinfo, 1) = Int32GetDatum(raw_hypertable_id); - FC_ARG(fcinfo, 2) = ObjectIdGetDatum(dimtype); - FC_ARG(fcinfo, 3) = PointerGetDatum(mat_hypertable_ids); - FC_ARG(fcinfo, 4) = PointerGetDatum(bucket_widths); - FC_ARG(fcinfo, 5) = PointerGetDatum(construct_empty_array(INT8OID)); - FC_ARG(fcinfo, 6) = PointerGetDatum(bucket_functions); - /* Check for null result, since caller is clearly not expecting one */ - if (fcinfo->isnull) - elog(ERROR, "function %u returned NULL", flinfo.fn_oid); - - Hypertable *ht = ts_hypertable_get_by_id(raw_hypertable_id); - if (!ht || !hypertable_is_distributed(ht)) - elog(ERROR, "function was not provided with a valid distributed hypertable id"); - - DistCmdResult *result; - List *data_node_list = ts_hypertable_get_data_node_name_list(ht); - result = ts_dist_cmd_invoke_func_call_on_data_nodes(fcinfo, data_node_list); - if (result) - ts_dist_cmd_close_response(result); -} - InvalidationStore * invalidation_process_cagg_log(int32 mat_hypertable_id, int32 raw_hypertable_id, const InternalTimeRange *refresh_window, @@ -1404,265 +1271,6 @@ tsl_invalidation_process_cagg_log(PG_FUNCTION_ARGS) PG_RETURN_DATUM(HeapTupleGetDatum(tuple)); } -#define INVALIDATION_PROCESS_CAGG_LOG_NARGS 9 -#define INVALIDATION_PROCESS_CAGG_LOG_FUNCNAME "invalidation_process_cagg_log" - -void -remote_invalidation_process_cagg_log(int32 mat_hypertable_id, int32 raw_hypertable_id, - const InternalTimeRange *refresh_window, - const CaggsInfo *all_caggs, bool *do_merged_refresh, - InternalTimeRange *ret_merged_refresh_window) -{ - Oid func_oid; - ArrayType *mat_hypertable_ids; - ArrayType *bucket_widths; - ArrayType *bucket_functions; - LOCAL_FCINFO(fcinfo, INVALIDATION_PROCESS_CAGG_LOG_NARGS); - FmgrInfo flinfo; - unsigned int i; - - *do_merged_refresh = false; - - ts_create_arrays_from_caggs_info(all_caggs, - &mat_hypertable_ids, - &bucket_widths, - &bucket_functions); - - static const Oid type_id[INVALIDATION_PROCESS_CAGG_LOG_NARGS] = { - INT4OID, INT4OID, REGTYPEOID, INT8OID, INT8OID, - INT4ARRAYOID, INT8ARRAYOID, INT8ARRAYOID, TEXTARRAYOID, - }; - List *const fqn = list_make2(makeString(FUNCTIONS_SCHEMA_NAME), - makeString(INVALIDATION_PROCESS_CAGG_LOG_FUNCNAME)); - - /* - * Note that we have to explicitly specify the amount of arguments in this - * case, because there are several overloaded versions of invalidation_process_cagg_log(). - */ - func_oid = LookupFuncName(fqn, lengthof(type_id), type_id, false); - Assert(OidIsValid(func_oid)); - - fmgr_info(func_oid, &flinfo); - InitFunctionCallInfoData(*fcinfo, - &flinfo, - INVALIDATION_PROCESS_CAGG_LOG_NARGS, - InvalidOid, - NULL, - NULL); - - for (i = 0; i < INVALIDATION_PROCESS_CAGG_LOG_NARGS; ++i) - { - FC_NULL(fcinfo, i) = false; - } - FC_ARG(fcinfo, 0) = Int32GetDatum(mat_hypertable_id); - FC_ARG(fcinfo, 1) = Int32GetDatum(raw_hypertable_id); - FC_ARG(fcinfo, 2) = ObjectIdGetDatum(refresh_window->type); - FC_ARG(fcinfo, 3) = Int64GetDatum(refresh_window->start); - FC_ARG(fcinfo, 4) = Int64GetDatum(refresh_window->end); - FC_ARG(fcinfo, 5) = PointerGetDatum(mat_hypertable_ids); - FC_ARG(fcinfo, 6) = PointerGetDatum(bucket_widths); - FC_ARG(fcinfo, 7) = PointerGetDatum(construct_empty_array(INT8OID)); - FC_ARG(fcinfo, 8) = PointerGetDatum(bucket_functions); - /* Check for null result, since caller is clearly not expecting one */ - if (fcinfo->isnull) - elog(ERROR, "function %u returned NULL", flinfo.fn_oid); - - Hypertable *ht = ts_hypertable_get_by_id(raw_hypertable_id); - if (!ht || !hypertable_is_distributed(ht)) - elog(ERROR, "function was not provided with a valid distributed hypertable id"); - - DistCmdResult *dist_res; - List *data_node_list = ts_hypertable_get_data_node_name_list(ht); - dist_res = ts_dist_cmd_invoke_func_call_on_data_nodes(fcinfo, data_node_list); - if (dist_res) - { - unsigned num_dist_res = ts_dist_cmd_response_count(dist_res); - int64 start_time, end_time; - InternalTimeRange merged_window = { - .type = refresh_window->type, - .start = TS_TIME_NOEND, /* initial state invalid */ - .end = TS_TIME_NOBEGIN /* initial state invalid */ - }; - - for (i = 0; i < num_dist_res; ++i) - { - const char *node_name; - PGresult *result = ts_dist_cmd_get_result_by_index(dist_res, i, &node_name); - if (PQresultStatus(result) != PGRES_TUPLES_OK) - ereport(ERROR, - (errcode(ERRCODE_CONNECTION_EXCEPTION), - errmsg("%s", PQresultErrorMessage(result)))); - Assert(PQntuples(result) == 1); - Assert(PQnfields(result) == 2); - if (PQgetisnull(result, 0, 0)) - { /* No invalidations in this data node */ - Assert(PQgetisnull(result, 0, 1)); - continue; - } - start_time = pg_strtoint64(PQgetvalue(result, 0, 0)); - end_time = pg_strtoint64(PQgetvalue(result, 0, 1)); - elog(DEBUG1, - "merged invalidations for refresh on [" INT64_FORMAT ", " INT64_FORMAT "] from %s", - start_time, - end_time, - node_name); - - /* merge refresh windows from the data nodes */ - if (start_time < merged_window.start) - merged_window.start = start_time; - if (end_time > merged_window.end) - merged_window.end = end_time; - } - ts_dist_cmd_close_response(dist_res); - - if (merged_window.start <= merged_window.end) - { - *ret_merged_refresh_window = merged_window; - *do_merged_refresh = true; - } - } -} - -#define INVALIDATION_LOG_DELETE_NARGS 1 -#define HYPERTABLE_INVALIDATION_LOG_DELETE_FUNCNAME "hypertable_invalidation_log_delete" -#define MATERIALIZATION_INVALIDATION_LOG_DELETE_FUNCNAME "materialization_invalidation_log_delete" - -void -remote_invalidation_log_delete(int32 raw_hypertable_id, ContinuousAggHypertableStatus caggstatus) -{ - /* Execute on all data nodes if there are any */ - List *data_nodes = data_node_get_node_name_list(); - if (NIL == data_nodes) - return; - - Oid func_oid; - LOCAL_FCINFO(fcinfo, INVALIDATION_LOG_DELETE_NARGS); - FmgrInfo flinfo; - - Assert(HypertableIsMaterialization == caggstatus || HypertableIsRawTable == caggstatus); - - static const Oid type_id[INVALIDATION_LOG_DELETE_NARGS] = { INT4OID }; - List *const fqn = list_make2(makeString(FUNCTIONS_SCHEMA_NAME), - makeString((caggstatus == HypertableIsMaterialization) ? - MATERIALIZATION_INVALIDATION_LOG_DELETE_FUNCNAME : - HYPERTABLE_INVALIDATION_LOG_DELETE_FUNCNAME)); - - func_oid = LookupFuncName(fqn, -1 /* lengthof(type_id) */, type_id, false); - Assert(OidIsValid(func_oid)); - - fmgr_info(func_oid, &flinfo); - InitFunctionCallInfoData(*fcinfo, - &flinfo, - INVALIDATION_LOG_DELETE_NARGS, - InvalidOid, - NULL, - NULL); - - FC_NULL(fcinfo, 0) = false; - FC_ARG(fcinfo, 0) = Int32GetDatum(raw_hypertable_id); - /* Check for null result, since caller is clearly not expecting one */ - if (fcinfo->isnull) - elog(ERROR, "function %u returned NULL", flinfo.fn_oid); - - DistCmdResult *result; - result = ts_dist_cmd_invoke_func_call_on_data_nodes(fcinfo, data_nodes); - if (result) - ts_dist_cmd_close_response(result); -} - -/** - * Delete invalidation trigger for distributed hypertable member with hypertable ID - * 'raw_hypertable_id' in the Data Node. - * - * @param raw_hypertable_id - The hypertable ID of the distributed hypertable member in the - * Data Node. - */ -Datum -tsl_drop_dist_ht_invalidation_trigger(PG_FUNCTION_ARGS) -{ - Cache *hcache; - Hypertable *ht; - int32 raw_hypertable_id = PG_GETARG_INT32(0); - - hcache = ts_hypertable_cache_pin(); - ht = ts_hypertable_cache_get_entry_by_id(hcache, raw_hypertable_id); - if (!ht || !hypertable_is_distributed_member(ht)) - elog(ERROR, "function was not provided with a valid distributed hypertable id"); - - ts_materialization_invalidation_log_delete_inner(raw_hypertable_id); - ts_hypertable_drop_trigger(ht->main_table_relid, CAGGINVAL_TRIGGER_NAME); - - ts_cache_release(hcache); - - PG_RETURN_VOID(); -} - -#define DROP_DIST_HT_INVALIDATION_TRIGGER_NARGS 1 -#define DROP_DIST_HT_INVALIDATION_TRIGGER_FUNCNAME "drop_dist_ht_invalidation_trigger" - -void -remote_drop_dist_ht_invalidation_trigger(int32 raw_hypertable_id) -{ - Cache *hcache; - Hypertable *ht; - - hcache = ts_hypertable_cache_pin(); - ht = ts_hypertable_cache_get_entry_by_id(hcache, raw_hypertable_id); - Assert(ht); - if (!hypertable_is_distributed(ht)) - { - ts_cache_release(hcache); - return; - } - List *data_node_list = ts_hypertable_get_data_node_name_list(ht); - DistCmdResult *result; - ListCell *cell; - Oid func_oid; - - static const Oid type_id[DROP_DIST_HT_INVALIDATION_TRIGGER_NARGS] = { INT4OID }; - List *const fqn = list_make2(makeString(FUNCTIONS_SCHEMA_NAME), - makeString(DROP_DIST_HT_INVALIDATION_TRIGGER_FUNCNAME)); - - func_oid = LookupFuncName(fqn, -1 /* lengthof(type_id) */, type_id, false); - Assert(OidIsValid(func_oid)); - - FunctionCallInfo fcinfo = - palloc(SizeForFunctionCallInfo(DROP_DIST_HT_INVALIDATION_TRIGGER_NARGS)); - FmgrInfo flinfo; - List *cmd_descriptors = NIL; /* same order as ht->data_nodes */ - DistCmdDescr *cmd_descr_data = palloc(list_length(data_node_list) * sizeof(*cmd_descr_data)); - unsigned i = 0; - - foreach (cell, ht->data_nodes) - { - HypertableDataNode *node = lfirst(cell); - - fmgr_info(func_oid, &flinfo); - InitFunctionCallInfoData(*fcinfo, - &flinfo, - DROP_DIST_HT_INVALIDATION_TRIGGER_NARGS, - InvalidOid, - NULL, - NULL); - - FC_NULL(fcinfo, 0) = false; - /* distributed member hypertable ID */ - FC_ARG(fcinfo, 0) = Int32GetDatum(node->fd.node_hypertable_id); - /* Check for null result, since caller is clearly not expecting one */ - if (fcinfo->isnull) - elog(ERROR, "function %u returned NULL", flinfo.fn_oid); - - cmd_descr_data[i].sql = deparse_func_call(fcinfo); - cmd_descr_data[i].params = NULL; - cmd_descriptors = lappend(cmd_descriptors, &cmd_descr_data[i++]); - } - - result = ts_dist_multi_cmds_params_invoke_on_data_nodes(cmd_descriptors, data_node_list, true); - if (result) - ts_dist_cmd_close_response(result); - ts_cache_release(hcache); -} - void invalidation_store_free(InvalidationStore *store) { diff --git a/tsl/src/continuous_aggs/refresh.c b/tsl/src/continuous_aggs/refresh.c index a4b02c662cc..7b35761a365 100644 --- a/tsl/src/continuous_aggs/refresh.c +++ b/tsl/src/continuous_aggs/refresh.c @@ -694,36 +694,16 @@ process_cagg_invalidations_and_refresh(const ContinuousAgg *cagg, * windows. */ LockRelationOid(hyper_relid, ExclusiveLock); - Hypertable *ht = cagg_get_hypertable_or_fail(cagg->data.raw_hypertable_id); - bool is_raw_ht_distributed = hypertable_is_distributed(ht); const CaggsInfo all_caggs_info = ts_continuous_agg_get_all_caggs_info(cagg->data.raw_hypertable_id); max_materializations = materialization_per_refresh_window(); - if (is_raw_ht_distributed) - { - invalidations = NULL; - /* Force to always merge the refresh ranges in the distributed raw HyperTable case. - * Session variable MATERIALIZATIONS_PER_REFRESH_WINDOW_OPT_NAME was checked for - * validity in materialization_per_refresh_window(). - */ - max_materializations = 0; - remote_invalidation_process_cagg_log(cagg->data.mat_hypertable_id, - cagg->data.raw_hypertable_id, - refresh_window, - &all_caggs_info, - &do_merged_refresh, - &merged_refresh_window); - } - else - { - invalidations = invalidation_process_cagg_log(cagg->data.mat_hypertable_id, - cagg->data.raw_hypertable_id, - refresh_window, - &all_caggs_info, - max_materializations, - &do_merged_refresh, - &merged_refresh_window); - } + invalidations = invalidation_process_cagg_log(cagg->data.mat_hypertable_id, + cagg->data.raw_hypertable_id, + refresh_window, + &all_caggs_info, + max_materializations, + &do_merged_refresh, + &merged_refresh_window); if (invalidations != NULL || do_merged_refresh) { @@ -745,7 +725,7 @@ process_cagg_invalidations_and_refresh(const ContinuousAgg *cagg, invalidations, bucket_width, chunk_id, - is_raw_ht_distributed, + false, do_merged_refresh, merged_refresh_window); if (invalidations) @@ -765,7 +745,6 @@ continuous_agg_refresh_internal(const ContinuousAgg *cagg, int32 mat_id = cagg->data.mat_hypertable_id; InternalTimeRange refresh_window = *refresh_window_arg; int64 invalidation_threshold; - bool is_raw_ht_distributed; int rc; /* Connect to SPI manager due to the underlying SPI calls */ @@ -794,9 +773,6 @@ continuous_agg_refresh_internal(const ContinuousAgg *cagg, * prevent transaction blocks. */ PreventInTransactionBlock(true, REFRESH_FUNCTION_NAME); - Hypertable *ht = cagg_get_hypertable_or_fail(cagg->data.raw_hypertable_id); - is_raw_ht_distributed = hypertable_is_distributed(ht); - /* No bucketing when open ended */ if (!(start_isnull && end_isnull)) { @@ -872,20 +848,10 @@ continuous_agg_refresh_internal(const ContinuousAgg *cagg, /* Process invalidations in the hypertable invalidation log */ const CaggsInfo all_caggs_info = ts_continuous_agg_get_all_caggs_info(cagg->data.raw_hypertable_id); - if (is_raw_ht_distributed) - { - remote_invalidation_process_hypertable_log(cagg->data.mat_hypertable_id, - cagg->data.raw_hypertable_id, - refresh_window.type, - &all_caggs_info); - } - else - { - invalidation_process_hypertable_log(cagg->data.mat_hypertable_id, - cagg->data.raw_hypertable_id, - refresh_window.type, - &all_caggs_info); - } + invalidation_process_hypertable_log(cagg->data.mat_hypertable_id, + cagg->data.raw_hypertable_id, + refresh_window.type, + &all_caggs_info); /* Commit and Start a new transaction */ SPI_commit_and_chain(); diff --git a/tsl/src/data_node.c b/tsl/src/data_node.c deleted file mode 100644 index a6c71d2d2c6..00000000000 --- a/tsl/src/data_node.c +++ /dev/null @@ -1,2157 +0,0 @@ -/* - * 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. - */ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include "compat/compat.h" -#include "config.h" -#include "extension.h" -#include "cache.h" -#include "chunk.h" -#include "remote/async.h" -#include "remote/connection.h" -#include "remote/connection_cache.h" -#include "remote/dist_commands.h" -#include "data_node.h" -#include "remote/utils.h" -#include "hypertable_cache.h" -#include "errors.h" -#include "dist_util.h" -#include "utils/uuid.h" -#include "mb/pg_wchar.h" -#include "scan_iterator.h" -#include "ts_catalog/catalog.h" -#include "ts_catalog/chunk_data_node.h" -#include "ts_catalog/dimension_partition.h" -#include "ts_catalog/hypertable_data_node.h" - -#define TS_DEFAULT_POSTGRES_PORT 5432 -#define TS_DEFAULT_POSTGRES_HOST "localhost" - -#define ERRCODE_DUPLICATE_DATABASE_STR "42P04" -#define ERRCODE_DUPLICATE_SCHEMA_STR "42P06" - -typedef struct DbInfo -{ - NameData name; - int32 encoding; - const char *chartype; - const char *collation; -} DbInfo; - -/* A list of databases we try to connect to when bootstrapping a data node */ -static const char *bootstrap_databases[] = { "postgres", "template1", "defaultdb" }; - -static bool data_node_validate_database(TSConnection *conn, const DbInfo *database); - -/* - * get_database_info - given a database OID, look up info about the database - * - * Returns: - * True if a record for the OID was found, false otherwise. - */ -static bool -get_database_info(Oid dbid, DbInfo *database) -{ - HeapTuple dbtuple; - Form_pg_database dbrecord; - - dbtuple = SearchSysCache1(DATABASEOID, ObjectIdGetDatum(dbid)); - - if (!HeapTupleIsValid(dbtuple)) - return false; - - dbrecord = (Form_pg_database) GETSTRUCT(dbtuple); - - database->encoding = dbrecord->encoding; - -#if PG15_LT - database->collation = NameStr(dbrecord->datcollate); - database->chartype = NameStr(dbrecord->datctype); -#else - /* - * Since datcollate and datctype are varlen fields in PG15+ we cannot rely - * on GETSTRUCT filling them in as GETSTRUCT only works for fixed-length - * non-NULLABLE columns. - */ - Datum datum; - bool isnull; - - datum = SysCacheGetAttr(DATABASEOID, dbtuple, Anum_pg_database_datcollate, &isnull); - Assert(!isnull); - database->collation = TextDatumGetCString(datum); - - datum = SysCacheGetAttr(DATABASEOID, dbtuple, Anum_pg_database_datctype, &isnull); - Assert(!isnull); - database->chartype = TextDatumGetCString(datum); -#endif - - database->collation = pstrdup(database->collation); - database->chartype = pstrdup(database->chartype); - - ReleaseSysCache(dbtuple); - return true; -} - -/* - * Verify that server is TimescaleDB data node and perform optional ACL check. - * - * The function returns true if the server is valid TimescaleDB data node and - * the ACL check succeeds. Otherwise, false is returned, or, an error is thrown - * if fail_on_aclcheck is set to true. - */ -static bool -validate_foreign_server(const ForeignServer *server, AclMode const mode, bool fail_on_aclcheck) -{ - Oid const fdwid = get_foreign_data_wrapper_oid(EXTENSION_FDW_NAME, false); - Oid curuserid = GetUserId(); - AclResult aclresult; - bool valid; - - Assert(NULL != server); - if (server->fdwid != fdwid) - ereport(ERROR, - (errcode(ERRCODE_INVALID_PARAMETER_VALUE), - errmsg("data node \"%s\" is not a TimescaleDB server", server->servername))); - - if (mode == ACL_NO_CHECK) - return true; - - /* Must have permissions on the server object */ - aclresult = object_aclcheck(ForeignServerRelationId, server->serverid, curuserid, mode); - - valid = (aclresult == ACLCHECK_OK); - - if (!valid && fail_on_aclcheck) - aclcheck_error(aclresult, OBJECT_FOREIGN_SERVER, server->servername); - - return valid; -} - -/* - * Lookup the foreign server by name - */ -ForeignServer * -data_node_get_foreign_server(const char *node_name, AclMode mode, bool fail_on_aclcheck, - bool missing_ok) -{ - ForeignServer *server; - bool valid; - - if (node_name == NULL) - ereport(ERROR, - (errcode(ERRCODE_INVALID_PARAMETER_VALUE), - errmsg("data node name cannot be NULL"))); - - server = GetForeignServerByName(node_name, missing_ok); - if (NULL == server) - return NULL; - - valid = validate_foreign_server(server, mode, fail_on_aclcheck); - - if (mode != ACL_NO_CHECK && !valid) - return NULL; - - return server; -} - -ForeignServer * -data_node_get_foreign_server_by_oid(Oid server_oid, AclMode mode) -{ - ForeignServer *server = GetForeignServer(server_oid); - bool PG_USED_FOR_ASSERTS_ONLY valid = validate_foreign_server(server, mode, true); - Assert(valid); /* Should always be valid since we should see error otherwise */ - return server; -} - -/* - * Create a foreign server. - * - * Returns true if the server was created and set the `oid` to the server oid. - */ -static bool -create_foreign_server(const char *const node_name, const char *const host, int32 port, - const char *const dbname, bool if_not_exists) -{ - ForeignServer *server = NULL; - ObjectAddress objaddr; - CreateForeignServerStmt stmt = { - .type = T_CreateForeignServerStmt, - .servername = (char *) node_name, - .fdwname = EXTENSION_FDW_NAME, - .options = list_make3(makeDefElem("host", (Node *) makeString(pstrdup(host)), -1), - makeDefElem("port", (Node *) makeInteger(port), -1), - makeDefElem("dbname", (Node *) makeString(pstrdup(dbname)), -1)), - .if_not_exists = if_not_exists, - }; - - if (NULL == host) - ereport(ERROR, - (errcode(ERRCODE_INVALID_PARAMETER_VALUE), - (errmsg("invalid host"), - (errhint("A hostname or IP address must be specified when " - "a data node does not already exist."))))); - - if (if_not_exists) - { - server = data_node_get_foreign_server(node_name, ACL_NO_CHECK, false, true); - - if (NULL != server) - { - ereport(NOTICE, - (errcode(ERRCODE_DUPLICATE_OBJECT), - errmsg("data node \"%s\" already exists, skipping", node_name))); - return false; - } - } - - /* Permissions checks done in CreateForeignServer() */ - objaddr = CreateForeignServer(&stmt); - - /* CreateForeignServer returns InvalidOid if server already exists */ - if (!OidIsValid(objaddr.objectId)) - { - Assert(if_not_exists); - return false; - } - - return true; -} - -TSConnection * -data_node_get_connection(const char *const data_node, RemoteTxnPrepStmtOption const ps_opt, - bool transactional) -{ - const ForeignServer *server; - TSConnectionId id; - - server = data_node_get_foreign_server(data_node, ACL_NO_CHECK, false, false); - id = remote_connection_id(server->serverid, GetUserId()); - - if (transactional) - return remote_dist_txn_get_connection(id, ps_opt); - - return remote_connection_cache_get_connection(id); -} - -/* Attribute numbers for datum returned by create_data_node() */ -enum Anum_create_data_node -{ - Anum_create_data_node_name = 1, - Anum_create_data_node_host, - Anum_create_data_node_port, - Anum_create_data_node_dbname, - Anum_create_data_node_node_created, - Anum_create_data_node_database_created, - Anum_create_data_node_extension_created, - _Anum_create_data_node_max, -}; - -#define Natts_create_data_node (_Anum_create_data_node_max - 1) - -static Datum -create_data_node_datum(FunctionCallInfo fcinfo, const char *node_name, const char *host, int32 port, - const char *dbname, bool node_created, bool database_created, - bool extension_created) -{ - TupleDesc tupdesc; - Datum values[_Anum_create_data_node_max]; - bool nulls[_Anum_create_data_node_max] = { false }; - HeapTuple tuple; - - if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE) - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("function returning record called in " - "context that cannot accept type record"))); - - /* - * Some columns are of Postgres type 'name' which is fixed-width 64-byte - * type. The input strings might not have the necessary padding after them, - * so we must pad them here. - */ - NameData node_namedata; - namestrcpy(&node_namedata, node_name); - NameData dbnamedata; - namestrcpy(&dbnamedata, dbname); - - tupdesc = BlessTupleDesc(tupdesc); - values[AttrNumberGetAttrOffset(Anum_create_data_node_name)] = NameGetDatum(&node_namedata); - values[AttrNumberGetAttrOffset(Anum_create_data_node_host)] = CStringGetTextDatum(host); - values[AttrNumberGetAttrOffset(Anum_create_data_node_port)] = Int32GetDatum(port); - values[AttrNumberGetAttrOffset(Anum_create_data_node_dbname)] = NameGetDatum(&dbnamedata); - values[AttrNumberGetAttrOffset(Anum_create_data_node_node_created)] = - BoolGetDatum(node_created); - values[AttrNumberGetAttrOffset(Anum_create_data_node_database_created)] = - BoolGetDatum(database_created); - values[AttrNumberGetAttrOffset(Anum_create_data_node_extension_created)] = - BoolGetDatum(extension_created); - tuple = heap_form_tuple(tupdesc, values, nulls); - - return HeapTupleGetDatum(tuple); -} - -static Datum -create_hypertable_data_node_datum(FunctionCallInfo fcinfo, HypertableDataNode *node) -{ - TupleDesc tupdesc; - Datum values[Natts_hypertable_data_node]; - bool nulls[Natts_hypertable_data_node] = { false }; - HeapTuple tuple; - - if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE) - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("function returning record called in " - "context that cannot accept type record"))); - - tupdesc = BlessTupleDesc(tupdesc); - values[AttrNumberGetAttrOffset(Anum_hypertable_data_node_hypertable_id)] = - Int32GetDatum(node->fd.hypertable_id); - values[AttrNumberGetAttrOffset(Anum_hypertable_data_node_node_hypertable_id)] = - Int32GetDatum(node->fd.node_hypertable_id); - values[AttrNumberGetAttrOffset(Anum_hypertable_data_node_node_name)] = - NameGetDatum(&node->fd.node_name); - tuple = heap_form_tuple(tupdesc, values, nulls); - - return HeapTupleGetDatum(tuple); -} - -static List * -create_data_node_options(const char *host, int32 port, const char *dbname, const char *user, - const char *password) -{ - DefElem *host_elm = makeDefElem("host", (Node *) makeString(pstrdup(host)), -1); - DefElem *port_elm = makeDefElem("port", (Node *) makeInteger(port), -1); - DefElem *dbname_elm = makeDefElem("dbname", (Node *) makeString(pstrdup(dbname)), -1); - DefElem *user_elm = makeDefElem("user", (Node *) makeString(pstrdup(user)), -1); - - if (NULL != password) - { - DefElem *password_elm = makeDefElem("password", (Node *) makeString(pstrdup(password)), -1); - return list_make5(host_elm, port_elm, dbname_elm, user_elm, password_elm); - } - - return list_make4(host_elm, port_elm, dbname_elm, user_elm); -} - -/* Returns 'true' if the database was created. */ -static bool -data_node_bootstrap_database(TSConnection *conn, const DbInfo *database) -{ - const char *const username = PQuser(remote_connection_get_pg_conn(conn)); - - Assert(database); - - if (data_node_validate_database(conn, database)) - { - /* If the database already existed on the remote node, we will log a - * notice and proceed since it is not an error if the database already - * existed on the remote node. */ - elog(NOTICE, - "database \"%s\" already exists on data node, skipping", - NameStr(database->name)); - return false; - } - - /* Create the database with the user as owner. There is no need to - * validate the database after this command since it should be created - * correctly. */ - PGresult *res = - remote_connection_execf(conn, - "CREATE DATABASE %s ENCODING %s LC_COLLATE %s LC_CTYPE %s " - "TEMPLATE template0 OWNER %s", - quote_identifier(NameStr(database->name)), - quote_identifier(pg_encoding_to_char(database->encoding)), - quote_literal_cstr(database->collation), - quote_literal_cstr(database->chartype), - quote_identifier(username)); - if (PQresultStatus(res) != PGRES_COMMAND_OK) - remote_result_elog(res, ERROR); - return true; -} - -/* Validate the database. - * - * Errors: - * Will abort with errors if the database exists but is not correctly set - * up. - * Returns: - * true if the database exists and is valid - * false if it does not exist. - */ -static bool -data_node_validate_database(TSConnection *conn, const DbInfo *database) -{ - PGresult *res; - int32 actual_encoding; - const char *actual_chartype; - const char *actual_collation; - - res = remote_connection_execf(conn, - "SELECT encoding, datcollate, datctype " - "FROM pg_database WHERE datname = %s", - quote_literal_cstr(NameStr(database->name))); - - if (PQresultStatus(res) != PGRES_TUPLES_OK) - ereport(ERROR, - (errcode(ERRCODE_CONNECTION_EXCEPTION), errmsg("%s", PQresultErrorMessage(res)))); - - if (PQntuples(res) == 0) - return false; - - Assert(PQnfields(res) > 2); - - actual_encoding = atoi(PQgetvalue(res, 0, 0)); - if (actual_encoding != database->encoding) - ereport(ERROR, - (errcode(ERRCODE_TS_DATA_NODE_INVALID_CONFIG), - errmsg("database exists but has wrong encoding"), - errdetail("Expected database encoding to be \"%s\" (%u) but it was \"%s\" (%u).", - pg_encoding_to_char(database->encoding), - database->encoding, - pg_encoding_to_char(actual_encoding), - actual_encoding))); - - actual_collation = PQgetvalue(res, 0, 1); - Assert(actual_collation != NULL); - if (strcmp(actual_collation, database->collation) != 0) - ereport(ERROR, - (errcode(ERRCODE_TS_DATA_NODE_INVALID_CONFIG), - errmsg("database exists but has wrong collation"), - errdetail("Expected collation \"%s\" but it was \"%s\".", - database->collation, - actual_collation))); - - actual_chartype = PQgetvalue(res, 0, 2); - Assert(actual_chartype != NULL); - if (strcmp(actual_chartype, database->chartype) != 0) - ereport(ERROR, - (errcode(ERRCODE_TS_DATA_NODE_INVALID_CONFIG), - errmsg("database exists but has wrong LC_CTYPE"), - errdetail("Expected LC_CTYPE \"%s\" but it was \"%s\".", - database->chartype, - actual_chartype))); - return true; -} - -static void -data_node_validate_extension(TSConnection *conn) -{ - const char *const dbname = PQdb(remote_connection_get_pg_conn(conn)); - const char *const host = PQhost(remote_connection_get_pg_conn(conn)); - const char *const port = PQport(remote_connection_get_pg_conn(conn)); - - if (!remote_connection_check_extension(conn)) - ereport(ERROR, - (errcode(ERRCODE_TS_DATA_NODE_INVALID_CONFIG), - errmsg("database does not have TimescaleDB extension loaded"), - errdetail("The TimescaleDB extension is not loaded in database %s on node at " - "%s:%s.", - dbname, - host, - port))); -} - -static void -data_node_validate_as_data_node(TSConnection *conn) -{ - PGresult *res = - remote_connection_exec(conn, "SELECT _timescaledb_functions.validate_as_data_node()"); - - if (PQresultStatus(res) != PGRES_TUPLES_OK) - ereport(ERROR, - (errcode(ERRCODE_TS_DATA_NODE_INVALID_CONFIG), - (errmsg("cannot add \"%s\" as a data node", remote_connection_node_name(conn)), - errdetail("%s", PQresultErrorMessage(res))))); - - remote_result_close(res); -} - -/* - * Bootstrap the extension and associated objects. - */ -static bool -data_node_bootstrap_extension(TSConnection *conn) -{ - const char *const username = PQuser(remote_connection_get_pg_conn(conn)); - const char *schema_name = ts_extension_schema_name(); - const char *schema_name_quoted = quote_identifier(schema_name); - Oid schema_oid = get_namespace_oid(schema_name, true); - - /* We only count the number of tuples in the code below, but having the - * name and version are useful for debugging purposes. */ - PGresult *res = - remote_connection_execf(conn, - "SELECT extname, extversion FROM pg_extension WHERE extname = %s", - quote_literal_cstr(EXTENSION_NAME)); - - if (PQresultStatus(res) != PGRES_TUPLES_OK) - remote_result_elog(res, ERROR); - - if (PQntuples(res) == 0) - { - remote_result_close(res); - - if (schema_oid != PG_PUBLIC_NAMESPACE) - { - res = remote_connection_execf(conn, - "CREATE SCHEMA %s AUTHORIZATION %s", - schema_name_quoted, - quote_identifier(username)); - - if (PQresultStatus(res) != PGRES_COMMAND_OK) - { - const char *const sqlstate = PQresultErrorField(res, PG_DIAG_SQLSTATE); - bool schema_exists = - (sqlstate && strcmp(sqlstate, ERRCODE_DUPLICATE_SCHEMA_STR) == 0); - if (!schema_exists) - remote_result_elog(res, ERROR); - - remote_result_close(res); - /* If the schema already existed on the remote node, we got a - * duplicate schema error and the schema was not created. In - * that case, we log an error with a hint on how to fix the - * issue. */ - ereport(ERROR, - (errcode(ERRCODE_DUPLICATE_SCHEMA), - errmsg("schema \"%s\" already exists in database, aborting", schema_name), - errhint("Make sure that the data node does not contain any " - "existing objects prior to adding it."))); - } - - remote_result_close(res); - } - - remote_connection_cmdf_ok(conn, - "CREATE EXTENSION " EXTENSION_NAME - " WITH SCHEMA %s VERSION %s CASCADE", - schema_name_quoted, - quote_literal_cstr(ts_extension_get_version())); - return true; - } - else - { - ereport(NOTICE, - (errmsg("extension \"%s\" already exists on data node, skipping", - PQgetvalue(res, 0, 0)), - errdetail("TimescaleDB extension version on %s:%s was %s.", - PQhost(remote_connection_get_pg_conn(conn)), - PQport(remote_connection_get_pg_conn(conn)), - PQgetvalue(res, 0, 1)))); - remote_result_close(res); - data_node_validate_extension(conn); - return false; - } -} - -/* Add dist_uuid on the remote node. - * - * If the remote node is set to use the current database, `set_dist_id` will report an error and not - * set it. */ -static void -add_distributed_id_to_data_node(TSConnection *conn) -{ - Datum id_string = DirectFunctionCall1(uuid_out, dist_util_get_id()); - PGresult *res = remote_connection_queryf_ok(conn, - "SELECT _timescaledb_functions.set_dist_id('%s')", - DatumGetCString(id_string)); - remote_result_close(res); -} - -/* - * Connect to do bootstrapping. - * - * We iterate through the list of databases and try to connect to so we can - * bootstrap the data node. - */ -static TSConnection * -connect_for_bootstrapping(const char *node_name, const char *const host, int32 port, - const char *username, const char *password) -{ - TSConnection *conn = NULL; - char *err = NULL; - - for (size_t i = 0; i < lengthof(bootstrap_databases); i++) - { - List *node_options = - create_data_node_options(host, port, bootstrap_databases[i], username, password); - conn = remote_connection_open(node_name, node_options, TS_NO_TIMEOUT, &err); - - if (conn) - return conn; - } - - ereport(ERROR, - (errcode(ERRCODE_SQLCLIENT_UNABLE_TO_ESTABLISH_SQLCONNECTION), - errmsg("could not connect to \"%s\"", node_name), - err == NULL ? 0 : errdetail("%s", err))); - - pg_unreachable(); - - return NULL; -} - -/* - * Validate that compatible extension is available on the data node. - * - * We check all available extension versions. Since we are connected to - * template DB when performing this check, it means we can't - * really tell if a compatible extension is installed in the database we - * are trying to add to the cluster. However we can make sure that a user - * will be able to manually upgrade the extension on the data node if needed. - * - * Will abort with error if there is no compatible version available, otherwise do nothing. - */ -static void -data_node_validate_extension_availability(TSConnection *conn) -{ - StringInfo concat_versions = makeStringInfo(); - bool compatible = false; - PGresult *res; - int i; - - res = - remote_connection_execf(conn, - "SELECT version FROM pg_available_extension_versions WHERE name = " - "%s AND version ~ '\\d+.\\d+.\\d+.*' ORDER BY version DESC", - quote_literal_cstr(EXTENSION_NAME)); - - if (PQresultStatus(res) != PGRES_TUPLES_OK) - ereport(ERROR, - (errcode(ERRCODE_CONNECTION_EXCEPTION), - errmsg("failed to validate remote extension: %s", PQresultErrorMessage(res)))); - - if (PQntuples(res) == 0) - ereport(ERROR, - (errcode(ERRCODE_TS_DATA_NODE_INVALID_CONFIG), - errmsg("TimescaleDB extension not available on remote PostgreSQL instance"), - errhint("Install the TimescaleDB extension on the remote PostgresSQL instance."))); - - Assert(PQnfields(res) == 1); - - for (i = 0; i < PQntuples(res); i++) - { - appendStringInfo(concat_versions, "%s, ", PQgetvalue(res, i, 0)); - compatible = dist_util_is_compatible_version(PQgetvalue(res, i, 0), TIMESCALEDB_VERSION); - if (compatible) - break; - } - - if (!compatible) - ereport(ERROR, - (errcode(ERRCODE_TS_DATA_NODE_INVALID_CONFIG), - errmsg("remote PostgreSQL instance has an incompatible timescaledb extension " - "version"), - errdetail_internal("Access node version: %s, available remote versions: %s.", - TIMESCALEDB_VERSION_MOD, - concat_versions->data))); -} - -/** - * Get the configured server port for the server as an integer. - * - * Returns: - * Port number if a port is configured, -1 if it is not able to get - * the port number. - * - * Note: - * We cannot use `inet_server_port()` since that will return NULL if - * connecting to a server on localhost since a UNIX socket will be - * used. This is the case even if explicitly using a port when - * connecting. Regardless of how the user connected, we want to use the same - * port as the one that the server listens on. - */ -static int32 -get_server_port() -{ - const char *const portstr = - GetConfigOption("port", /* missing_ok= */ false, /* restrict_privileged= */ false); - return pg_strtoint32(portstr); -} - -static void -validate_data_node_port(int port) -{ - if (port < 1 || port > PG_UINT16_MAX) - ereport(ERROR, - (errcode(ERRCODE_INVALID_PARAMETER_VALUE), - (errmsg("invalid port number %d", port), - errhint("The port number must be between 1 and %u.", PG_UINT16_MAX)))); -} - -/* set_distid may need to be false for some otherwise invalid configurations - * that are useful for testing */ -static Datum -data_node_add_internal(PG_FUNCTION_ARGS, bool set_distid) -{ - Oid userid = GetUserId(); - const char *username = GetUserNameFromId(userid, false); - const char *node_name = PG_ARGISNULL(0) ? NULL : PG_GETARG_CSTRING(0); - const char *host = PG_ARGISNULL(1) ? NULL : TextDatumGetCString(PG_GETARG_DATUM(1)); - const char *dbname = PG_ARGISNULL(2) ? get_database_name(MyDatabaseId) : PG_GETARG_CSTRING(2); - int32 port = PG_ARGISNULL(3) ? get_server_port() : PG_GETARG_INT32(3); - bool if_not_exists = PG_ARGISNULL(4) ? false : PG_GETARG_BOOL(4); - bool bootstrap = PG_ARGISNULL(5) ? true : PG_GETARG_BOOL(5); - const char *password = PG_ARGISNULL(6) ? NULL : TextDatumGetCString(PG_GETARG_DATUM(6)); - bool server_created = false; - bool database_created = false; - bool extension_created = false; - bool PG_USED_FOR_ASSERTS_ONLY result; - DbInfo database; - - TS_PREVENT_FUNC_IF_READ_ONLY(); - - namestrcpy(&database.name, dbname); - - if (host == NULL) - ereport(ERROR, - (errcode(ERRCODE_INVALID_PARAMETER_VALUE), - (errmsg("a host needs to be specified"), - errhint("Provide a host name or IP address of a data node to add.")))); - - if (set_distid && dist_util_membership() == DIST_MEMBER_DATA_NODE) - ereport(ERROR, - (errcode(ERRCODE_TS_DATA_NODE_ASSIGNMENT_ALREADY_EXISTS), - (errmsg("unable to assign data nodes from an existing distributed database")))); - - if (NULL == node_name) - ereport(ERROR, - (errcode(ERRCODE_INVALID_PARAMETER_VALUE), - (errmsg("data node name cannot be NULL")))); - - validate_data_node_port(port); - - result = get_database_info(MyDatabaseId, &database); - Assert(result); - - /* - * Since this function creates databases on remote nodes, and CREATE DATABASE - * cannot run in a transaction block, we cannot run the function in a - * transaction block either. - */ - TS_PREVENT_IN_TRANSACTION_BLOCK(true); - - /* Try to create the foreign server, or get the existing one in case of - * if_not_exists true. */ - if (create_foreign_server(node_name, host, port, dbname, if_not_exists)) - { - List *node_options; - TSConnection *conn; - - server_created = true; - - /* Make the foreign server visible in current transaction. */ - CommandCounterIncrement(); - - /* If bootstrapping, we check the extension availability here and - * abort if the extension is not available. We should not start - * creating databases and other cruft on the datanode unless we know - * that the extension is installed. - * - * We ensure that there is a database if we are bootstrapping. This is - * done using a separate connection since the database that is going - * to be used for the data node does not exist yet, so we cannot - * connect to it. */ - if (bootstrap) - { - TSConnection *conn = - connect_for_bootstrapping(node_name, host, port, username, password); - Assert(NULL != conn); - data_node_validate_extension_availability(conn); - database_created = data_node_bootstrap_database(conn, &database); - remote_connection_close(conn); - } - - /* Connect to the database we are bootstrapping and either install the - * extension or validate that the extension is installed. The - * following statements are executed inside a transaction so that they - * can be rolled back in the event of a failure. - * - * We could use `remote_dist_txn_get_connection` here, but it is - * comparably heavy and make the code more complicated than - * necessary. Instead using a more straightforward approach here since - * we do not need 2PC support. */ - node_options = create_data_node_options(host, port, dbname, username, password); - conn = remote_connection_open_session(node_name, node_options, false); - Assert(NULL != conn); - remote_connection_cmd_ok(conn, "BEGIN"); - - if (bootstrap) - extension_created = data_node_bootstrap_extension(conn); - - if (!database_created) - { - data_node_validate_database(conn, &database); - data_node_validate_as_data_node(conn); - } - - if (!extension_created) - data_node_validate_extension(conn); - - /* After the node is verified or bootstrapped, we set the `dist_uuid` - * using the same connection. We skip this if clustering checks are - * disabled, which means that the `dist_uuid` is neither set nor - * checked. - * - * This is done inside a transaction so that we can roll it back if - * there are any failures. Note that any failure at this point will - * not rollback the creates above. */ - if (set_distid) - { - if (dist_util_membership() != DIST_MEMBER_ACCESS_NODE) - dist_util_set_as_access_node(); - add_distributed_id_to_data_node(conn); - } - - /* If there were an error before, we will not reach this point to the - * transaction will be aborted when the connection is closed. */ - remote_connection_cmd_ok(conn, "COMMIT"); - remote_connection_close(conn); - } - - PG_RETURN_DATUM(create_data_node_datum(fcinfo, - node_name, - host, - port, - dbname, - server_created, - database_created, - extension_created)); -} - -Datum -data_node_add(PG_FUNCTION_ARGS) -{ -#if PG16_GE - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("adding data node is not supported"), - errdetail("Multi-node is not supported anymore on PostgreSQL >= 16."))); -#else - ereport(WARNING, - (errcode(ERRCODE_WARNING_DEPRECATED_FEATURE), - errmsg("adding data node is deprecated"), - errdetail("Multi-node is deprecated and will be removed in future releases."))); -#endif - return data_node_add_internal(fcinfo, true); -} - -Datum -data_node_add_without_dist_id(PG_FUNCTION_ARGS) -{ -#if PG16_GE - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("adding data node is not supported"), - errdetail("Multi-node is not supported anymore on PostgreSQL >= 16."))); -#else - ereport(WARNING, - (errcode(ERRCODE_WARNING_DEPRECATED_FEATURE), - errmsg("adding data node is deprecated"), - errdetail("Multi-node is deprecated and will be removed in future releases."))); -#endif - return data_node_add_internal(fcinfo, false); -} - -Datum -data_node_attach(PG_FUNCTION_ARGS) -{ -#if PG16_GE - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("attaching data node is not supported"), - errdetail("Multi-node is not supported anymore on PostgreSQL >= 16."))); -#else - ereport(WARNING, - (errcode(ERRCODE_WARNING_DEPRECATED_FEATURE), - errmsg("attaching data node is deprecated"), - errdetail("Multi-node is deprecated and will be removed in future releases."))); -#endif - - const char *node_name = PG_ARGISNULL(0) ? NULL : PG_GETARG_CSTRING(0); - Oid table_id = PG_GETARG_OID(1); - bool if_not_attached = PG_ARGISNULL(2) ? false : PG_GETARG_BOOL(2); - bool repartition = PG_ARGISNULL(3) ? false : PG_GETARG_BOOL(3); - ForeignServer *fserver; - HypertableDataNode *node; - Cache *hcache; - Hypertable *ht; - Dimension *space_dim; - List *result; - int num_nodes; - ListCell *lc; - Oid uid, saved_uid; - int sec_ctx; - Relation rel; - - TS_PREVENT_FUNC_IF_READ_ONLY(); - - if (PG_ARGISNULL(1)) - ereport(ERROR, - (errcode(ERRCODE_INVALID_PARAMETER_VALUE), errmsg("hypertable cannot be NULL"))); - Assert(get_rel_name(table_id)); - - ht = ts_hypertable_cache_get_cache_and_entry(table_id, CACHE_FLAG_NONE, &hcache); - Assert(ht != NULL); - - if (!hypertable_is_distributed(ht)) - ereport(ERROR, - (errcode(ERRCODE_TS_HYPERTABLE_NOT_DISTRIBUTED), - errmsg("hypertable \"%s\" is not distributed", get_rel_name(table_id)))); - - /* Must have owner permissions on the hypertable to attach a new data - node. Must also have USAGE on the foreign server. */ - ts_hypertable_permissions_check(table_id, GetUserId()); - fserver = data_node_get_foreign_server(node_name, ACL_USAGE, true, false); - - Assert(NULL != fserver); - - foreach (lc, ht->data_nodes) - { - node = lfirst(lc); - - if (node->foreign_server_oid == fserver->serverid) - { - ts_cache_release(hcache); - - if (if_not_attached) - { - ereport(NOTICE, - (errcode(ERRCODE_TS_DATA_NODE_ALREADY_ATTACHED), - errmsg("data node \"%s\" is already attached to hypertable \"%s\", " - "skipping", - node_name, - get_rel_name(table_id)))); - PG_RETURN_DATUM(create_hypertable_data_node_datum(fcinfo, node)); - } - else - ereport(ERROR, - (errcode(ERRCODE_TS_DATA_NODE_ALREADY_ATTACHED), - errmsg("data node \"%s\" is already attached to hypertable \"%s\"", - node_name, - get_rel_name(table_id)))); - } - } - - /* - * Change to the hypertable owner so that the same permissions will be set up on the - * datanode being attached to as well. We need to do this explicitly because the - * caller of this function could be a superuser and we definitely don't want to create - * this hypertable with superuser ownership on the datanode being attached to! - * - * We retain the lock on the hypertable till the end of the traction to avoid any - * possibility of a concurrent "ALTER TABLE OWNER TO" changing the owner underneath - * us. - */ - rel = table_open(ht->main_table_relid, AccessShareLock); - uid = rel->rd_rel->relowner; - table_close(rel, NoLock); - GetUserIdAndSecContext(&saved_uid, &sec_ctx); - - if (uid != saved_uid) - SetUserIdAndSecContext(uid, sec_ctx | SECURITY_LOCAL_USERID_CHANGE); - - result = hypertable_assign_data_nodes(ht->fd.id, list_make1((char *) node_name)); - Assert(result->length == 1); - - /* Refresh the cached hypertable entry to get the attached node */ - ts_cache_release(hcache); - hcache = ts_hypertable_cache_pin(); - ht = ts_hypertable_cache_get_entry(hcache, table_id, CACHE_FLAG_NONE); - - /* Get the first closed (space) dimension, which is the one along which we - * partition across data nodes. */ - space_dim = ts_hyperspace_get_mutable_dimension(ht->space, DIMENSION_TYPE_CLOSED, 0); - num_nodes = list_length(ht->data_nodes); - - if (num_nodes > MAX_NUM_HYPERTABLE_DATA_NODES) - ereport(ERROR, - (errcode(ERRCODE_INVALID_PARAMETER_VALUE), - errmsg("max number of data nodes already attached"), - errdetail("The number of data nodes in a hypertable cannot exceed %d.", - MAX_NUM_HYPERTABLE_DATA_NODES))); - - /* If there are less slices (partitions) in the space dimension than there - * are data nodes, we'd like to expand the number of slices to be able to - * make use of the new data node. */ - if (NULL != space_dim) - { - List *data_node_names = NIL; - int num_partitions = space_dim->fd.num_slices; - - if (num_nodes > space_dim->fd.num_slices) - { - if (repartition) - { - ts_dimension_set_number_of_slices(space_dim, num_nodes & 0xFFFF); - num_partitions = num_nodes; - - ereport(NOTICE, - (errmsg("the number of partitions in dimension \"%s\" was increased to %u", - NameStr(space_dim->fd.column_name), - num_nodes), - errdetail("To make use of all attached data nodes, a distributed " - "hypertable needs at least as many partitions in the first " - "closed (space) dimension as there are attached data nodes."))); - } - else - { - /* Raise a warning if the number of partitions are too few to make - * use of all data nodes. Need to refresh cache first to get the - * updated data node list. */ - int dimension_id = space_dim->fd.id; - ts_hypertable_check_partitioning(ht, dimension_id); - } - } - - data_node_names = ts_hypertable_get_available_data_node_names(ht, true); - ts_dimension_partition_info_recreate(space_dim->fd.id, - num_partitions, - data_node_names, - ht->fd.replication_factor); - } - - node = linitial(result); - ts_cache_release(hcache); - - /* Need to restore security context */ - if (uid != saved_uid) - SetUserIdAndSecContext(saved_uid, sec_ctx); - - PG_RETURN_DATUM(create_hypertable_data_node_datum(fcinfo, node)); -} - -/* Only used for generating proper error message */ -typedef enum OperationType -{ - OP_BLOCK, - OP_DETACH, - OP_DELETE -} OperationType; - -static void -check_replication_for_new_data(const char *node_name, Hypertable *ht, bool force) -{ - List *available_nodes = ts_hypertable_get_available_data_nodes(ht, false); - - if (ht->fd.replication_factor < list_length(available_nodes)) - return; - - ereport(force ? WARNING : ERROR, - (errcode(ERRCODE_TS_INSUFFICIENT_NUM_DATA_NODES), - errmsg("insufficient number of data nodes for distributed hypertable \"%s\"", - NameStr(ht->fd.table_name)), - errdetail("Reducing the number of available data nodes on distributed" - " hypertable \"%s\" prevents full replication of new chunks.", - NameStr(ht->fd.table_name)), - force ? 0 : errhint("Use force => true to force this operation."))); -} - -static bool -data_node_contains_non_replicated_chunks(List *chunk_data_nodes) -{ - ListCell *lc; - - foreach (lc, chunk_data_nodes) - { - ChunkDataNode *cdn = lfirst(lc); - List *replicas = - ts_chunk_data_node_scan_by_chunk_id(cdn->fd.chunk_id, CurrentMemoryContext); - - if (list_length(replicas) < 2) - return true; - } - - return false; -} - -static List * -data_node_detach_or_delete_validate(const char *node_name, Hypertable *ht, bool force, - OperationType op_type) -{ - List *chunk_data_nodes = - ts_chunk_data_node_scan_by_node_name_and_hypertable_id(node_name, - ht->fd.id, - CurrentMemoryContext); - bool has_non_replicated_chunks = data_node_contains_non_replicated_chunks(chunk_data_nodes); - - Assert(op_type == OP_DELETE || op_type == OP_DETACH); - - if (has_non_replicated_chunks) - ereport(ERROR, - (errcode(ERRCODE_TS_INSUFFICIENT_NUM_DATA_NODES), - errmsg("insufficient number of data nodes"), - errdetail("Distributed hypertable \"%s\" would lose data if" - " data node \"%s\" is %s.", - NameStr(ht->fd.table_name), - node_name, - (op_type == OP_DELETE) ? "deleted" : "detached"), - errhint("Ensure all chunks on the data node are fully replicated before %s it.", - (op_type == OP_DELETE) ? "deleting" : "detaching"))); - - if (list_length(chunk_data_nodes) > 0) - { - if (force) - ereport(WARNING, - (errcode(ERRCODE_TS_INSUFFICIENT_NUM_DATA_NODES), - errmsg("distributed hypertable \"%s\" is under-replicated", - NameStr(ht->fd.table_name)), - errdetail("Some chunks no longer meet the replication target" - " after %s data node \"%s\".", - (op_type == OP_DELETE) ? "deleting" : "detaching", - node_name))); - else - ereport(ERROR, - (errcode(ERRCODE_TS_DATA_NODE_IN_USE), - errmsg("data node \"%s\" still holds data for distributed hypertable \"%s\"", - node_name, - NameStr(ht->fd.table_name)))); - } - - check_replication_for_new_data(node_name, ht, force); - - return chunk_data_nodes; -} - -static int -data_node_modify_hypertable_data_nodes(const char *node_name, List *hypertable_data_nodes, - bool all_hypertables, OperationType op_type, - bool block_chunks, bool force, bool repartition, - bool drop_remote_data) -{ - Cache *hcache = ts_hypertable_cache_pin(); - ListCell *lc; - int removed = 0; - - foreach (lc, hypertable_data_nodes) - { - HypertableDataNode *node = lfirst(lc); - Oid relid = ts_hypertable_id_to_relid(node->fd.hypertable_id, false); - Hypertable *ht = ts_hypertable_cache_get_entry(hcache, relid, CACHE_FLAG_NONE); - bool has_privs = ts_hypertable_has_privs_of(relid, GetUserId()); - bool update_dimension_partitions = false; - Dimension *space_dim; - - Assert(ht != NULL); - space_dim = ts_hyperspace_get_mutable_dimension(ht->space, DIMENSION_TYPE_CLOSED, 0); - - if (!has_privs) - { - /* If the operation is OP_DELETE, we MUST be able to detach the data - * node from ALL tables since the foreign server object will be - * deleted. Therefore, we fail the operation if we find a table - * that we don't have owner permissions on in this case. */ - if (all_hypertables && op_type != OP_DELETE) - ereport(NOTICE, - (errcode(ERRCODE_INSUFFICIENT_PRIVILEGE), - errmsg("skipping hypertable \"%s\" due to missing permissions", - get_rel_name(relid)))); - else - ereport(ERROR, - (errcode(ERRCODE_INSUFFICIENT_PRIVILEGE), - errmsg("permission denied for hypertable \"%s\"", get_rel_name(relid)), - errdetail("The data node is attached to hypertables that the current " - "user lacks permissions for."))); - } - else if (op_type == OP_DETACH || op_type == OP_DELETE) - { - /* we have permissions to detach */ - List *chunk_data_nodes = - data_node_detach_or_delete_validate(NameStr(node->fd.node_name), - ht, - force, - op_type); - ListCell *cs_lc; - - /* update chunk foreign table server and delete chunk mapping */ - foreach (cs_lc, chunk_data_nodes) - { - ChunkDataNode *cdn = lfirst(cs_lc); - const Chunk *chunk = ts_chunk_get_by_id(cdn->fd.chunk_id, true); - LockRelationOid(chunk->table_id, ShareUpdateExclusiveLock); - - ts_chunk_data_node_delete_by_chunk_id_and_node_name(cdn->fd.chunk_id, - NameStr(cdn->fd.node_name)); - } - - /* delete hypertable mapping */ - removed += - ts_hypertable_data_node_delete_by_node_name_and_hypertable_id(node_name, ht->fd.id); - - if (repartition) - { - int num_nodes = list_length(ht->data_nodes) - 1; - - if (space_dim != NULL && num_nodes < space_dim->fd.num_slices && num_nodes > 0) - { - ts_dimension_set_number_of_slices(space_dim, num_nodes & 0xFFFF); - - ereport(NOTICE, - (errmsg("the number of partitions in dimension \"%s\" of hypertable " - "\"%s\" was decreased to %u", - NameStr(space_dim->fd.column_name), - get_rel_name(ht->main_table_relid), - num_nodes), - errdetail( - "To make efficient use of all attached data nodes, the number of " - "space partitions was set to match the number of data nodes."))); - } - } - - /* Update dimension partitions. First remove the detach/deleted - * data node from the list of remaining nodes so that it is not - * used in the new partitioning scheme. - * - * Note that the cached dimension partition info in the Dimension - * object is not updated. The cache will be invalidated and - * released at the end of this function. - */ - update_dimension_partitions = NULL != space_dim; - - if (op_type == OP_DETACH && drop_remote_data) - { - /* Drop the hypertable on the data node */ - ts_dist_cmd_run_on_data_nodes( - psprintf("DROP TABLE IF EXISTS %s", - quote_qualified_identifier(NameStr(ht->fd.schema_name), - NameStr(ht->fd.table_name))), - list_make1(NameStr(node->fd.node_name)), - true); - } - } - else - { - /* set block new chunks */ - if (block_chunks) - { - if (node->fd.block_chunks) - { - elog(NOTICE, - "new chunks already blocked on data node \"%s\" for" - " hypertable \"%s\"", - NameStr(node->fd.node_name), - get_rel_name(relid)); - continue; - } - - check_replication_for_new_data(node_name, ht, force); - } - node->fd.block_chunks = block_chunks; - removed += ts_hypertable_data_node_update(node); - update_dimension_partitions = NULL != space_dim; - } - - if (update_dimension_partitions) - { - /* Refresh the cached hypertable to get the updated list of data nodes */ - ts_cache_release(hcache); - hcache = ts_hypertable_cache_pin(); - ht = ts_hypertable_cache_get_entry(hcache, relid, CACHE_FLAG_NONE); - ts_hypertable_update_dimension_partitions(ht); - } - } - - ts_cache_release(hcache); - return removed; -} - -static int -data_node_block_hypertable_data_nodes(const char *node_name, List *hypertable_data_nodes, - bool all_hypertables, bool block_chunks, bool force) -{ - return data_node_modify_hypertable_data_nodes(node_name, - hypertable_data_nodes, - all_hypertables, - OP_BLOCK, - block_chunks, - force, - false, - false); -} - -static int -data_node_detach_hypertable_data_nodes(const char *node_name, List *hypertable_data_nodes, - bool all_hypertables, bool force, bool repartition, - bool drop_remote_data, OperationType op_type) -{ - return data_node_modify_hypertable_data_nodes(node_name, - hypertable_data_nodes, - all_hypertables, - op_type, - false, - force, - repartition, - drop_remote_data); -} - -HypertableDataNode * -data_node_hypertable_get_by_node_name(const Hypertable *ht, const char *node_name, - bool attach_check) -{ - HypertableDataNode *hdn = NULL; - ListCell *lc; - - if (!hypertable_is_distributed(ht)) - ereport(ERROR, - (errcode(ERRCODE_TS_HYPERTABLE_NOT_DISTRIBUTED), - errmsg("hypertable \"%s\" is not distributed", - get_rel_name(ht->main_table_relid)))); - - foreach (lc, ht->data_nodes) - { - hdn = lfirst(lc); - if (namestrcmp(&hdn->fd.node_name, node_name) == 0) - break; - else - hdn = NULL; - } - - if (hdn == NULL) - { - if (attach_check) - ereport(ERROR, - (errcode(ERRCODE_TS_DATA_NODE_NOT_ATTACHED), - errmsg("data node \"%s\" is not attached to hypertable \"%s\"", - node_name, - get_rel_name(ht->main_table_relid)))); - else - ereport(NOTICE, - (errcode(ERRCODE_TS_DATA_NODE_NOT_ATTACHED), - errmsg("data node \"%s\" is not attached to hypertable \"%s\", " - "skipping", - node_name, - get_rel_name(ht->main_table_relid)))); - } - - return hdn; -} - -static HypertableDataNode * -get_hypertable_data_node(Oid table_id, const char *node_name, bool owner_check, bool attach_check) -{ - HypertableDataNode *hdn = NULL; - Cache *hcache = ts_hypertable_cache_pin(); - const Hypertable *ht = ts_hypertable_cache_get_entry(hcache, table_id, CACHE_FLAG_NONE); - - if (owner_check) - ts_hypertable_permissions_check(table_id, GetUserId()); - - hdn = data_node_hypertable_get_by_node_name(ht, node_name, attach_check); - - ts_cache_release(hcache); - - return hdn; -} - -static Datum -data_node_block_or_allow_new_chunks(const char *node_name, Oid const table_id, bool force, - bool block_chunks) -{ - int affected = 0; - bool all_hypertables = !OidIsValid(table_id); - List *hypertable_data_nodes = NIL; - ForeignServer *server = data_node_get_foreign_server(node_name, ACL_USAGE, true, false); - - Assert(NULL != server); - - if (OidIsValid(table_id)) - { - /* Early abort on missing hypertable permissions */ - ts_hypertable_permissions_check(table_id, GetUserId()); - hypertable_data_nodes = - list_make1(get_hypertable_data_node(table_id, server->servername, true, true)); - } - else - { - /* block or allow for all hypertables */ - hypertable_data_nodes = - ts_hypertable_data_node_scan_by_node_name(server->servername, CurrentMemoryContext); - } - - affected = data_node_block_hypertable_data_nodes(server->servername, - hypertable_data_nodes, - all_hypertables, - block_chunks, - force); - return Int32GetDatum(affected); -} - -Datum -data_node_allow_new_chunks(PG_FUNCTION_ARGS) -{ - const char *node_name = PG_ARGISNULL(0) ? NULL : NameStr(*PG_GETARG_NAME(0)); - Oid table_id = PG_ARGISNULL(1) ? InvalidOid : PG_GETARG_OID(1); - - TS_PREVENT_FUNC_IF_READ_ONLY(); - - return data_node_block_or_allow_new_chunks(node_name, table_id, false, false); -} - -Datum -data_node_block_new_chunks(PG_FUNCTION_ARGS) -{ - const char *node_name = PG_ARGISNULL(0) ? NULL : NameStr(*PG_GETARG_NAME(0)); - Oid table_id = PG_ARGISNULL(1) ? InvalidOid : PG_GETARG_OID(1); - bool force = PG_ARGISNULL(2) ? false : PG_GETARG_BOOL(2); - - TS_PREVENT_FUNC_IF_READ_ONLY(); - - return data_node_block_or_allow_new_chunks(node_name, table_id, force, true); -} - -Datum -data_node_detach(PG_FUNCTION_ARGS) -{ -#if PG16_GE - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("detaching data node is not supported"), - errdetail("Multi-node is not supported anymore on PostgreSQL >= 16."))); -#else - ereport(WARNING, - (errcode(ERRCODE_WARNING_DEPRECATED_FEATURE), - errmsg("detaching data node is deprecated"), - errdetail("Multi-node is deprecated and will be removed in future releases."))); -#endif - - const char *node_name = PG_ARGISNULL(0) ? NULL : NameStr(*PG_GETARG_NAME(0)); - Oid table_id = PG_ARGISNULL(1) ? InvalidOid : PG_GETARG_OID(1); - bool all_hypertables = PG_ARGISNULL(1); - bool if_attached = PG_ARGISNULL(2) ? false : PG_GETARG_BOOL(2); - bool force = PG_ARGISNULL(3) ? InvalidOid : PG_GETARG_OID(3); - bool repartition = PG_ARGISNULL(4) ? false : PG_GETARG_BOOL(4); - bool drop_remote_data = PG_ARGISNULL(5) ? false : PG_GETARG_BOOL(5); - int removed = 0; - List *hypertable_data_nodes = NIL; - ForeignServer *server; - - TS_PREVENT_FUNC_IF_READ_ONLY(); - - server = data_node_get_foreign_server(node_name, ACL_USAGE, true, false); - Assert(NULL != server); - - if (OidIsValid(table_id)) - { - HypertableDataNode *node; - - /* Early abort on missing hypertable permissions */ - ts_hypertable_permissions_check(table_id, GetUserId()); - - node = get_hypertable_data_node(table_id, server->servername, true, !if_attached); - if (node) - hypertable_data_nodes = list_make1(node); - } - else - { - /* Detach data node for all hypertables where user has - * permissions. Permissions checks done in - * data_node_detach_hypertable_data_nodes(). */ - hypertable_data_nodes = - ts_hypertable_data_node_scan_by_node_name(server->servername, CurrentMemoryContext); - } - - removed = data_node_detach_hypertable_data_nodes(server->servername, - hypertable_data_nodes, - all_hypertables, - force, - repartition, - drop_remote_data, - OP_DETACH); - - PG_RETURN_INT32(removed); -} - -enum Anum_show_conn -{ - Anum_alter_data_node_node_name = 1, - Anum_alter_data_node_host, - Anum_alter_data_node_port, - Anum_alter_data_node_database, - Anum_alter_data_node_available, - _Anum_alter_data_node_max, -}; - -#define Natts_alter_data_node (_Anum_alter_data_node_max - 1) - -static HeapTuple -create_alter_data_node_tuple(TupleDesc tupdesc, const char *node_name, List *options) -{ - Datum values[Natts_alter_data_node]; - bool nulls[Natts_alter_data_node] = { false }; - ListCell *lc; - - MemSet(nulls, false, sizeof(nulls)); - - NameData node_namedata; - NameData dbnamedata; - - namestrcpy(&node_namedata, node_name); - values[AttrNumberGetAttrOffset(Anum_alter_data_node_node_name)] = NameGetDatum(&node_namedata); - values[AttrNumberGetAttrOffset(Anum_alter_data_node_available)] = BoolGetDatum(true); - - foreach (lc, options) - { - DefElem *elem = lfirst(lc); - - if (strcmp("host", elem->defname) == 0) - { - values[AttrNumberGetAttrOffset(Anum_alter_data_node_host)] = - CStringGetTextDatum(defGetString(elem)); - } - else if (strcmp("port", elem->defname) == 0) - { - int port = atoi(defGetString(elem)); - values[AttrNumberGetAttrOffset(Anum_alter_data_node_port)] = Int32GetDatum(port); - } - else if (strcmp("dbname", elem->defname) == 0) - { - namestrcpy(&dbnamedata, defGetString(elem)); - values[AttrNumberGetAttrOffset(Anum_alter_data_node_database)] = - NameGetDatum(&dbnamedata); - } - else if (strcmp("available", elem->defname) == 0) - { - values[AttrNumberGetAttrOffset(Anum_alter_data_node_available)] = - BoolGetDatum(defGetBoolean(elem)); - } - } - - return heap_form_tuple(tupdesc, values, nulls); -} - -/* - * Append new data node options. - * - * When setting options via AlterForeignServer(), the defelem list must - * account for whether the an option already exists (is set) in the current - * options or it is newly added. These are different operations on a foreign - * server. - * - * Any options that already exist are purged from the current_options list so - * that only the options not set or added remains. This list can be merged - * with the new options to produce the full list of options (new and old). - */ -static List * -append_data_node_option(List *new_options, List **current_options, const char *name, Node *value) -{ - DefElem *elem; - ListCell *lc; - bool option_found = false; - - foreach (lc, *current_options) - { - elem = lfirst(lc); - - if (strcmp(elem->defname, name) == 0) - { - option_found = true; - /* Remove the option which is replaced so that the remaining - * options can be merged later into an updated list */ - *current_options = list_delete_cell(*current_options, lc); - break; - } - } - - elem = makeDefElemExtended(NULL, - pstrdup(name), - value, - option_found ? DEFELEM_SET : DEFELEM_ADD, - -1); - return lappend(new_options, elem); -} - -/* - * Alter a data node. - * - * Change the configuration of a data node, including host, port, and - * database. - * - * Can also be used to mark a data node "unavailable", which ensures it is no - * longer used for reads as long as there are replica chunks on other data - * nodes to use for reads instead. If it is not possible to fail over all - * chunks, a warning will be raised. - */ -Datum -data_node_alter(PG_FUNCTION_ARGS) -{ -#if PG16_GE - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("altering data node is not supported"), - errdetail("Multi-node is not supported anymore on PostgreSQL >= 16."))); -#else - ereport(WARNING, - (errcode(ERRCODE_WARNING_DEPRECATED_FEATURE), - errmsg("altering data node is deprecated"), - errdetail("Multi-node is deprecated and will be removed in future releases."))); -#endif - - const char *node_name = PG_ARGISNULL(0) ? NULL : NameStr(*PG_GETARG_NAME(0)); - const char *host = PG_ARGISNULL(1) ? NULL : TextDatumGetCString(PG_GETARG_DATUM(1)); - const char *database = PG_ARGISNULL(2) ? NULL : NameStr(*PG_GETARG_NAME(2)); - int port = PG_ARGISNULL(3) ? -1 : PG_GETARG_INT32(3); - bool available_is_null = PG_ARGISNULL(4); - bool available = available_is_null ? true : PG_GETARG_BOOL(4); - ForeignServer *server = NULL; - List *current_options = NIL; - List *options = NIL; - TupleDesc tupdesc; - AlterForeignServerStmt alter_server_stmt = { - .type = T_AlterForeignServerStmt, - .servername = node_name ? pstrdup(node_name) : NULL, - .has_version = false, - .version = NULL, - .options = NIL, - }; - - TS_PREVENT_FUNC_IF_READ_ONLY(); - - if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE) - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("function returning record called in context " - "that cannot accept type record"))); - - tupdesc = BlessTupleDesc(tupdesc); - - /* Check if a data node with the given name actually exists, or raise an error. */ - server = data_node_get_foreign_server(node_name, ACL_NO_CHECK, false, false /* missing_ok */); - - if (host == NULL && database == NULL && port == -1 && available_is_null) - PG_RETURN_DATUM( - HeapTupleGetDatum(create_alter_data_node_tuple(tupdesc, node_name, server->options))); - - current_options = list_copy(server->options); - - if (host != NULL) - options = append_data_node_option(options, - ¤t_options, - "host", - (Node *) makeString((char *) host)); - - if (database != NULL) - options = append_data_node_option(options, - ¤t_options, - "dbname", - (Node *) makeString((char *) database)); - - if (port != -1) - { - validate_data_node_port(port); - options = - append_data_node_option(options, ¤t_options, "port", (Node *) makeInteger(port)); - } - - if (!available_is_null) - options = append_data_node_option(options, - ¤t_options, - "available", - (Node *) makeString(available ? "true" : "false")); - - alter_server_stmt.options = options; - AlterForeignServer(&alter_server_stmt); - - /* Make changes to the data node (foreign server object) visible so that - * the changes are present when we switch "primary" data node on chunks */ - CommandCounterIncrement(); - - /* Add updated options last as they will take precedence over old options - * when creating the result tuple. */ - options = list_concat(current_options, options); - - PG_RETURN_DATUM(HeapTupleGetDatum(create_alter_data_node_tuple(tupdesc, node_name, options))); -} - -/* - * Drop a data node's database. - * - * To drop the database on the data node, a connection must be made to another - * database since one cannot drop the database currently connected - * to. Therefore, we bypass the connection cache and use a "raw" connection to - * a standard database (e.g., template0 or postgres), similar to how - * bootstrapping does it. - * - * Note that no password is provided on the command line as is done when - * bootstrapping. Instead, it is assumed that the current user already has a - * method to authenticate with the remote data node (e.g., via a password - * file, certificate, or user mapping). This should normally be the case or - * otherwise the user wouldn't have been able to use the data node. - * - * Note that the user that deletes a data node also must be the database owner - * on the data node. The database will only be dropped if there are no other - * concurrent connections so all connections must be closed before being able - * to drop the database. - */ -static void -drop_data_node_database(const ForeignServer *server) -{ - ListCell *lc; - TSConnection *conn = NULL; - Oid userid = GetUserId(); - TSConnectionId connid = { - .server_id = server->serverid, - .user_id = userid, - }; - /* Make a copy of the node name since the server pointer will be - * updated */ - char *nodename = pstrdup(server->servername); - char *dbname = NULL; - char *err = NULL; - - /* Figure out the name of the database that should be dropped */ - foreach (lc, server->options) - { - DefElem *d = lfirst(lc); - - if (strcmp(d->defname, "dbname") == 0) - { - dbname = defGetString(d); - break; - } - } - - if (NULL == dbname) - { - /* This should not happen unless the configuration is messed up */ - ereport(ERROR, - (errcode(ERRCODE_TS_DATA_NODE_INVALID_CONFIG), - errmsg("could not drop the database on data node \"%s\"", nodename), - errdetail("The data node configuration lacks the \"dbname\" option."))); - pg_unreachable(); - return; - } - - /* Clear potentially cached connection to the data node for the current - * session so that it won't block dropping the database */ - remote_connection_cache_remove(connid); - - /* Cannot connect to the database that is being dropped, so try to connect - * to a "standard" bootstrap database that we expect to exist on the data - * node */ - for (size_t i = 0; i < lengthof(bootstrap_databases); i++) - { - List *conn_options; - DefElem dbname_elem = { - .type = T_DefElem, - .defaction = DEFELEM_SET, - .defname = "dbname", - .arg = (Node *) makeString(pstrdup(bootstrap_databases[i])), - }; - AlterForeignServerStmt stmt = { - .type = T_AlterForeignServerStmt, - .servername = nodename, - .has_version = false, - .options = list_make1(&dbname_elem), - }; - - /* - * We assume that the user already has credentials configured to - * connect to the data node, e.g., via a user mapping, password file, - * or certificate. But in order to easily make use of those - * authentication methods, we need to establish the connection using - * the standard connection functions to pick up the foreign server - * options and associated user mapping (if such a mapping - * exists). However, the foreign server configuration references the - * database we are trying to drop, so we first need to update the - * foreign server definition to use the bootstrap database. */ - AlterForeignServer(&stmt); - - /* Make changes to foreign server database visible */ - CommandCounterIncrement(); - - /* Get the updated server definition */ - server = data_node_get_foreign_server(nodename, ACL_USAGE, true, false); - /* Open a connection to the bootstrap database using the new server options */ - conn_options = remote_connection_prepare_auth_options(server, userid); - conn = remote_connection_open(nodename, conn_options, TS_NO_TIMEOUT, &err); - - if (NULL != conn) - break; - } - - if (NULL != conn) - { - /* Do not include FORCE or IF EXISTS options when dropping the - * database. Instead, we expect the database to exist, or the user - * has to rerun the command without drop_database=>true set. We - * don't force removal if there are other connections to the - * database out of caution. If the user wants to forcefully remove - * the database, they can do it manually. From PG15, the backend - * executing the DROP forces all other backends to close all smgr - * fds using the ProcSignalBarrier mechanism. To allow this backend - * to handle that interrupt, send the DROP request using the async - * API. */ - char *cmd; - AsyncRequest *req; - - cmd = psprintf("DROP DATABASE %s", quote_identifier(dbname)); - req = async_request_send(conn, cmd); - Assert(NULL != req); - async_request_wait_ok_result(req); - remote_connection_close(conn); - pfree(req); - pfree(cmd); - } - else - ereport(ERROR, - (errcode(ERRCODE_SQLCLIENT_UNABLE_TO_ESTABLISH_SQLCONNECTION), - errmsg("could not connect to data node \"%s\"", nodename), - err == NULL ? 0 : errdetail("%s", err))); -} - -Datum -data_node_delete(PG_FUNCTION_ARGS) -{ -#if PG16_GE - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("deleting data node is not supported"), - errdetail("Multi-node is not supported anymore on PostgreSQL >= 16."))); -#else - ereport(WARNING, - (errcode(ERRCODE_WARNING_DEPRECATED_FEATURE), - errmsg("deleting data node is deprecated"), - errdetail("Multi-node is deprecated and will be removed in future releases."))); -#endif - - const char *node_name = PG_ARGISNULL(0) ? NULL : PG_GETARG_CSTRING(0); - bool if_exists = PG_ARGISNULL(1) ? false : PG_GETARG_BOOL(1); - bool force = PG_ARGISNULL(2) ? false : PG_GETARG_BOOL(2); - bool repartition = PG_ARGISNULL(3) ? false : PG_GETARG_BOOL(3); - bool drop_database = PG_ARGISNULL(4) ? false : PG_GETARG_BOOL(4); - List *hypertable_data_nodes = NIL; - DropStmt stmt; - ObjectAddress address; - ObjectAddress secondary_object = { - .classId = InvalidOid, - .objectId = InvalidOid, - .objectSubId = 0, - }; - Node *parsetree = NULL; - TSConnectionId cid; - ForeignServer *server; - - TS_PREVENT_FUNC_IF_READ_ONLY(); - - /* Need USAGE to detach. Further owner check done when executing the DROP - * statement. */ - server = data_node_get_foreign_server(node_name, ACL_USAGE, true, if_exists); - - Assert(server == NULL ? if_exists : true); - - if (NULL == server) - { - elog(NOTICE, "data node \"%s\" does not exist, skipping", node_name); - PG_RETURN_BOOL(false); - } - - if (drop_database) - { - TS_PREVENT_IN_TRANSACTION_BLOCK(true); - } - - /* close any pending connections */ - remote_connection_id_set(&cid, server->serverid, GetUserId()); - remote_connection_cache_remove(cid); - - /* detach data node */ - hypertable_data_nodes = - ts_hypertable_data_node_scan_by_node_name(node_name, CurrentMemoryContext); - - data_node_detach_hypertable_data_nodes(node_name, - hypertable_data_nodes, - true, - force, - repartition, - false, - OP_DELETE); - - /* clean up persistent transaction records */ - remote_txn_persistent_record_delete_for_data_node(server->serverid, NULL); - - stmt = (DropStmt){ - .type = T_DropStmt, - .objects = list_make1(makeString(pstrdup(node_name))), - .removeType = OBJECT_FOREIGN_SERVER, - .behavior = DROP_RESTRICT, - .missing_ok = if_exists, - }; - - parsetree = (Node *) &stmt; - - if (drop_database) - drop_data_node_database(server); - - /* Make sure event triggers are invoked so that all dropped objects - * are collected during a cascading drop. This ensures all dependent - * objects get cleaned up. */ - EventTriggerBeginCompleteQuery(); - - PG_TRY(); - { - ObjectAddressSet(address, ForeignServerRelationId, server->serverid); - EventTriggerDDLCommandStart(parsetree); - RemoveObjects(&stmt); - EventTriggerCollectSimpleCommand(address, secondary_object, parsetree); - EventTriggerSQLDrop(parsetree); - EventTriggerDDLCommandEnd(parsetree); - } - PG_CATCH(); - { - EventTriggerEndCompleteQuery(); - PG_RE_THROW(); - } - PG_END_TRY(); - - /* Remove self from dist db if no longer have data_nodes */ - if (data_node_get_node_name_list() == NIL) - dist_util_remove_from_db(); - - EventTriggerEndCompleteQuery(); - CommandCounterIncrement(); - CacheInvalidateRelcacheByRelid(ForeignServerRelationId); - - PG_RETURN_BOOL(true); -} - -/* - * Get server list, performing an ACL check on each of them in the process. - */ -List * -data_node_get_node_name_list_with_aclcheck(AclMode mode, bool fail_on_aclcheck) -{ - HeapTuple tuple; - ScanKeyData scankey[1]; - SysScanDesc scandesc; - Relation rel; - ForeignDataWrapper *fdw = GetForeignDataWrapperByName(EXTENSION_FDW_NAME, false); - List *nodes = NIL; - - rel = table_open(ForeignServerRelationId, AccessShareLock); - - ScanKeyInit(&scankey[0], - Anum_pg_foreign_server_srvfdw, - BTEqualStrategyNumber, - F_OIDEQ, - ObjectIdGetDatum(fdw->fdwid)); - - scandesc = systable_beginscan(rel, InvalidOid, false, NULL, 1, scankey); - - while (HeapTupleIsValid(tuple = systable_getnext(scandesc))) - { - Form_pg_foreign_server form = (Form_pg_foreign_server) GETSTRUCT(tuple); - ForeignServer *server; - - server = - data_node_get_foreign_server(NameStr(form->srvname), mode, fail_on_aclcheck, false); - - if (server != NULL) - nodes = lappend(nodes, pstrdup(NameStr(form->srvname))); - } - - systable_endscan(scandesc); - table_close(rel, AccessShareLock); - - return nodes; -} - -bool -data_node_some_unavailable(void) -{ - /* Get a list of data nodes and check if one is unavailable */ - List *data_node_list = data_node_get_node_name_list_with_aclcheck(ACL_NO_CHECK, false); - ListCell *lc; - - foreach (lc, data_node_list) - { - const char *node_name = lfirst(lc); - const ForeignServer *server; - - server = data_node_get_foreign_server(node_name, ACL_NO_CHECK, false, false); - if (!ts_data_node_is_available_by_server(server)) - { - list_free(data_node_list); - return true; - } - } - list_free(data_node_list); - return false; -} - -/* - * Get server list with optional ACL check. - * - * Returns: - * - * If nodearr is NULL, returns all system-configured data nodes that fulfill - * the ACL check. - * - * If nodearr is non-NULL, returns all the data nodes in the specified array - * subject to ACL checks. - */ -List * -data_node_get_filtered_node_name_list(ArrayType *nodearr, AclMode mode, bool fail_on_aclcheck) -{ - ArrayIterator it; - Datum node_datum; - bool isnull; - List *nodes = NIL; - - if (NULL == nodearr) - return data_node_get_node_name_list_with_aclcheck(mode, fail_on_aclcheck); - - it = array_create_iterator(nodearr, 0, NULL); - - while (array_iterate(it, &node_datum, &isnull)) - { - if (!isnull) - { - const char *node_name = DatumGetCString(node_datum); - ForeignServer *server = - data_node_get_foreign_server(node_name, mode, fail_on_aclcheck, false); - - if (NULL != server) - nodes = lappend(nodes, server->servername); - } - } - - array_free_iterator(it); - - return nodes; -} - -List * -data_node_get_node_name_list(void) -{ - return data_node_get_node_name_list_with_aclcheck(ACL_NO_CHECK, false); -} - -/* - * Turn an array of data nodes into a list of names. - * - * The function will verify that all the servers in the list belong to the - * TimescaleDB foreign data wrapper. Optionally, perform ACL check on each - * data node's foreign server. Checks are skipped when specificing - * ACL_NO_CHECK. If fail_on_aclcheck is false, then no errors will be thrown - * on ACL check failures. Instead, data nodes that fail ACL checks will simply - * be filtered. - */ -List * -data_node_array_to_node_name_list_with_aclcheck(ArrayType *nodearr, AclMode mode, - bool fail_on_aclcheck) -{ - if (NULL == nodearr) - return NIL; - - Assert(ARR_NDIM(nodearr) <= 1); - - return data_node_get_filtered_node_name_list(nodearr, mode, fail_on_aclcheck); -} - -List * -data_node_array_to_node_name_list(ArrayType *nodearr) -{ - return data_node_array_to_node_name_list_with_aclcheck(nodearr, ACL_NO_CHECK, false); -} - -Datum -data_node_ping(PG_FUNCTION_ARGS) -{ - const char *node_name = PG_ARGISNULL(0) ? NULL : PG_GETARG_CSTRING(0); - Interval *timeout = PG_ARGISNULL(1) ? NULL : PG_GETARG_INTERVAL_P(1); - TimestampTz endtime = TS_NO_TIMEOUT; - - /* Allow anyone to ping a data node. Otherwise the - * timescaledb_information.data_node view won't work for those users. */ - ForeignServer *server = data_node_get_foreign_server(node_name, ACL_NO_CHECK, false, false); - bool success; - - Assert(NULL != server); - - /* Get endtime in microseconds */ - if (timeout) - endtime = GetCurrentTimestamp() + ts_get_interval_period_approx(timeout); - - success = remote_connection_ping(server->servername, endtime); - - PG_RETURN_DATUM(BoolGetDatum(success)); -} - -List * -data_node_oids_to_node_name_list(List *data_node_oids, AclMode mode) -{ - List *node_names = NIL; - ListCell *lc; - ForeignServer *fs; - - foreach (lc, data_node_oids) - { - Oid foreign_server_oid = lfirst_oid(lc); - fs = data_node_get_foreign_server_by_oid(foreign_server_oid, mode); - node_names = lappend(node_names, pstrdup(fs->servername)); - } - - return node_names; -} - -void -data_node_name_list_check_acl(List *data_node_names, AclMode mode) -{ - AclResult aclresult; - Oid curuserid; - ListCell *lc; - - if (data_node_names == NIL) - return; - - curuserid = GetUserId(); - - foreach (lc, data_node_names) - { - /* Validate the servers, but privilege check is optional */ - ForeignServer *server = GetForeignServerByName(lfirst(lc), false); - - if (mode != ACL_NO_CHECK) - { - /* Must have permissions on the server object */ - aclresult = object_aclcheck(ForeignServerRelationId, server->serverid, curuserid, mode); - - if (aclresult != ACLCHECK_OK) - aclcheck_error(aclresult, OBJECT_FOREIGN_SERVER, server->servername); - } - } -} diff --git a/tsl/src/data_node.h b/tsl/src/data_node.h deleted file mode 100644 index da9ac02ec57..00000000000 --- a/tsl/src/data_node.h +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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. - */ -#pragma once - -#include - -#include "ts_catalog/catalog.h" -#include "ts_catalog/hypertable_data_node.h" -#include "hypertable.h" -#include "remote/dist_txn.h" - -/* Used to skip ACL checks */ -#define ACL_NO_CHECK N_ACL_RIGHTS - -extern ForeignServer *data_node_get_foreign_server(const char *node_name, AclMode mode, - bool fail_on_aclcheck, bool missing_ok); -extern ForeignServer *data_node_get_foreign_server_by_oid(Oid foreign_server_oid, AclMode mode); - -extern TSConnection *data_node_get_connection(const char *const data_node, - RemoteTxnPrepStmtOption const ps_opt, - bool transactional); - -extern Datum data_node_add(PG_FUNCTION_ARGS); -extern Datum data_node_delete(PG_FUNCTION_ARGS); -extern Datum data_node_attach(PG_FUNCTION_ARGS); -extern Datum data_node_detach(PG_FUNCTION_ARGS); -extern Datum data_node_alter(PG_FUNCTION_ARGS); -extern Datum data_node_block_new_chunks(PG_FUNCTION_ARGS); -extern Datum data_node_allow_new_chunks(PG_FUNCTION_ARGS); -extern List *data_node_get_node_name_list_with_aclcheck(AclMode mode, bool fail_on_aclcheck); -extern List *data_node_get_filtered_node_name_list(ArrayType *nodearr, AclMode mode, - bool fail_on_aclcheck); -extern List *data_node_get_node_name_list(void); -extern bool data_node_some_unavailable(void); -extern List *data_node_array_to_node_name_list_with_aclcheck(ArrayType *nodearr, AclMode mode, - bool fail_on_aclcheck); -extern List *data_node_array_to_node_name_list(ArrayType *nodearr); -extern List *data_node_oids_to_node_name_list(List *data_node_oids, AclMode mode); -extern void data_node_name_list_check_acl(List *data_node_names, AclMode mode); -extern Datum data_node_ping(PG_FUNCTION_ARGS); - -extern HypertableDataNode *data_node_hypertable_get_by_node_name(const Hypertable *ht, - const char *node_name, - bool attach_check); - -/* This should only be used for testing */ -extern Datum data_node_add_without_dist_id(PG_FUNCTION_ARGS); diff --git a/tsl/src/deparse.c b/tsl/src/deparse.c deleted file mode 100644 index 04a9cff99dd..00000000000 --- a/tsl/src/deparse.c +++ /dev/null @@ -1,1115 +0,0 @@ -/* - * 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. - */ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include - -#include "deparse.h" -#include "guc.h" -#include "utils.h" - -/* - * Deparse a table into a set of SQL commands that can be used to recreate it. - * Together with column definition it deparses constraints, indexes, triggers - * and rules as well. There are some table types that are not supported: - * temporary, partitioned, foreign, inherited and a table that uses - * options. Row security is also not supported. - */ -typedef const char *(*GetCmdFunc)(Oid oid); - -static const char * -get_index_cmd(Oid oid) -{ - return pg_get_indexdef_string(oid); -} - -static const char * -get_constraint_cmd(Oid oid) -{ - return pg_get_constraintdef_command(oid); -} - -static FunctionCallInfo -build_fcinfo_data(Oid oid) -{ - FunctionCallInfo fcinfo = palloc(SizeForFunctionCallInfo(1)); - - InitFunctionCallInfoData(*fcinfo, NULL, 1, InvalidOid, NULL, NULL); - FC_ARG(fcinfo, 0) = ObjectIdGetDatum(oid); - FC_NULL(fcinfo, 0) = false; - - return fcinfo; -} - -static const char * -get_trigger_cmd(Oid oid) -{ - return TextDatumGetCString(pg_get_triggerdef(build_fcinfo_data(oid))); -} - -static const char * -get_function_cmd(Oid oid) -{ - return TextDatumGetCString(pg_get_functiondef(build_fcinfo_data(oid))); -} - -static const char * -get_rule_cmd(Oid oid) -{ - return TextDatumGetCString(pg_get_ruledef(build_fcinfo_data(oid))); -} - -static List * -get_cmds(List *oids, GetCmdFunc get_cmd) -{ - List *cmds = NIL; - ListCell *cell; - - foreach (cell, oids) - { - StringInfo cmd = makeStringInfo(); - - appendStringInfo(cmd, "%s;", get_cmd(lfirst_oid(cell))); - cmds = lappend(cmds, cmd->data); - } - return cmds; -} - -static List * -get_constraint_cmds(List *constraint_oids) -{ - return get_cmds(constraint_oids, get_constraint_cmd); -} - -static List * -get_index_cmds(List *index_oids) -{ - return get_cmds(index_oids, get_index_cmd); -} - -static List * -get_trigger_cmds(List *trigger_oids) -{ - return get_cmds(trigger_oids, get_trigger_cmd); -} - -static List * -get_function_cmds(List *function_oids) -{ - return get_cmds(function_oids, get_function_cmd); -} - -static List * -get_rule_cmds(List *rule_oids) -{ - return get_cmds(rule_oids, get_rule_cmd); -} - -static bool -column_is_serial(Relation rel, Name column) -{ - const char *relation_name; - LOCAL_FCINFO(fcinfo, 2); - - /* Prepare call to pg_get_serial_sequence() function. - * - * We have to manually prepare the function call context here instead - * of using the DirectFunctionCall2() because we expect to get - * NULL return value. */ - relation_name = quote_qualified_identifier(get_namespace_name(rel->rd_rel->relnamespace), - NameStr(rel->rd_rel->relname)); - InitFunctionCallInfoData(*fcinfo, NULL, 2, InvalidOid, NULL, NULL); - FC_ARG(fcinfo, 0) = CStringGetTextDatum(relation_name); - FC_ARG(fcinfo, 1) = CStringGetTextDatum(NameStr(*column)); - FC_NULL(fcinfo, 0) = false; - FC_NULL(fcinfo, 1) = false; - pg_get_serial_sequence(fcinfo); - - return !fcinfo->isnull; -} - -static void -deparse_columns(StringInfo stmt, Relation rel) -{ - int att_idx; - TupleDesc rel_desc = RelationGetDescr(rel); - TupleConstr *constraints = rel_desc->constr; - - for (att_idx = 0; att_idx < rel_desc->natts; att_idx++) - { - int dim_idx; - Form_pg_attribute attr = TupleDescAttr(rel_desc, att_idx); - bits16 flags = FORMAT_TYPE_TYPEMOD_GIVEN; - - if (attr->attisdropped) - continue; - - /* - * if it's not a builtin type then schema qualify the same. There's a function - * deparse_type_name in fdw, but we don't want cross linking unnecessarily - */ - if (attr->atttypid >= FirstBootstrapObjectIdCompat) - flags |= FORMAT_TYPE_FORCE_QUALIFY; - - appendStringInfo(stmt, - "\"%s\" %s", - NameStr(attr->attname), - format_type_extended(attr->atttypid, attr->atttypmod, flags)); - - if (attr->attnotnull) - appendStringInfoString(stmt, " NOT NULL"); - - if (OidIsValid(attr->attcollation)) - appendStringInfo(stmt, " COLLATE \"%s\"", get_collation_name(attr->attcollation)); - - if (attr->atthasdef) - { - int co_idx; - - for (co_idx = 0; co_idx < constraints->num_defval; co_idx++) - { - AttrDefault attr_def = constraints->defval[co_idx]; - - if (attr->attnum == attr_def.adnum) - { - char *attr_default; - - /* Skip default expression in case if column is serial - * (has dependant sequence object) */ - if (column_is_serial(rel, &attr->attname)) - break; - - attr_default = - TextDatumGetCString(DirectFunctionCall2(pg_get_expr, - CStringGetTextDatum(attr_def.adbin), - ObjectIdGetDatum(rel->rd_id))); - - if (attr->attgenerated == ATTRIBUTE_GENERATED_STORED) - appendStringInfo(stmt, " GENERATED ALWAYS AS %s STORED", attr_default); - else - { - appendStringInfo(stmt, " DEFAULT %s", attr_default); - } - break; - } - } - } - - for (dim_idx = 1; dim_idx < attr->attndims; dim_idx++) - appendStringInfoString(stmt, "[]"); - - if (att_idx != (rel_desc->natts - 1)) - appendStringInfoString(stmt, ", "); - } -} - -typedef struct ConstraintContext -{ - List *constraints; - List **constraint_indexes; -} ConstraintContext; - -static ConstraintProcessStatus -add_constraint(HeapTuple constraint_tuple, void *ctx) -{ - ConstraintContext *cc = ctx; - Form_pg_constraint constraint = (Form_pg_constraint) GETSTRUCT(constraint_tuple); - Oid constroid; - - if (OidIsValid(constraint->conindid)) - *cc->constraint_indexes = lappend_oid(*cc->constraint_indexes, constraint->conindid); - constroid = constraint->oid; - cc->constraints = lappend_oid(cc->constraints, constroid); - return CONSTR_PROCESSED; -} - -static List * -get_constraint_oids(Oid relid, List **constraint_indexes) -{ - ConstraintContext cc = { - .constraints = NIL, - .constraint_indexes = constraint_indexes, - }; - - ts_constraint_process(relid, add_constraint, &cc); - - return cc.constraints; -} - -static List * -get_index_oids(Relation rel, List *exclude_indexes) -{ - List *indexes = NIL; - ListCell *cell; - - foreach (cell, RelationGetIndexList(rel)) - { - Oid indexid = lfirst_oid(cell); - - if (!list_member_oid(exclude_indexes, indexid)) - indexes = lappend_oid(indexes, indexid); - } - return indexes; -} - -/* - * Specifically exclude the hypertable insert blocker from this list. A table which was recreated - * with that trigger present would not be able to made into a hypertable. - */ -static List * -get_trigger_oids(Relation rel) -{ - List *triggers = NIL; - - if (rel->trigdesc != NULL) - { - int i; - - for (i = 0; i < rel->trigdesc->numtriggers; i++) - { - const Trigger trigger = rel->trigdesc->triggers[i]; - - if (!trigger.tgisinternal && strcmp(trigger.tgname, INSERT_BLOCKER_NAME) != 0) - triggers = lappend_oid(triggers, trigger.tgoid); - } - } - return triggers; -} - -static List * -get_trigger_function_oids(Relation rel) -{ - List *functions = NIL; - - if (rel->trigdesc != NULL) - { - int i; - - for (i = 0; i < rel->trigdesc->numtriggers; i++) - { - const Trigger trigger = rel->trigdesc->triggers[i]; - - if (!trigger.tgisinternal && strcmp(trigger.tgname, INSERT_BLOCKER_NAME) != 0) - functions = lappend_oid(functions, trigger.tgfoid); - } - } - - return functions; -} - -static List * -get_rule_oids(Relation rel) -{ - List *rules = NIL; - - if (rel->rd_rules != NULL) - { - int i; - - for (i = 0; i < rel->rd_rules->numLocks; i++) - { - const RewriteRule *rule = rel->rd_rules->rules[i]; - - rules = lappend_oid(rules, rule->ruleId); - } - } - return rules; -} - -static void -validate_relation(Relation rel) -{ - if (rel->rd_rel->relkind != RELKIND_RELATION) - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("given relation is not an ordinary table"))); - - if (rel->rd_rel->relpersistence == RELPERSISTENCE_TEMP) - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("temporary table is not supported"))); - - if (rel->rd_rel->relrowsecurity) - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), errmsg("row security is not supported"))); -} - -TableInfo * -deparse_create_table_info(Oid relid) -{ - List *exclude_indexes = NIL; - TableInfo *table_info = palloc0(sizeof(TableInfo)); - Relation rel = table_open(relid, AccessShareLock); - - if (rel == NULL) - ereport(ERROR, (errmsg("relation with id %u not found", relid))); - - validate_relation(rel); - - table_info->relid = relid; - table_info->constraints = get_constraint_oids(relid, &exclude_indexes); - table_info->indexes = get_index_oids(rel, exclude_indexes); - table_info->triggers = get_trigger_oids(rel); - table_info->functions = get_trigger_function_oids(rel); - table_info->rules = get_rule_oids(rel); - table_close(rel, AccessShareLock); - return table_info; -} - -static void -deparse_get_tabledef_with(const TableInfo *table_info, StringInfo create_table) -{ - ListCell *cell; - List *opts = ts_get_reloptions(table_info->relid); - - if (list_length(opts) == 0) - return; - - appendStringInfoString(create_table, " WITH ("); - - foreach (cell, opts) - { - DefElem *def = (DefElem *) lfirst(cell); - - appendStringInfo(create_table, - "%s%s=%s", - cell != list_head(opts) ? "," : "", - def->defname, - defGetString(def)); - } - - appendStringInfoChar(create_table, ')'); -} - -TableDef * -deparse_get_tabledef(TableInfo *table_info) -{ - StringInfo create_table = makeStringInfo(); - StringInfo set_schema = makeStringInfo(); - TableDef *table_def = palloc0(sizeof(TableDef)); - Relation rel = table_open(table_info->relid, AccessShareLock); - - appendStringInfo(set_schema, - "SET SCHEMA %s;", - quote_literal_cstr(get_namespace_name(rel->rd_rel->relnamespace))); - table_def->schema_cmd = set_schema->data; - - appendStringInfoString(create_table, "CREATE"); - if (rel->rd_rel->relpersistence == RELPERSISTENCE_UNLOGGED) - appendStringInfoString(create_table, " UNLOGGED"); - appendStringInfoString(create_table, " TABLE"); - - appendStringInfo(create_table, - " \"%s\".\"%s\" (", - get_namespace_name(rel->rd_rel->relnamespace), - NameStr(rel->rd_rel->relname)); - - deparse_columns(create_table, rel); - - appendStringInfoChar(create_table, ')'); - appendStringInfo(create_table, " USING \"%s\" ", get_am_name(rel->rd_rel->relam)); - deparse_get_tabledef_with(table_info, create_table); - - appendStringInfoChar(create_table, ';'); - table_def->create_cmd = create_table->data; - - table_def->constraint_cmds = get_constraint_cmds(table_info->constraints); - table_def->index_cmds = get_index_cmds(table_info->indexes); - table_def->trigger_cmds = get_trigger_cmds(table_info->triggers); - table_def->function_cmds = get_function_cmds(table_info->functions); - table_def->rule_cmds = get_rule_cmds(table_info->rules); - - table_close(rel, AccessShareLock); - return table_def; -} - -/* - * Append a privilege name to a string if the privilege is set. - * - * Parameters: - * buf: Buffer to append to. - * pfirst: Pointer to variable to remember if elements are already added. - * privs: Bitmap of privilege flags. - * mask: Mask for privilege to check. - * priv_name: String with name of privilege to add. - */ -static void -append_priv_if_set(StringInfo buf, bool *priv_added, uint32 privs, uint32 mask, - const char *priv_name) -{ - if (privs & mask) - { - if (*priv_added) - appendStringInfoString(buf, ", "); - else - *priv_added = true; - appendStringInfoString(buf, priv_name); - } -} - -static void -append_privs_as_text(StringInfo buf, uint32 privs) -{ - bool priv_added = false; - append_priv_if_set(buf, &priv_added, privs, ACL_INSERT, "INSERT"); - append_priv_if_set(buf, &priv_added, privs, ACL_SELECT, "SELECT"); - append_priv_if_set(buf, &priv_added, privs, ACL_UPDATE, "UPDATE"); - append_priv_if_set(buf, &priv_added, privs, ACL_DELETE, "DELETE"); - append_priv_if_set(buf, &priv_added, privs, ACL_TRUNCATE, "TRUNCATE"); - append_priv_if_set(buf, &priv_added, privs, ACL_REFERENCES, "REFERENCES"); - append_priv_if_set(buf, &priv_added, privs, ACL_TRIGGER, "TRIGGER"); -} - -/* - * Create grant statements for a relation. - * - * This will create a list of grant statements, one for each role. - */ -static List * -deparse_grant_commands_for_relid(Oid relid) -{ - HeapTuple reltup; - Form_pg_class pg_class_tuple; - List *cmds = NIL; - Datum acl_datum; - bool is_null; - Oid owner_id; - Acl *acl; - int i; - const AclItem *acldat; - - reltup = SearchSysCache1(RELOID, ObjectIdGetDatum(relid)); - if (!HeapTupleIsValid(reltup)) - elog(ERROR, "cache lookup failed for relation %u", relid); - pg_class_tuple = (Form_pg_class) GETSTRUCT(reltup); - - if (pg_class_tuple->relkind != RELKIND_RELATION) - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("\"%s\" is not an ordinary table", NameStr(pg_class_tuple->relname)))); - - owner_id = pg_class_tuple->relowner; - acl_datum = SysCacheGetAttr(RELOID, reltup, Anum_pg_class_relacl, &is_null); - - if (is_null) - acl = acldefault(OBJECT_TABLE, owner_id); - else - acl = DatumGetAclP(acl_datum); - - acldat = ACL_DAT(acl); - for (i = 0; i < ACL_NUM(acl); i++) - { - const AclItem *aclitem = &acldat[i]; - Oid role_id = aclitem->ai_grantee; - StringInfo grant_cmd; - HeapTuple utup; - - /* We skip the owner of the table since she automatically have all - * privileges on the table. */ - if (role_id == owner_id) - continue; - - grant_cmd = makeStringInfo(); - utup = SearchSysCache1(AUTHOID, ObjectIdGetDatum(role_id)); - - if (!HeapTupleIsValid(utup)) - continue; - - appendStringInfoString(grant_cmd, "GRANT "); - append_privs_as_text(grant_cmd, aclitem->ai_privs); - appendStringInfo(grant_cmd, - " ON TABLE %s.%s TO %s", - quote_identifier(get_namespace_name(pg_class_tuple->relnamespace)), - quote_identifier(NameStr(pg_class_tuple->relname)), - quote_identifier(NameStr(((Form_pg_authid) GETSTRUCT(utup))->rolname))); - - ReleaseSysCache(utup); - cmds = lappend(cmds, grant_cmd->data); - } - - ReleaseSysCache(reltup); - - return cmds; -} - -List * -deparse_get_tabledef_commands(Oid relid) -{ - TableInfo *table_info = deparse_create_table_info(relid); - TableDef *table_def = deparse_get_tabledef(table_info); - - return deparse_get_tabledef_commands_from_tabledef(table_def); -} - -List * -deparse_get_tabledef_commands_from_tabledef(TableDef *table_def) -{ - List *cmds = NIL; - - cmds = lappend(cmds, (char *) table_def->schema_cmd); - cmds = lappend(cmds, (char *) table_def->create_cmd); - cmds = list_concat(cmds, table_def->constraint_cmds); - cmds = list_concat(cmds, table_def->index_cmds); - cmds = list_concat(cmds, table_def->function_cmds); - cmds = list_concat(cmds, table_def->trigger_cmds); - cmds = list_concat(cmds, table_def->rule_cmds); - return cmds; -} - -const char * -deparse_get_tabledef_commands_concat(Oid relid) -{ - StringInfo tabledef = makeStringInfo(); - ListCell *cell; - - foreach (cell, deparse_get_tabledef_commands(relid)) - appendStringInfoString(tabledef, lfirst(cell)); - - return tabledef->data; -} - -static const char * -deparse_get_add_dimension_command(Hypertable *ht, Dimension *dimension) -{ - StringInfo dim_cmd = makeStringInfo(); - - appendStringInfo(dim_cmd, - "SELECT * FROM %s.add_dimension(%s, %s, ", - quote_identifier(ts_extension_schema_name()), - quote_literal_cstr( - quote_qualified_identifier(get_namespace_name( - get_rel_namespace(ht->main_table_relid)), - get_rel_name(ht->main_table_relid))), - quote_literal_cstr(NameStr(dimension->fd.column_name))); - - if (dimension->type == DIMENSION_TYPE_CLOSED) - appendStringInfo(dim_cmd, - "number_partitions => %d, partitioning_func => %s);", - dimension->fd.num_slices, - quote_literal_cstr( - quote_qualified_identifier(NameStr( - dimension->fd.partitioning_func_schema), - NameStr(dimension->fd.partitioning_func)))); - else - appendStringInfo(dim_cmd, - "chunk_time_interval => " INT64_FORMAT ");", - dimension->fd.interval_length); - - return dim_cmd->data; -} - -DeparsedHypertableCommands * -deparse_get_distributed_hypertable_create_command(Hypertable *ht) -{ - Hyperspace *space = ht->space; - Dimension *time_dim = &space->dimensions[0]; - StringInfo hypertable_cmd = makeStringInfo(); - DeparsedHypertableCommands *result = palloc(sizeof(DeparsedHypertableCommands)); - - appendStringInfo(hypertable_cmd, - "SELECT * FROM %s.create_hypertable(%s", - quote_identifier(ts_extension_schema_name()), - quote_literal_cstr( - quote_qualified_identifier(get_namespace_name( - get_rel_namespace(ht->main_table_relid)), - get_rel_name(ht->main_table_relid)))); - - appendStringInfo(hypertable_cmd, - ", time_column_name => %s", - quote_literal_cstr(NameStr(time_dim->fd.column_name))); - - if (NameStr(time_dim->fd.partitioning_func)[0] != '\0') - appendStringInfo(hypertable_cmd, - ", time_partitioning_func => %s", - quote_literal_cstr( - quote_qualified_identifier(NameStr( - time_dim->fd.partitioning_func_schema), - NameStr(time_dim->fd.partitioning_func)))); - - appendStringInfo(hypertable_cmd, - ", associated_schema_name => %s", - quote_literal_cstr(NameStr(ht->fd.associated_schema_name))); - appendStringInfo(hypertable_cmd, - ", associated_table_prefix => %s", - quote_literal_cstr(NameStr(ht->fd.associated_table_prefix))); - - appendStringInfo(hypertable_cmd, - ", chunk_time_interval => " INT64_FORMAT "", - time_dim->fd.interval_length); - - if (OidIsValid(ht->chunk_sizing_func)) - { - appendStringInfo(hypertable_cmd, - ", chunk_sizing_func => %s", - quote_literal_cstr( - quote_qualified_identifier(NameStr(ht->fd.chunk_sizing_func_schema), - NameStr(ht->fd.chunk_sizing_func_name)))); - appendStringInfo(hypertable_cmd, - ", chunk_target_size => '" INT64_FORMAT "'", - ht->fd.chunk_target_size); - } - - /* - * Data node is assumed to not have any preexisting conflicting table or hypertable. - * Any default indices will have already been created by the access node. - */ - appendStringInfoString(hypertable_cmd, ", if_not_exists => FALSE"); - appendStringInfoString(hypertable_cmd, ", migrate_data => FALSE"); - appendStringInfoString(hypertable_cmd, ", create_default_indexes => FALSE"); - appendStringInfo(hypertable_cmd, ", replication_factor => %d", HYPERTABLE_DISTRIBUTED_MEMBER); - - appendStringInfoString(hypertable_cmd, ");"); - - result->table_create_command = hypertable_cmd->data; - result->dimension_add_commands = NIL; - - if (space->num_dimensions > 1) - { - int i; - - for (i = 1; i < space->num_dimensions; i++) - result->dimension_add_commands = - lappend(result->dimension_add_commands, - (char *) deparse_get_add_dimension_command(ht, &space->dimensions[i])); - } - - result->grant_commands = deparse_grant_commands_for_relid(ht->main_table_relid); - - return result; -} - -#define DEFAULT_SCALAR_RESULT_NAME "*" - -static void -deparse_result_type(StringInfo sql, FunctionCallInfo fcinfo) -{ - TupleDesc tupdesc; - char *scalarname; - Oid resulttypeid; - int i; - - switch (get_call_result_type(fcinfo, &resulttypeid, &tupdesc)) - { - case TYPEFUNC_SCALAR: - /* scalar result type */ - Assert(NULL == tupdesc); - Assert(OidIsValid(resulttypeid)); - - /* Check if the function has a named OUT parameter */ - scalarname = get_func_result_name(fcinfo->flinfo->fn_oid); - - /* If there is no named OUT parameter, use the default name */ - if (NULL != scalarname) - { - appendStringInfoString(sql, scalarname); - pfree(scalarname); - } - else - appendStringInfoString(sql, DEFAULT_SCALAR_RESULT_NAME); - break; - case TYPEFUNC_COMPOSITE: - /* determinable rowtype result */ - Assert(NULL != tupdesc); - - for (i = 0; i < tupdesc->natts; i++) - { - if (!tupdesc->attrs[i].attisdropped) - { - appendStringInfoString(sql, NameStr(tupdesc->attrs[i].attname)); - - if (i < (tupdesc->natts - 1)) - appendStringInfoChar(sql, ','); - } - } - break; - case TYPEFUNC_RECORD: - /* indeterminate rowtype result */ - case TYPEFUNC_COMPOSITE_DOMAIN: - /* domain over determinable rowtype result */ - case TYPEFUNC_OTHER: - elog(ERROR, "unsupported result type for deparsing"); - break; - } -} - -/* - * Deparse a function call. - * - * Turn a function call back into a string. In theory, we could just call - * deparse_expression() (ruleutils.c) on the original function expression (as - * given by fcinfo->flinfo->fn_expr), but we'd like to support deparsing also - * when the expression is not available (e.g., when invoking by OID from C - * code). Further, deparse_expression() doesn't explicitly give the parameter - * names, which is important in order to maintain forward-compatibility with - * the remote version of the function in case it has reordered the parameters. - */ -const char * -deparse_func_call(FunctionCallInfo fcinfo) -{ - HeapTuple ftup; - Form_pg_proc procform; - StringInfoData sql; - const char *funcnamespace; - OverrideSearchPath search_path = { - .schemas = NIL, - .addCatalog = false, - .addTemp = false, - }; - Oid funcid = fcinfo->flinfo->fn_oid; - Oid *argtypes; - char **argnames; - char *argmodes; - int i; - - initStringInfo(&sql); - appendStringInfoString(&sql, "SELECT "); - deparse_result_type(&sql, fcinfo); - - /* First fetch the function's pg_proc row to inspect its rettype */ - ftup = SearchSysCache1(PROCOID, ObjectIdGetDatum(funcid)); - - if (!HeapTupleIsValid(ftup)) - elog(ERROR, "cache lookup failed for function %u", funcid); - - procform = (Form_pg_proc) GETSTRUCT(ftup); - funcnamespace = get_namespace_name(procform->pronamespace); - - /* If the function has named OUT-only parameters, then number of arguments - * returned by this get_func_arg_info can be greater than - * fcinfo->nargs. But we don't care about OUT-only arguements here. */ - get_func_arg_info(ftup, &argtypes, &argnames, &argmodes); - - appendStringInfo(&sql, - " FROM %s(", - quote_qualified_identifier(funcnamespace, NameStr(procform->proname))); - ReleaseSysCache(ftup); - - /* Temporarily set a NULL search path. This makes identifier types (e.g., - * regclass / tables) be fully qualified, which is needed since the search - * path on a remote node is not guaranteed to be the same. */ - PushOverrideSearchPath(&search_path); - - for (i = 0; i < fcinfo->nargs; i++) - { - const char *argvalstr = "NULL"; - bool add_type_cast = false; - - switch (argtypes[i]) - { - case ANYOID: - case ANYELEMENTOID: - /* For pseudo types, try to resolve the "real" argument type - * from the function expression, if present */ - if (NULL != fcinfo->flinfo && NULL != fcinfo->flinfo->fn_expr) - { - Oid expr_argtype = get_fn_expr_argtype(fcinfo->flinfo, i); - - /* Function parameters that aren't typed need type casts, - * but only add a cast if the expr contained a "real" type - * and not an unknown or pseudo type. */ - if (OidIsValid(expr_argtype) && expr_argtype != UNKNOWNOID && - expr_argtype != argtypes[i]) - add_type_cast = true; - - argtypes[i] = expr_argtype; - } - break; - default: - break; - } - - if (!FC_NULL(fcinfo, i)) - { - bool isvarlena; - Oid outfuncid; - - if (!OidIsValid(argtypes[i])) - elog(ERROR, "invalid type for argument %d", i); - - getTypeOutputInfo(argtypes[i], &outfuncid, &isvarlena); - Assert(OidIsValid(outfuncid)); - argvalstr = quote_literal_cstr(OidOutputFunctionCall(outfuncid, FC_ARG(fcinfo, i))); - } - - appendStringInfo(&sql, "%s => %s", argnames[i], argvalstr); - - if (add_type_cast) - appendStringInfo(&sql, "::%s", format_type_be(argtypes[i])); - - if (i < (fcinfo->nargs - 1)) - appendStringInfoChar(&sql, ','); - } - - PopOverrideSearchPath(); - - if (NULL != argtypes) - pfree(argtypes); - - if (NULL != argnames) - pfree(argnames); - - if (NULL != argmodes) - pfree(argmodes); - - appendStringInfoChar(&sql, ')'); - - return sql.data; -} - -/* - * Deparse a function by OID. - * - * The function arguments should be given as datums in the vararg list and - * need to be specified in the order given by the (OID) function's signature. - */ -const char * -deparse_oid_function_call_coll(Oid funcid, Oid collation, unsigned int num_args, ...) -{ - FunctionCallInfo fcinfo = palloc(SizeForFunctionCallInfo(num_args)); - FmgrInfo flinfo; - const char *result; - va_list args; - unsigned int i; - - fmgr_info(funcid, &flinfo); - InitFunctionCallInfoData(*fcinfo, &flinfo, num_args, collation, NULL, NULL); - va_start(args, num_args); - - for (i = 0; i < num_args; i++) - { - FC_ARG(fcinfo, i) = va_arg(args, Datum); - FC_NULL(fcinfo, i) = false; - } - - va_end(args); - - result = deparse_func_call(fcinfo); - - /* Check for null result, since caller is clearly not expecting one */ - if (fcinfo->isnull) - elog(ERROR, "function %u returned NULL", flinfo.fn_oid); - - return result; -} - -const char * -deparse_grant_revoke_on_database(const GrantStmt *stmt, const char *dbname) -{ - ListCell *lc; - - /* - GRANT { { CREATE | CONNECT | TEMPORARY | TEMP } [, ...] | ALL [ PRIVILEGES ] } - ON DATABASE database_name [, ...] - TO role_specification [, ...] [ WITH GRANT OPTION ] - [ GRANTED BY role_specification ] - - REVOKE [ GRANT OPTION FOR ] - { { CREATE | CONNECT | TEMPORARY | TEMP } [, ...] | ALL [ PRIVILEGES ] } - ON DATABASE database_name [, ...] - FROM role_specification [, ...] - [ GRANTED BY role_specification ] - [ CASCADE | RESTRICT ] - */ - StringInfo command = makeStringInfo(); - - /* GRANT/REVOKE */ - if (stmt->is_grant) - appendStringInfoString(command, "GRANT "); - else - appendStringInfoString(command, "REVOKE "); - - /* privileges [, ...] | ALL */ - if (stmt->privileges == NULL) - { - /* ALL */ - appendStringInfoString(command, "ALL "); - } - else - { - foreach (lc, stmt->privileges) - { - AccessPriv *priv = lfirst(lc); - - appendStringInfo(command, - "%s%s ", - priv->priv_name, - lnext(stmt->privileges, lc) != NULL ? "," : ""); - } - } - - /* Set database name of the data node */ - appendStringInfo(command, "ON DATABASE %s ", quote_identifier(dbname)); - - /* TO/FROM role_spec [, ...] */ - if (stmt->is_grant) - appendStringInfoString(command, "TO "); - else - appendStringInfoString(command, "FROM "); - - foreach (lc, stmt->grantees) - { - RoleSpec *role_spec = lfirst(lc); - const char *role_name = NULL; - switch (role_spec->roletype) - { - case ROLESPEC_CSTRING: - role_name = role_spec->rolename; - break; - case ROLESPEC_PUBLIC: - role_name = "PUBLIC"; - break; - case ROLESPEC_SESSION_USER: - role_name = "SESSION_USER"; - break; - case ROLESPEC_CURRENT_USER: - role_name = "CURRENT_USER"; - break; -#if PG14_GE - case ROLESPEC_CURRENT_ROLE: - role_name = "CURRENT_ROLE"; - break; -#endif - } - appendStringInfo(command, "%s%s ", role_name, lnext(stmt->grantees, lc) != NULL ? "," : ""); - } - - if (stmt->grant_option) - appendStringInfoString(command, "WITH GRANT OPTION "); - -#if PG14_GE - /* [ GRANTED BY role_specification ] */ - if (stmt->grantor) - appendStringInfo(command, "GRANTED BY %s ", quote_identifier(stmt->grantor->rolename)); -#endif - - /* CASCADE | RESTRICT */ - if (!stmt->is_grant && stmt->behavior == DROP_CASCADE) - appendStringInfoString(command, "CASCADE"); - - return command->data; -} - -/* Deparse user-defined trigger */ -const char * -deparse_create_trigger(CreateTrigStmt *stmt) -{ - ListCell *lc; - bool found_event = false; - bool found_first_arg = false; - - /* - * CREATE [ OR REPLACE ] [ CONSTRAINT ] TRIGGER name - * { BEFORE | AFTER | INSTEAD OF } { event [ OR ... ] } - * ON table_name - * [ FROM referenced_table_name ] - * [ NOT DEFERRABLE | [ DEFERRABLE ] [ INITIALLY IMMEDIATE | INITIALLY DEFERRED ] ] - * [ REFERENCING { { OLD | NEW } TABLE [ AS ] transition_relation_name } [ ... ] ] - * [ FOR [ EACH ] { ROW | STATEMENT } ] - * [ WHEN ( condition ) ] - * EXECUTE { FUNCTION | PROCEDURE } function_name ( arguments ) - */ - if (stmt->isconstraint) - elog(ERROR, "deparsing constraint triggers is not supported"); - - StringInfo command = makeStringInfo(); -#if PG14_LT - appendStringInfo(command, "CREATE TRIGGER %s ", quote_identifier(stmt->trigname)); -#else - /* Postgres 14 introduced OR REPLACE option */ - appendStringInfo(command, - "CREATE %sTRIGGER %s ", - stmt->replace ? "OR REPLACE " : "", - quote_identifier(stmt->trigname)); -#endif - - if (TRIGGER_FOR_BEFORE(stmt->timing)) - appendStringInfoString(command, "BEFORE"); - else if (TRIGGER_FOR_AFTER(stmt->timing)) - appendStringInfoString(command, "AFTER"); - else if (TRIGGER_FOR_INSTEAD(stmt->timing)) - appendStringInfoString(command, "INSTEAD OF"); - else - elog(ERROR, "unexpected timing value: %d", stmt->timing); - - if (TRIGGER_FOR_INSERT(stmt->events)) - { - appendStringInfoString(command, " INSERT"); - found_event = true; - } - if (TRIGGER_FOR_DELETE(stmt->events)) - { - if (found_event) - appendStringInfoString(command, " OR"); - appendStringInfoString(command, " DELETE"); - found_event = true; - } - if (TRIGGER_FOR_UPDATE(stmt->events)) - { - if (found_event) - appendStringInfoString(command, " OR"); - appendStringInfoString(command, " UPDATE"); - found_event = true; - } - if (TRIGGER_FOR_TRUNCATE(stmt->events)) - { - if (found_event) - appendStringInfoString(command, " OR"); - appendStringInfoString(command, " TRUNCATE"); - } - appendStringInfo(command, - " ON %s.%s", - quote_identifier(stmt->relation->schemaname), - quote_identifier(stmt->relation->relname)); - - if (stmt->row) - appendStringInfoString(command, " FOR EACH ROW"); - else - appendStringInfoString(command, " FOR EACH STATEMENT"); - - if (stmt->whenClause) - elog(ERROR, "deparsing trigger WHEN clause is not supported"); - - appendStringInfo(command, " EXECUTE FUNCTION %s(", NameListToQuotedString(stmt->funcname)); - foreach (lc, stmt->args) - { - if (found_first_arg) - appendStringInfoString(command, ", "); - else - found_first_arg = true; - appendStringInfoString(command, strVal(lfirst(lc))); - } - appendStringInfoString(command, ")"); - - return command->data; -} diff --git a/tsl/src/deparse.h b/tsl/src/deparse.h deleted file mode 100644 index 5332ad8445a..00000000000 --- a/tsl/src/deparse.h +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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. - */ -#pragma once - -#include -#include -#include - -typedef struct TableInfo -{ - Oid relid; - List *constraints; - List *indexes; - List *triggers; - List *functions; - List *rules; -} TableInfo; - -typedef struct TableDef -{ - const char *schema_cmd; - const char *create_cmd; - List *constraint_cmds; - List *index_cmds; - List *trigger_cmds; - List *rule_cmds; - List *function_cmds; -} TableDef; - -typedef struct DeparsedHypertableCommands -{ - const char *table_create_command; - List *dimension_add_commands; - List *grant_commands; -} DeparsedHypertableCommands; - -typedef struct Hypertable Hypertable; - -TableInfo *deparse_create_table_info(Oid relid); -TableDef *deparse_get_tabledef(TableInfo *table_info); -List *deparse_get_tabledef_commands(Oid relid); -List *deparse_get_tabledef_commands_from_tabledef(TableDef *table_def); -const char *deparse_get_tabledef_commands_concat(Oid relid); - -DeparsedHypertableCommands *deparse_get_distributed_hypertable_create_command(Hypertable *ht); - -const char *deparse_func_call(FunctionCallInfo fcinfo); -const char *deparse_oid_function_call_coll(Oid funcid, Oid collation, unsigned int num_args, ...); -const char *deparse_grant_revoke_on_database(const GrantStmt *stmt, const char *dbname); -const char *deparse_create_trigger(CreateTrigStmt *stmt); diff --git a/tsl/src/dist_backup.c b/tsl/src/dist_backup.c deleted file mode 100644 index d9e32262879..00000000000 --- a/tsl/src/dist_backup.c +++ /dev/null @@ -1,215 +0,0 @@ -/* - * 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. - */ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include "errors.h" -#include "guc.h" -#include "ts_catalog/catalog.h" -#include "debug_point.h" -#include "dist_util.h" -#include "remote/dist_commands.h" -#include "data_node.h" -#include "dist_backup.h" - -#define TS_ACCESS_NODE_TYPE "access_node" -#define TS_DATA_NODE_TYPE "data_node" - -enum -{ - Anum_restore_point_node_name = 1, - Anum_restore_point_node_type, - Anum_restore_point_lsn, - _Anum_restore_point_max -}; - -static Datum -create_restore_point_datum(TupleDesc tupdesc, const char *node_name, XLogRecPtr lsn) -{ - Datum values[_Anum_restore_point_max] = { 0 }; - bool nulls[_Anum_restore_point_max] = { false }; - HeapTuple tuple; - NameData node_name_nd; - - tupdesc = BlessTupleDesc(tupdesc); - if (node_name == NULL) - { - nulls[AttrNumberGetAttrOffset(Anum_restore_point_node_name)] = true; - values[AttrNumberGetAttrOffset(Anum_restore_point_node_type)] = - CStringGetTextDatum(TS_ACCESS_NODE_TYPE); - } - else - { - namestrcpy(&node_name_nd, node_name); - values[AttrNumberGetAttrOffset(Anum_restore_point_node_name)] = NameGetDatum(&node_name_nd); - values[AttrNumberGetAttrOffset(Anum_restore_point_node_type)] = - CStringGetTextDatum(TS_DATA_NODE_TYPE); - } - values[AttrNumberGetAttrOffset(Anum_restore_point_lsn)] = LSNGetDatum(lsn); - tuple = heap_form_tuple(tupdesc, values, nulls); - return HeapTupleGetDatum(tuple); -} - -Datum -create_distributed_restore_point(PG_FUNCTION_ARGS) -{ -#if PG16_GE - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("creating distributed restore point is not supported"), - errdetail("Multi-node is not supported anymore on PostgreSQL >= 16."))); -#else - ereport(WARNING, - (errcode(ERRCODE_WARNING_DEPRECATED_FEATURE), - errmsg("creating distributed restore point is deprecated"), - errdetail("Multi-node is deprecated and will be removed in future releases."))); -#endif - - const char *name = TextDatumGetCString(PG_GETARG_DATUM(0)); - DistCmdResult *result_cmd; - FuncCallContext *funcctx; - XLogRecPtr lsn; - - if (SRF_IS_FIRSTCALL()) - { - int name_len = strlen(name); - MemoryContext oldctx; - TupleDesc tupdesc; - char *sql; - - if (name_len >= MAXFNAMELEN) - ereport(ERROR, - (errcode(ERRCODE_INVALID_PARAMETER_VALUE), - errmsg("restore point name is too long"), - errdetail("Maximum length is %d, while provided name has %d chars.", - MAXFNAMELEN - 1, - name_len))); - - if (RecoveryInProgress()) - ereport(ERROR, - (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE), - (errmsg("recovery is in progress"), - errdetail("WAL control functions cannot be executed during recovery.")))); - - if (!XLogIsNeeded()) - ereport(ERROR, - (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE), - errmsg("WAL level '%s' is not sufficient for creating a restore point", - GetConfigOptionByName("wal_level", NULL, false)), - errhint("Set wal_level to \"replica\" or \"logical\" at server start."))); - - if (!superuser()) - ereport(ERROR, - (errcode(ERRCODE_INSUFFICIENT_PRIVILEGE), - errmsg("must be superuser to create restore point"))); - - if (!ts_guc_enable_2pc) - ereport(ERROR, - (errcode(ERRCODE_TS_OPERATION_NOT_SUPPORTED), - errmsg("two-phase commit transactions are not enabled"), - errhint("Set timescaledb.enable_2pc to TRUE."))); - - if (dist_util_membership() != DIST_MEMBER_ACCESS_NODE) - ereport(ERROR, - (errcode(ERRCODE_TS_OPERATION_NOT_SUPPORTED), - errmsg("distributed restore point must be created on the access node"), - errhint("Connect to the access node and create the distributed restore point " - "from there."))); - - /* Ensure all data nodes are available */ - if (data_node_some_unavailable()) - ereport(ERROR, (errmsg("some data nodes are not available"))); - - /* - * In order to achieve synchronization across the multinode cluster, - * we must ensure that the restore point created on the access node is - * synchronized with each data node. - * - * We must ensure that no concurrent prepared transactions are - * committed (COMMIT PREPARED) while we create the restore point. - * Otherwise, the distributed restore point might include prepared transactions - * that have committed on some data nodes but not others, leading to an - * inconsistent state when the distributed database is restored from a backup - * using the restore point. - * - * To do that we take an access exclusive lock on the remote transaction - * table, which will force any concurrent transaction - * wait during their PREPARE phase. - */ - LockRelationOid(ts_catalog_get()->tables[REMOTE_TXN].id, AccessExclusiveLock); - - /* Prevent situation when new data node added during the execution */ - LockRelationOid(ForeignServerRelationId, ExclusiveLock); - - DEBUG_WAITPOINT("create_distributed_restore_point_lock"); - - funcctx = SRF_FIRSTCALL_INIT(); - oldctx = MemoryContextSwitchTo(funcctx->multi_call_memory_ctx); - - if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE) - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("function returning record called in context " - "that cannot accept type record"))); - - /* Create local restore point and on each data node */ - lsn = XLogRestorePoint(name); - - sql = psprintf("SELECT pg_create_restore_point AS lsn " - "FROM " - "pg_catalog.pg_create_restore_point(%s)", - quote_literal_cstr(name)); - - result_cmd = ts_dist_cmd_invoke_on_all_data_nodes(sql); - - funcctx->attinmeta = TupleDescGetAttInMetadata(tupdesc); - funcctx->user_fctx = result_cmd; - - MemoryContextSwitchTo(oldctx); - - /* Return access node restore point first */ - SRF_RETURN_NEXT(funcctx, create_restore_point_datum(tupdesc, NULL, lsn)); - } - - funcctx = SRF_PERCALL_SETUP(); - result_cmd = funcctx->user_fctx; - - /* Return data node restore point data */ - if (result_cmd) - { - int result_index = funcctx->call_cntr - 1; - - if (result_index < (int) ts_dist_cmd_response_count(result_cmd)) - { - const char *node_name; - PGresult *result = - ts_dist_cmd_get_result_by_index(result_cmd, result_index, &node_name); - AttInMetadata *attinmeta = funcctx->attinmeta; - const int lsn_attr_pos = AttrNumberGetAttrOffset(Anum_restore_point_lsn); - - lsn = DatumGetLSN(InputFunctionCall(&attinmeta->attinfuncs[lsn_attr_pos], - PQgetvalue(result, 0, 0), - attinmeta->attioparams[lsn_attr_pos], - attinmeta->atttypmods[lsn_attr_pos])); - - SRF_RETURN_NEXT(funcctx, - create_restore_point_datum(attinmeta->tupdesc, node_name, lsn)); - } - - ts_dist_cmd_close_response(result_cmd); - } - - SRF_RETURN_DONE(funcctx); -} diff --git a/tsl/src/dist_backup.h b/tsl/src/dist_backup.h deleted file mode 100644 index dacd64797e7..00000000000 --- a/tsl/src/dist_backup.h +++ /dev/null @@ -1,10 +0,0 @@ -/* - * 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. - */ -#pragma once - -#include - -extern Datum create_distributed_restore_point(PG_FUNCTION_ARGS); diff --git a/tsl/src/dist_util.c b/tsl/src/dist_util.c deleted file mode 100644 index d67f693615b..00000000000 --- a/tsl/src/dist_util.c +++ /dev/null @@ -1,431 +0,0 @@ -/* - * 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. - */ - -#include -#include -#include -#include -#include -#include - -#include "ts_catalog/catalog.h" -#include "config.h" -#include "dist_util.h" -#include "errors.h" -#include "funcapi.h" -#include "loader/seclabel.h" -#include "ts_catalog/metadata.h" -#include "remote/dist_commands.h" -#ifdef USE_TELEMETRY -#include "telemetry/telemetry_metadata.h" -#endif -#include "utils/uuid.h" -#include "debug_assert.h" - -static Datum dist_util_remote_srf_query(FunctionCallInfo fcinfo, const char *node_name, - const char *sql_query); - -/* UUID associated with remote connection */ -static pg_uuid_t *peer_dist_id = NULL; - -static bool dist_util_set_id_with_uuid_check(Datum dist_id, bool check_uuid); - -/* Requires non-null arguments */ -static bool -uuid_matches(Datum a, Datum b) -{ - Assert(DatumGetUUIDP(a) != NULL && DatumGetUUIDP(b) != NULL); - return DatumGetBool(DirectFunctionCall2(uuid_eq, a, b)); -} - -static Datum -local_get_dist_id(bool *isnull) -{ - return ts_metadata_get_value(METADATA_DISTRIBUTED_UUID_KEY_NAME, UUIDOID, isnull); -} - -static Datum -local_get_uuid(bool *isnull) -{ - return ts_metadata_get_value(METADATA_UUID_KEY_NAME, UUIDOID, isnull); -} - -DistUtilMembershipStatus -dist_util_membership(void) -{ - bool isnull; - Datum dist_id = local_get_dist_id(&isnull); - - if (isnull) - return DIST_MEMBER_NONE; - else if (uuid_matches(dist_id, local_get_uuid(&isnull))) - return DIST_MEMBER_ACCESS_NODE; - else - return DIST_MEMBER_DATA_NODE; -} - -const char * -dist_util_membership_str(DistUtilMembershipStatus status) -{ - static const char *dist_util_membership_status_str[] = { [DIST_MEMBER_NONE] = "none", - [DIST_MEMBER_DATA_NODE] = "data node", - [DIST_MEMBER_ACCESS_NODE] = - "access node" }; - return dist_util_membership_status_str[status]; -} - -static void -seclabel_set_dist_uuid(Oid dbid, Datum dist_uuid) -{ - ObjectAddress dbobj; - Datum uuid_string = DirectFunctionCall1(uuid_out, dist_uuid); - const char *label = psprintf("%s%c%s", - SECLABEL_DIST_TAG, - SECLABEL_DIST_TAG_SEPARATOR, - DatumGetCString(uuid_string)); - - ObjectAddressSet(dbobj, DatabaseRelationId, dbid); - SetSecurityLabel(&dbobj, SECLABEL_DIST_PROVIDER, label); -} - -void -dist_util_set_as_access_node() -{ - bool isnull; - dist_util_set_id_with_uuid_check(local_get_uuid(&isnull), false); - - /* - * Set security label to mark current database as the access node database. - * - * Presence of this label is used as a flag to send NOTICE message - * after a DROP DATABASE operation completion. - */ - seclabel_set_dist_uuid(MyDatabaseId, local_get_dist_id(NULL)); -} - -bool -dist_util_set_id(Datum dist_id) -{ - return dist_util_set_id_with_uuid_check(dist_id, true); -} - -static bool -dist_util_set_id_with_uuid_check(Datum dist_id, bool check_uuid) -{ - bool isnull; - if (dist_util_membership() != DIST_MEMBER_NONE) - { - if (uuid_matches(dist_id, dist_util_get_id())) - return false; - else - ereport(ERROR, - (errcode(ERRCODE_TS_DATA_NODE_ASSIGNMENT_ALREADY_EXISTS), - (errmsg("database is already a member of a distributed database")))); - } - - Datum uuid = local_get_uuid(&isnull); - if (check_uuid && !isnull && uuid_matches(dist_id, uuid)) - ereport(ERROR, - (errcode(ERRCODE_TS_DATA_NODE_INVALID_CONFIG), - (errmsg("cannot add the current database as a data node to itself"), - errdetail("Adding the current database as a data node to itself would create a " - "cycle. Use a different instance or database for the data node."), - errhint("Check that the 'port' parameter refers to a different " - "instance or that the 'database' parameter refers to a " - "different database.")))); - - ts_metadata_insert(METADATA_DISTRIBUTED_UUID_KEY_NAME, dist_id, UUIDOID, true); - return true; -} - -Datum -dist_util_get_id() -{ - return local_get_dist_id(NULL); -} - -const char * -dist_util_internal_key_name() -{ - return METADATA_DISTRIBUTED_UUID_KEY_NAME; -} - -bool -dist_util_remove_from_db() -{ - if (dist_util_membership() != DIST_MEMBER_NONE) - { - CatalogSecurityContext sec_ctx; - - ts_catalog_database_info_become_owner(ts_catalog_database_info_get(), &sec_ctx); - ts_metadata_drop(METADATA_DISTRIBUTED_UUID_KEY_NAME); - ts_catalog_restore_user(&sec_ctx); - - return true; - } - - return false; -} - -void -dist_util_set_peer_id(Datum dist_id) -{ - pg_uuid_t *uuid = DatumGetUUIDP(dist_id); - static pg_uuid_t id; - - if (peer_dist_id != NULL) - ereport(ERROR, - (errcode(ERRCODE_TS_INTERNAL_ERROR), (errmsg("distributed peer ID already set")))); - - memcpy(id.data, uuid->data, UUID_LEN); - peer_dist_id = &id; -} - -bool -dist_util_is_access_node_session_on_data_node(void) -{ - Datum dist_id; - - if (dist_util_membership() == DIST_MEMBER_NONE) - return false; - - if (!peer_dist_id) - return false; - - dist_id = local_get_dist_id(NULL); - return uuid_matches(UUIDPGetDatum(peer_dist_id), dist_id); -} - -Datum -dist_util_remote_hypertable_info(PG_FUNCTION_ARGS) -{ - char *node_name; - StringInfo query_str = makeStringInfo(); - /* Strict function */ - Name schema_name = PG_GETARG_NAME(1); - Name table_name = PG_GETARG_NAME(2); - Ensure(!PG_ARGISNULL(0) && !PG_ARGISNULL(1) && !PG_ARGISNULL(2), - "three non-null arguments required"); - appendStringInfo(query_str, - "SELECT * from _timescaledb_functions.hypertable_local_size( %s, %s );", - quote_literal_cstr(NameStr(*schema_name)), - quote_literal_cstr(NameStr(*table_name))); - node_name = PG_GETARG_NAME(0)->data; - return dist_util_remote_srf_query(fcinfo, node_name, query_str->data); -} - -Datum -dist_util_remote_chunk_info(PG_FUNCTION_ARGS) -{ - char *node_name; - StringInfo query_str = makeStringInfo(); - /* Strict function */ - Name schema_name = PG_GETARG_NAME(1); - Name table_name = PG_GETARG_NAME(2); - Assert(!PG_ARGISNULL(0) && !PG_ARGISNULL(1) && !PG_ARGISNULL(2)); - appendStringInfo(query_str, - "SELECT * from _timescaledb_functions.chunks_local_size( %s, %s );", - quote_literal_cstr(NameStr(*schema_name)), - quote_literal_cstr(NameStr(*table_name))); - node_name = NameStr(*PG_GETARG_NAME(0)); - return dist_util_remote_srf_query(fcinfo, node_name, query_str->data); -} - -Datum -dist_util_remote_compressed_chunk_info(PG_FUNCTION_ARGS) -{ - char *node_name; - StringInfo query_str; - Name schema_name, table_name; - /* Strict function */ - if (PG_NARGS() != 3 || PG_ARGISNULL(0) || PG_ARGISNULL(1) || PG_ARGISNULL(2)) - PG_RETURN_NULL(); - schema_name = PG_GETARG_NAME(1); - table_name = PG_GETARG_NAME(2); - query_str = makeStringInfo(); - appendStringInfo(query_str, - "SELECT * from _timescaledb_functions.compressed_chunk_local_stats( %s, %s );", - quote_literal_cstr(NameStr(*schema_name)), - quote_literal_cstr(NameStr(*table_name))); - node_name = NameStr(*PG_GETARG_NAME(0)); - return dist_util_remote_srf_query(fcinfo, node_name, query_str->data); -} - -Datum -dist_util_remote_hypertable_index_info(PG_FUNCTION_ARGS) -{ - char *node_name; - StringInfo query_str; - Name schema_name, index_name; - /* Strict function */ - if (PG_NARGS() != 3 || PG_ARGISNULL(0) || PG_ARGISNULL(1) || PG_ARGISNULL(2)) - PG_RETURN_NULL(); - schema_name = PG_GETARG_NAME(1); - index_name = PG_GETARG_NAME(2); - query_str = makeStringInfo(); - appendStringInfo(query_str, - "SELECT * from _timescaledb_functions.indexes_local_size( %s, %s );", - quote_literal_cstr(NameStr(*schema_name)), - quote_literal_cstr(NameStr(*index_name))); - node_name = NameStr(*PG_GETARG_NAME(0)); - return dist_util_remote_srf_query(fcinfo, node_name, query_str->data); -} - -void -validate_data_node_settings(void) -{ - switch (dist_util_membership()) - { - case DIST_MEMBER_DATA_NODE: - ereport(ERROR, - (errcode(ERRCODE_TS_DATA_NODE_INVALID_CONFIG), - errmsg("node is already a data node"))); - break; - - case DIST_MEMBER_ACCESS_NODE: - ereport(ERROR, - (errcode(ERRCODE_TS_DATA_NODE_INVALID_CONFIG), - errmsg("node is already an access node"))); - break; - - default: - /* Nothing to do */ - break; - } - - /* - * We skip printing the warning if we have already printed the error. - */ - if (max_prepared_xacts == 0) - ereport(ERROR, - (errcode(ERRCODE_TS_DATA_NODE_INVALID_CONFIG), - errmsg("prepared transactions need to be enabled"), - errhint("Configuration parameter max_prepared_transactions must be set >0 " - "(changes will require restart)."), - errdetail("Parameter max_prepared_transactions=%d.", max_prepared_xacts))); - else if (max_prepared_xacts < MaxConnections) - ereport(WARNING, - (errcode(ERRCODE_TS_DATA_NODE_INVALID_CONFIG), - errmsg("max_prepared_transactions is set low"), - errhint("It is recommended that max_prepared_transactions >= max_connections " - "(changes will require restart)."), - errdetail("Parameters max_prepared_transactions=%d, max_connections=%d.", - max_prepared_xacts, - MaxConnections))); -} - -/* - * Check that the data node version is compatible with the version on this - * node by checking that all of the following are true: - * - * - The major version is identical or newer on the data node as compared to the access node. - * - The minor version on the data node is newer or the same as on the access - * node in case major version is identical. - * - * We explicitly do *not* check the patch version since changes between patch - * versions will only fix bugs and there should be no problem using an older - * patch version of the extension on the data node. - */ -bool -dist_util_is_compatible_version(const char *data_node_version, const char *access_node_version) -{ - unsigned int data_node_major, data_node_minor, data_node_patch; - unsigned int access_node_major, access_node_minor, access_node_patch; - - Ensure(data_node_version, "missing data node version when checking compatibility"); - - if (sscanf(data_node_version, - "%u.%u.%u", - &data_node_major, - &data_node_minor, - &data_node_patch) != 3) - ereport(ERROR, - (errcode(ERRCODE_INTERNAL_ERROR), - errmsg("invalid data node version %s", data_node_version))); - if (sscanf(access_node_version, - "%u.%u.%u", - &access_node_major, - &access_node_minor, - &access_node_patch) != 3) - ereport(ERROR, - (errcode(ERRCODE_INTERNAL_ERROR), - errmsg("invalid access node version %s", access_node_version))); - - /* if major versions are same, compare minor versions */ - if (data_node_major == access_node_major) - /* - * if minor versions are same (or newer on DN), we ignore patch versions and - * declare compatibility. If minor version of DN is lower then it's not - * compatible - */ - return (data_node_minor >= access_node_minor) ? true : false; - else - /* if DN major version is newer, then still compatible. Else not */ - return (data_node_major > access_node_major); -} - -/* - * Returns results from SRFs on remote nodes - * Pass the fcinfo information from the original PG function - * args: node_name is NAME and sql_query is of TEXT type - */ -static Datum -dist_util_remote_srf_query(FunctionCallInfo fcinfo, const char *node_name, const char *sql_query) -{ - FuncCallContext *funcctx; - PGresult *result; - - Assert(node_name != NULL && sql_query != NULL); - if (SRF_IS_FIRSTCALL()) - { - MemoryContext oldcontext; - TupleDesc tupdesc; - - funcctx = SRF_FIRSTCALL_INIT(); - oldcontext = MemoryContextSwitchTo(funcctx->multi_call_memory_ctx); - - if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE) - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("function returning record called in context " - "that cannot accept type record"))); - - funcctx->user_fctx = - ts_dist_cmd_invoke_on_data_nodes(sql_query, list_make1((void *) node_name), true); - funcctx->attinmeta = TupleDescGetAttInMetadata(tupdesc); - - MemoryContextSwitchTo(oldcontext); - } - funcctx = SRF_PERCALL_SETUP(); - result = ts_dist_cmd_get_result_by_node_name(funcctx->user_fctx, node_name); - - if (funcctx->call_cntr < (uint64) PQntuples(result)) - { - HeapTuple tuple; - char **fields = palloc(sizeof(char *) * PQnfields(result)); - int i; - - for (i = 0; i < PQnfields(result); ++i) - { - if (PQgetisnull(result, funcctx->call_cntr, i) != 1) - { - fields[i] = PQgetvalue(result, funcctx->call_cntr, i); - - if (fields[i][0] == '\0') - fields[i] = NULL; - } - else - fields[i] = NULL; - } - - tuple = BuildTupleFromCStrings(funcctx->attinmeta, fields); - - SRF_RETURN_NEXT(funcctx, HeapTupleGetDatum(tuple)); - } - - ts_dist_cmd_close_response(funcctx->user_fctx); - SRF_RETURN_DONE(funcctx); -} diff --git a/tsl/src/dist_util.h b/tsl/src/dist_util.h deleted file mode 100644 index df3b4848aa2..00000000000 --- a/tsl/src/dist_util.h +++ /dev/null @@ -1,39 +0,0 @@ -/* - * 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. - */ -#pragma once - -#include -#include -#include "hypertable.h" - -typedef enum DistUtilMembershipStatus -{ - DIST_MEMBER_NONE, /* Database doesn't belong to a distributed database */ - DIST_MEMBER_DATA_NODE, /* Database is a data node */ - DIST_MEMBER_ACCESS_NODE /* Database is an access node */ -} DistUtilMembershipStatus; - -DistUtilMembershipStatus dist_util_membership(void); -const char *dist_util_membership_str(DistUtilMembershipStatus status); - -void dist_util_set_as_access_node(void); -bool dist_util_set_id(Datum dist_id); -Datum dist_util_get_id(void); -bool dist_util_remove_from_db(void); - -const char *dist_util_internal_key_name(void); - -void dist_util_set_peer_id(Datum dist_id); -bool dist_util_is_access_node_session_on_data_node(void); - -Datum dist_util_remote_hypertable_info(PG_FUNCTION_ARGS); -Datum dist_util_remote_chunk_info(PG_FUNCTION_ARGS); -Datum dist_util_remote_compressed_chunk_info(PG_FUNCTION_ARGS); -Datum dist_util_remote_hypertable_index_info(PG_FUNCTION_ARGS); - -void validate_data_node_settings(void); -bool dist_util_is_compatible_version(const char *data_node_version, - const char *access_node_version); diff --git a/tsl/src/hypertable.c b/tsl/src/hypertable.c deleted file mode 100644 index 5e9962333fb..00000000000 --- a/tsl/src/hypertable.c +++ /dev/null @@ -1,276 +0,0 @@ -/* - * 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. - */ - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include "dimension.h" -#include "errors.h" -#include "hypertable.h" -#include "utils.h" -#include "hypertable_cache.h" -#include "chunk.h" -#include "ts_catalog/chunk_data_node.h" - -#include -#include - -#include "data_node.h" -#include "deparse.h" -#include "remote/dist_commands.h" -#include "compat/compat.h" -#include "ts_catalog/hypertable_data_node.h" -#include "extension.h" - -static List * -data_node_append(List *data_nodes, int32 hypertable_id, const char *node_name, - int32 node_hypertable_id, bool block_chunks) -{ - ForeignServer *server = data_node_get_foreign_server(node_name, ACL_NO_CHECK, true, false); - HypertableDataNode *hdn = palloc0(sizeof(HypertableDataNode)); - - hdn->fd.hypertable_id = hypertable_id; - namestrcpy(&hdn->fd.node_name, node_name); - hdn->fd.node_hypertable_id = node_hypertable_id; - hdn->foreign_server_oid = server->serverid; - hdn->fd.block_chunks = block_chunks; - - return lappend(data_nodes, hdn); -} - -/* Returns the remote hypertable ids for the data_nodes (in the same order) - */ -static List * -hypertable_create_data_node_tables(int32 hypertable_id, List *data_nodes) -{ - Hypertable *ht = ts_hypertable_get_by_id(hypertable_id); - ListCell *cell; - List *remote_ids = NIL; - DistCmdResult *dist_res; - DeparsedHypertableCommands *commands = deparse_get_distributed_hypertable_create_command(ht); - - foreach (cell, deparse_get_tabledef_commands(ht->main_table_relid)) - ts_dist_cmd_run_on_data_nodes(lfirst(cell), data_nodes, true); - - dist_res = ts_dist_cmd_invoke_on_data_nodes(commands->table_create_command, data_nodes, true); - foreach (cell, data_nodes) - { - PGresult *res = ts_dist_cmd_get_result_by_node_name(dist_res, lfirst(cell)); - - Assert(PQntuples(res) == 1); - Assert(PQnfields(res) == AttrNumberGetAttrOffset(_Anum_create_hypertable_max)); - remote_ids = - lappend(remote_ids, - (void *) Int32GetDatum(atoi( - PQgetvalue(res, 0, AttrNumberGetAttrOffset(Anum_create_hypertable_id))))); - } - ts_dist_cmd_close_response(dist_res); - - foreach (cell, commands->dimension_add_commands) - ts_dist_cmd_run_on_data_nodes(lfirst(cell), data_nodes, true); - - foreach (cell, commands->grant_commands) - ts_dist_cmd_run_on_data_nodes(lfirst(cell), data_nodes, true); - - return remote_ids; -} - -/* - * Assign data nodes to a hypertable. - * - * Given a list of data node names, add mappings to ensure the - * hypertable is distributed across those nodes. - * - * Returns a list of HypertableDataNode objects that correspond to the given - * data node names. - */ -List * -hypertable_assign_data_nodes(int32 hypertable_id, List *nodes) -{ - ListCell *lc; - List *assigned_nodes = NIL; - List *remote_ids = hypertable_create_data_node_tables(hypertable_id, nodes); - ListCell *id_cell; - - Assert(nodes->length == remote_ids->length); - forboth (lc, nodes, id_cell, remote_ids) - { - assigned_nodes = - data_node_append(assigned_nodes, hypertable_id, lfirst(lc), lfirst_int(id_cell), false); - } - - ts_hypertable_data_node_insert_multi(assigned_nodes); - - return assigned_nodes; -} - -/* - * Validate data nodes when creating a new hypertable. - * - * The function is passed the explicit array of data nodes given by the user, - * if any. - * - * If the data node array is NULL (no data nodes specified), we return all - * data nodes that the user is allowed to use. - * - */ -List * -hypertable_get_and_validate_data_nodes(ArrayType *nodearr) -{ - bool fail_on_aclcheck = nodearr != NULL; - List *data_nodes = NIL; - int num_data_nodes; - List *all_data_nodes = NIL; - - /* If the user explicitly specified a set of data nodes (data_node_arr is - * non-NULL), we validate the given array and fail if the user doesn't - * have USAGE on all of them. Otherwise, we get a list of all - * database-configured data nodes that the user has USAGE on. */ - data_nodes = data_node_get_filtered_node_name_list(nodearr, ACL_USAGE, fail_on_aclcheck); - num_data_nodes = list_length(data_nodes); - - if (NULL == nodearr) - { - /* No explicit set of data nodes given. Check if there are any data - * nodes that the user cannot use due to lack of permissions and - * raise a NOTICE if some of them cannot be used. */ - all_data_nodes = data_node_get_node_name_list(); - int num_nodes_not_used = list_length(all_data_nodes) - list_length(data_nodes); - - if (num_nodes_not_used > 0) - ereport(NOTICE, - (errmsg("%d of %d data nodes not used by this hypertable due to lack of " - "permissions", - num_nodes_not_used, - list_length(all_data_nodes)), - errhint("Grant USAGE on data nodes to attach them to a hypertable."))); - } - - /* - * In this case, if we couldn't find any valid data nodes to assign, it - * means that they do not have the right permissions on the data nodes or - * that there were no data nodes to assign. Depending on the case, we - * print different error details and hints to aid the user. - */ - if (num_data_nodes == 0) - ereport(ERROR, - (errcode(ERRCODE_TS_INSUFFICIENT_NUM_DATA_NODES), - errmsg("no data nodes can be assigned to the hypertable"), - errdetail(list_length(all_data_nodes) == 0 ? - "No data nodes where available to assign to the hypertable." : - "Data nodes exist, but none have USAGE privilege."), - errhint(list_length(all_data_nodes) == 0 ? - "Add data nodes to the database." : - "Grant USAGE on data nodes to attach them to the hypertable."))); - - if (num_data_nodes == 1) - ereport(WARNING, - (errmsg("only one data node was assigned to the hypertable"), - errdetail("A distributed hypertable should have at least two data nodes for best " - "performance."), - errhint( - list_length(all_data_nodes) == 1 ? - "Add more data nodes to the database and attach them to the hypertable." : - "Grant USAGE on data nodes and attach them to the hypertable."))); - - if (num_data_nodes > MAX_NUM_HYPERTABLE_DATA_NODES) - ereport(ERROR, - (errcode(ERRCODE_INVALID_PARAMETER_VALUE), - errmsg("max number of data nodes exceeded"), - errhint("The number of data nodes cannot exceed %d.", - MAX_NUM_HYPERTABLE_DATA_NODES))); - - return data_nodes; -} - -void -hypertable_make_distributed(Hypertable *ht, List *data_node_names) -{ - hypertable_assign_data_nodes(ht->fd.id, data_node_names); -} - -static bool -hypertable_is_underreplicated(Hypertable *const ht, const int16 replication_factor) -{ - ListCell *lc; - List *chunks = find_inheritance_children(ht->main_table_relid, NoLock); - - Assert(hypertable_is_distributed(ht)); - - foreach (lc, chunks) - { - Oid chunk_oid = lfirst_oid(lc); - Chunk *chunk = ts_chunk_get_by_relid(chunk_oid, true); - List *replicas = ts_chunk_data_node_scan_by_chunk_id(chunk->fd.id, CurrentMemoryContext); - - Assert(get_rel_relkind(chunk_oid) == RELKIND_FOREIGN_TABLE); - - if (list_length(replicas) < replication_factor) - return true; - } - return false; -} - -static void -update_replication_factor(Hypertable *const ht, const int32 replication_factor_in) -{ - const int16 replication_factor = - ts_validate_replication_factor(get_rel_name(ht->main_table_relid), - replication_factor_in, - list_length(ht->data_nodes)); - - ht->fd.replication_factor = replication_factor; - ts_hypertable_update(ht); - - if (hypertable_is_underreplicated(ht, replication_factor)) - ereport(WARNING, - (errcode(ERRCODE_WARNING), - errmsg("hypertable \"%s\" is under-replicated", NameStr(ht->fd.table_name)), - errdetail("Some chunks have less than %d replicas.", replication_factor))); - - ts_hypertable_update_dimension_partitions(ht); -} - -Datum -hypertable_set_replication_factor(PG_FUNCTION_ARGS) -{ - const Oid table_relid = PG_ARGISNULL(0) ? InvalidOid : PG_GETARG_OID(0); - const int32 replication_factor_in = PG_ARGISNULL(1) ? 0 : PG_GETARG_INT32(1); - Cache *hcache; - Hypertable *ht; - - TS_PREVENT_FUNC_IF_READ_ONLY(); - - if (!OidIsValid(table_relid)) - ereport(ERROR, - (errcode(ERRCODE_INVALID_PARAMETER_VALUE), - errmsg("invalid hypertable: cannot be NULL"))); - - hcache = ts_hypertable_cache_pin(); - ht = ts_hypertable_cache_get_entry(hcache, table_relid, CACHE_FLAG_NONE); - - if (!hypertable_is_distributed(ht)) - ereport(ERROR, - (errcode(ERRCODE_TS_HYPERTABLE_NOT_DISTRIBUTED), - errmsg("hypertable \"%s\" is not distributed", get_rel_name(table_relid)))); - - update_replication_factor(ht, replication_factor_in); - - ts_cache_release(hcache); - - PG_RETURN_VOID(); -} diff --git a/tsl/src/hypertable.h b/tsl/src/hypertable.h deleted file mode 100644 index 3640710cd56..00000000000 --- a/tsl/src/hypertable.h +++ /dev/null @@ -1,20 +0,0 @@ -/* - * 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. - */ -#pragma once - -#include -#include "dimension.h" -#include "config.h" -#include "ts_catalog/catalog.h" - -/* We cannot make use of more data nodes than we have slices in closed (space) - * dimensions, and the value for number of slices is an int16. */ -#define MAX_NUM_HYPERTABLE_DATA_NODES PG_INT16_MAX - -extern void hypertable_make_distributed(Hypertable *ht, List *data_node_names); -extern List *hypertable_assign_data_nodes(int32 hypertable_id, List *nodes); -extern List *hypertable_get_and_validate_data_nodes(ArrayType *nodearr); -extern Datum hypertable_set_replication_factor(PG_FUNCTION_ARGS); diff --git a/tsl/src/init.c b/tsl/src/init.c index 866af366dd9..bf4d489fcef 100644 --- a/tsl/src/init.c +++ b/tsl/src/init.c @@ -33,8 +33,6 @@ #include "continuous_aggs/repair.h" #include "continuous_aggs/utils.h" #include "cross_module_fn.h" -#include "data_node.h" -#include "dist_util.h" #include "export.h" #include "hypertable.h" #include "license_guc.h" @@ -45,15 +43,7 @@ #include "partialize_finalize.h" #include "planner.h" #include "process_utility.h" -#include "process_utility.h" -#include "remote/connection_cache.h" -#include "remote/connection.h" -#include "remote/dist_commands.h" -#include "remote/dist_txn.h" -#include "remote/txn_id.h" -#include "remote/txn_resolve.h" #include "reorder.h" -#include "dist_backup.h" #ifdef PG_MODULE_MAGIC PG_MODULE_MAGIC; @@ -65,12 +55,6 @@ PG_MODULE_MAGIC; extern void PGDLLEXPORT _PG_init(void); -static void -cache_syscache_invalidate(Datum arg, int cacheid, uint32 hashvalue) -{ - remote_connection_cache_invalidate_callback(arg, cacheid, hashvalue); -} - /* * Cross module function initialization. * @@ -148,9 +132,6 @@ CrossModuleFunctions tsl_cm_functions = { .continuous_agg_validate_query = continuous_agg_validate_query, .invalidation_cagg_log_add_entry = tsl_invalidation_cagg_log_add_entry, .invalidation_hyper_log_add_entry = tsl_invalidation_hyper_log_add_entry, - .remote_invalidation_log_delete = remote_invalidation_log_delete, - .drop_dist_ht_invalidation_trigger = tsl_drop_dist_ht_invalidation_trigger, - .remote_drop_dist_ht_invalidation_trigger = remote_drop_dist_ht_invalidation_trigger, .invalidation_process_hypertable_log = tsl_invalidation_process_hypertable_log, .invalidation_process_cagg_log = tsl_invalidation_process_cagg_log, .cagg_try_repair = tsl_cagg_try_repair, @@ -191,7 +172,6 @@ CrossModuleFunctions tsl_cm_functions = { .chunk_get_relstats = chunk_api_get_chunk_relstats, .chunk_get_colstats = chunk_api_get_chunk_colstats, .chunk_create_empty_table = chunk_create_empty_table, - .cache_syscache_invalidate = cache_syscache_invalidate, .recompress_chunk_segmentwise = tsl_recompress_chunk_segmentwise, .get_compressed_chunk_index_for_recompression = tsl_get_compressed_chunk_index_for_recompression, @@ -200,8 +180,6 @@ CrossModuleFunctions tsl_cm_functions = { static void ts_module_cleanup_on_pg_exit(int code, Datum arg) { - _remote_dist_txn_fini(); - _remote_connection_cache_fini(); _continuous_aggs_cache_inval_fini(); } @@ -218,8 +196,6 @@ ts_module_init(PG_FUNCTION_ARGS) _continuous_aggs_cache_inval_init(); _decompress_chunk_init(); _skip_scan_init(); - _remote_connection_cache_init(); - _remote_dist_txn_init(); /* Register a cleanup function to be called when the backend exits */ if (register_proc_exit) on_proc_exit(ts_module_cleanup_on_pg_exit, 0); @@ -242,6 +218,4 @@ _PG_init(void) * relative to the other libraries. */ ts_license_enable_module_loading(); - - _remote_connection_init(); } diff --git a/tsl/src/nodes/decompress_chunk/decompress_chunk.c b/tsl/src/nodes/decompress_chunk/decompress_chunk.c index 1e4f10b4ade..a8aa8662ba2 100644 --- a/tsl/src/nodes/decompress_chunk/decompress_chunk.c +++ b/tsl/src/nodes/decompress_chunk/decompress_chunk.c @@ -629,7 +629,7 @@ static void add_chunk_sorted_paths(PlannerInfo *root, RelOptInfo *chunk_rel, Hypertable *ht, Index ht_relid, Path *decompress_chunk_path, Path *compressed_path) { - if (root->query_pathkeys == NIL || hypertable_is_distributed(ht)) + if (root->query_pathkeys == NIL) return; /* We are only interested in regular (i.e., non index) paths */ diff --git a/tsl/src/nodes/gapfill/gapfill.h b/tsl/src/nodes/gapfill/gapfill.h index 45fc0a66338..300cb0f891b 100644 --- a/tsl/src/nodes/gapfill/gapfill.h +++ b/tsl/src/nodes/gapfill/gapfill.h @@ -13,7 +13,6 @@ #define GAPFILL_LOCF_FUNCTION "locf" #define GAPFILL_INTERPOLATE_FUNCTION "interpolate" -bool gapfill_in_expression(Expr *node); void plan_add_gapfill(PlannerInfo *root, RelOptInfo *group_rel); void gapfill_adjust_window_targetlist(PlannerInfo *root, RelOptInfo *input_rel, RelOptInfo *output_rel); diff --git a/tsl/src/nodes/gapfill/gapfill_plan.c b/tsl/src/nodes/gapfill/gapfill_plan.c index e2949ed9823..36f967b78cc 100644 --- a/tsl/src/nodes/gapfill/gapfill_plan.c +++ b/tsl/src/nodes/gapfill/gapfill_plan.c @@ -77,19 +77,6 @@ gapfill_function_walker(Node *node, gapfill_walker_context *context) return expression_tree_walker((Node *) node, gapfill_function_walker, context); } -/* - * Check if the given expression contains call to time_bucket_gapfill - */ -bool -gapfill_in_expression(Expr *node) -{ - gapfill_walker_context context = { .call.node = NULL, .count = 0 }; - - gapfill_function_walker((Node *) node, &context); - - return context.count > 0; -} - /* * Find locf/interpolate function call */ diff --git a/tsl/src/remote/CMakeLists.txt b/tsl/src/remote/CMakeLists.txt deleted file mode 100644 index b1990024a1d..00000000000 --- a/tsl/src/remote/CMakeLists.txt +++ /dev/null @@ -1,22 +0,0 @@ -set(SOURCES - ${CMAKE_CURRENT_SOURCE_DIR}/async.c - ${CMAKE_CURRENT_SOURCE_DIR}/connection.c - ${CMAKE_CURRENT_SOURCE_DIR}/connection_cache.c - ${CMAKE_CURRENT_SOURCE_DIR}/copy_fetcher.c - ${CMAKE_CURRENT_SOURCE_DIR}/cursor_fetcher.c - ${CMAKE_CURRENT_SOURCE_DIR}/data_fetcher.c - ${CMAKE_CURRENT_SOURCE_DIR}/data_format.c - ${CMAKE_CURRENT_SOURCE_DIR}/dist_txn.c - ${CMAKE_CURRENT_SOURCE_DIR}/dist_commands.c - ${CMAKE_CURRENT_SOURCE_DIR}/prepared_statement_fetcher.c - ${CMAKE_CURRENT_SOURCE_DIR}/copy_fetcher.c - ${CMAKE_CURRENT_SOURCE_DIR}/stmt_params.c - ${CMAKE_CURRENT_SOURCE_DIR}/tuplefactory.c - ${CMAKE_CURRENT_SOURCE_DIR}/txn.c - ${CMAKE_CURRENT_SOURCE_DIR}/txn_id.c - ${CMAKE_CURRENT_SOURCE_DIR}/txn_resolve.c - ${CMAKE_CURRENT_SOURCE_DIR}/txn_store.c - ${CMAKE_CURRENT_SOURCE_DIR}/utils.c) -target_sources(${TSL_LIBRARY_NAME} PRIVATE ${SOURCES}) -target_include_directories(${TSL_LIBRARY_NAME} - PRIVATE ${PG_INCLUDEDIR} ${PG_INCLUDEDIR}/postgresql) diff --git a/tsl/src/remote/README.md b/tsl/src/remote/README.md deleted file mode 100644 index b47fa66653f..00000000000 --- a/tsl/src/remote/README.md +++ /dev/null @@ -1,37 +0,0 @@ -# Remote Connection/Transaction Management - -This module handles connection and transaction management to remote nodes. - -# Connection Management - -The `connection` object allows you to create raw connections, report connection errors, -and make synchronous requests on those connections. - -The `async` module allows you to make asynchronous requests on connections and manage those requests. - -The (backend-local) `connection_cache` maintains a cache of connections to data nodes that survive -transaction boundaries. This prevents each transaction from having to incur the overhead of creating new -connections. - -# Transaction Management - -The main object in transaction management is `dist_txn`. Namely the `remote_dist_txn_get_connection` function -allows a user to obtain connections for remote nodes that are tied to the current local transaction in that they -get automatically committed/aborted when the local transaction commits/aborts. This fate-sharing is achieved by -using xact hooks to automatically perform commit/abort operations on the remote node when a local transaction -event happens. We allow either 2-pc or 1-pc commit protocols to be used (controlled by a guc). When using the -2-pc variant, fate-sharing is guaranteed, while in 1-pc it is best effort. - -`dist_txn` uses a `txn_store` to keep a map of `server_id`+`user_id` to `txn`s. A `txn` corresponds to the state maintained -for every remote connection that is part of a `dist_txn`. Thus a `dist_txn` contains multiple `txn`s. Note that since -a `txn` represents a remote connection and a txn on the remote side, it maintains a consistent snapshot throughout -its lifetime. We make sure that there is a unique `txn` for each `server_id`+`user_id` so that each local user maintains a -consistent snapshot to each remote node within one distributed txn. However, snapshots are not consistent across -mappings, even to the same data node. - -For 2-pc transactions, two other objects are used: -- `txn_id` represents the GID used in `PREPARE TRANSACTION`, `COMMIT PREPARED`, and `ROLLBACK PREPARED` SQL commands used in the 2-pc protocol. -- `txn_resolve` allows you to get the status of a distributed transaction based on a `txn_id`. It is used to recover the state of nodes - that failed in the middle of the 2-pc protocol (see the `ts_remote_txn_heal_data_node` function). - -txn_resolve.h has a good description of the 2-pc protocol variant we use. diff --git a/tsl/src/remote/async.c b/tsl/src/remote/async.c deleted file mode 100644 index ab2bbe9f593..00000000000 --- a/tsl/src/remote/async.c +++ /dev/null @@ -1,986 +0,0 @@ -/* - * 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. - */ - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include - -#include -#include "async.h" -#include "connection.h" -#include "utils.h" - -/** - * State machine for AsyncRequest: - * - * +-------------+ +--------------+ +--------------+ - * | | | | | | - * | DEFERRED +---------->+ EXECUTING +------->+ COMPLETED | - * | | | | | | - * +-------------+ +--------------+ +--------------+ - * - **/ - -typedef enum AsyncRequestState -{ - DEFERRED, - EXECUTING, - COMPLETED, -} AsyncRequestState; - -typedef struct AsyncRequest -{ - const char *sql; - TSConnection *conn; - AsyncRequestState state; - const char *stmt_name; - int prep_stmt_params; - async_response_callback response_cb; - void *user_data; /* custom data saved with the request */ - StmtParams *params; - int res_format; /* text or binary */ - bool is_xact_transition; -} AsyncRequest; - -typedef struct PreparedStmt -{ - const char *sql; - TSConnection *conn; - const char *stmt_name; - int n_params; -} PreparedStmt; - -/* It is often useful to get the request along with the result in the response */ - -typedef struct AsyncResponse -{ - AsyncResponseType type; -} AsyncResponse; - -typedef struct AsyncResponseResult -{ - AsyncResponse base; - PGresult *result; - AsyncRequest *request; -} AsyncResponseResult; - -typedef struct AsyncResponseCommunicationError -{ - AsyncResponse base; - AsyncRequest *request; -} AsyncResponseCommunicationError; - -typedef struct AsyncResponseError -{ - AsyncResponse base; - const char *errmsg; -} AsyncResponseError; - -typedef struct AsyncRequestSet -{ - List *requests; -} AsyncRequestSet; - -static AsyncRequest * -async_request_create(TSConnection *conn, const char *sql, const char *stmt_name, - int prep_stmt_params, StmtParams *stmt_params, int res_format) -{ - AsyncRequest *req; - - if (conn == NULL) - elog(ERROR, "can't create AsyncRequest with NULL connection"); - - req = palloc0(sizeof(AsyncRequest)); - *req = (AsyncRequest){ .conn = conn, - .state = DEFERRED, - .sql = pstrdup(sql), - .stmt_name = stmt_name, - .params = stmt_params, - .prep_stmt_params = prep_stmt_params, - .res_format = res_format }; - - return req; -} - -static void -async_request_set_state(AsyncRequest *req, AsyncRequestState new_state) -{ - if (req->state != DEFERRED) - Assert(req->state != new_state); - -#ifdef USE_ASSERT_CHECKING - switch (new_state) - { - case DEFERRED: - /* initial state */ - Assert(req->state == DEFERRED); - break; - case EXECUTING: - Assert(req->state == DEFERRED); - break; - case COMPLETED: - Assert(req->state == EXECUTING); - } -#endif - req->state = new_state; -} - -/* Send a request. In case there is an ongoing request for the connection, - we will not send the request but set its status to DEFERRED. - Getting a response from DEFERRED AsyncRequest will try sending it if - the connection is not in use. - - Note that we can only send one sql statement per request. - This is because we use `PQsendQueryParams` which uses the extended query protocol - instead of the simple one. The extended protocol does not support multiple - statements. In the future we can use a `PQsendQuery` variant for queries without parameters, - which can support multiple statements because it uses the simple protocol. But this is - an optimization for another time. -*/ -static AsyncRequest * -async_request_send_internal(AsyncRequest *req, int elevel) -{ - int ret; - - if (req->state != DEFERRED) - elog(elevel, "can't send async request in state \"%d\"", req->state); - - if (remote_connection_is_processing(req->conn)) - return req; - - /* Send configuration parameters if necessary */ - if (!remote_connection_configure_if_changed(req->conn)) - elog(elevel, "could not configure connection when preparing statement"); - - if (req->stmt_name) - { - ret = PQsendQueryPrepared(remote_connection_get_pg_conn(req->conn), - req->stmt_name, - stmt_params_total_values(req->params), - stmt_params_values(req->params), - stmt_params_lengths(req->params), - stmt_params_formats(req->params), - req->res_format); - } - else - { - /* - * We intentionally do not specify parameter types here, but leave the - * data node to derive them by default. This avoids possible problems - * with the data node using different type OIDs than we do. All of - * the prepared statements we use in this module are simple enough that - * the data node will make the right choices. - */ - ret = PQsendQueryParams(remote_connection_get_pg_conn(req->conn), - req->sql, - stmt_params_total_values(req->params), - /* param types - see note above */ NULL, - stmt_params_values(req->params), - stmt_params_lengths(req->params), - stmt_params_formats(req->params), - req->res_format); - } - - if (ret == 0) - { - /* - * null is fine to pass down as the res, the connection error message - * will get through - */ - remote_connection_elog(req->conn, elevel); - return NULL; - } - - async_request_set_state(req, EXECUTING); - remote_connection_set_status(req->conn, CONN_PROCESSING); - return req; -} - -AsyncRequest * -async_request_send_with_stmt_params_elevel_res_format(TSConnection *conn, const char *sql_statement, - StmtParams *params, int elevel, - int res_format) -{ - AsyncRequest *req = async_request_create(conn, sql_statement, NULL, 0, params, res_format); - req = async_request_send_internal(req, elevel); - return req; -} - -AsyncRequest * -async_request_send_prepare(TSConnection *conn, const char *sql, int n_params) -{ - AsyncRequest *req; - char *stmt_name; - int ret; - - Assert(!remote_connection_is_processing(conn)); - - /* Construct name we'll use for the prepared statement. */ - stmt_name = psprintf("ts_prep_%u", remote_connection_get_prep_stmt_number()); - - /* Send configuration parameters if necessary */ - if (!remote_connection_configure_if_changed(conn)) - elog(ERROR, "could not configure connection when preparing statement"); - - req = async_request_create(conn, sql, stmt_name, n_params, NULL, FORMAT_TEXT); - - /* Do not specify parameter types, see note above in - * async_request_send_internal */ - ret = PQsendPrepare(remote_connection_get_pg_conn(req->conn), - req->stmt_name, - req->sql, - req->prep_stmt_params, - NULL); - - if (ret == 0) - { - pfree(req); - remote_connection_elog(req->conn, ERROR); - return NULL; - } - - async_request_set_state(req, EXECUTING); - remote_connection_set_status(req->conn, CONN_PROCESSING); - - return req; -} - -extern AsyncRequest * -async_request_send_prepared_stmt(PreparedStmt *stmt, const char *const *param_values) -{ - AsyncRequest *req = - async_request_create(stmt->conn, - stmt->sql, - stmt->stmt_name, - stmt->n_params, - stmt_params_create_from_values((const char **) param_values, - stmt->n_params), - FORMAT_TEXT); - return async_request_send_internal(req, ERROR); -} - -AsyncRequest * -async_request_send_prepared_stmt_with_params(PreparedStmt *stmt, StmtParams *params, int res_format) -{ - AsyncRequest *req = async_request_create(stmt->conn, - stmt->sql, - stmt->stmt_name, - stmt->n_params, - params, - res_format); - return async_request_send_internal(req, ERROR); -} - -/* Set user data. Often it is useful to attach data with a request so - that it can later be fetched from the response. */ -void -async_request_attach_user_data(AsyncRequest *req, void *user_data) -{ - req->user_data = user_data; -} - -void -async_request_set_response_callback(AsyncRequest *req, async_response_callback cb, void *user_data) -{ - req->response_cb = cb; - req->user_data = user_data; -} - -static AsyncResponseResult * -async_response_result_create(AsyncRequest *req, PGresult *res) -{ - AsyncResponseResult *ares; - AsyncResponseType type = RESPONSE_RESULT; - - if (PQresultStatus(res) == PGRES_SINGLE_TUPLE) - type = RESPONSE_ROW; - - ares = palloc0(sizeof(AsyncResponseResult)); - - *ares = (AsyncResponseResult){ - .base = { .type = type }, - .request = req, - .result = res, - }; - - return ares; -} - -static AsyncResponseCommunicationError * -async_response_communication_error_create(AsyncRequest *req) -{ - AsyncResponseCommunicationError *ares = palloc0(sizeof(AsyncResponseCommunicationError)); - - *ares = (AsyncResponseCommunicationError){ - .base = { .type = RESPONSE_COMMUNICATION_ERROR }, - .request = req, - }; - - return ares; -} - -static AsyncResponse * -async_response_timeout_create() -{ - AsyncResponse *ares = palloc0(sizeof(AsyncResponse)); - - *ares = (AsyncResponse){ - .type = RESPONSE_TIMEOUT, - }; - - return ares; -} - -static AsyncResponse * -async_response_error_create(const char *errmsg) -{ - AsyncResponseError *ares = palloc0(sizeof(AsyncResponseError)); - - *ares = (AsyncResponseError){ - .base = { .type = RESPONSE_ERROR }, - .errmsg = pstrdup(errmsg), - }; - - return &ares->base; -} - -void -async_response_result_close(AsyncResponseResult *res) -{ - PQclear(res->result); - pfree(res); -} - -/* Closes the async response. Note that `async_response_report_error` does this automatically. */ -void -async_response_close(AsyncResponse *res) -{ - switch (res->type) - { - case RESPONSE_RESULT: - case RESPONSE_ROW: - async_response_result_close((AsyncResponseResult *) res); - break; - default: - pfree(res); - break; - } -} - -AsyncResponseType -async_response_get_type(AsyncResponse *res) -{ - return res->type; -} - -/* get the user data attached to the corresponding request */ -void * -async_response_result_get_user_data(AsyncResponseResult *res) -{ - return res->request->user_data; -} - -PGresult * -async_response_result_get_pg_result(AsyncResponseResult *res) -{ - return res->result; -} - -AsyncRequest * -async_response_result_get_request(AsyncResponseResult *res) -{ - return res->request; -} - -bool -async_request_set_single_row_mode(AsyncRequest *req) -{ - return remote_connection_set_single_row_mode(req->conn); -} - -TSConnection * -async_request_get_connection(AsyncRequest *req) -{ - return req->conn; -} - -void -async_response_report_error(AsyncResponse *res, int elevel) -{ - switch (res->type) - { - case RESPONSE_RESULT: - case RESPONSE_ROW: - { - AsyncResponseResult *aresult = (AsyncResponseResult *) res; - ExecStatusType status = PQresultStatus(aresult->result); - - switch (status) - { - case PGRES_COMMAND_OK: - case PGRES_TUPLES_OK: - case PGRES_SINGLE_TUPLE: - break; - case PGRES_NONFATAL_ERROR: - case PGRES_FATAL_ERROR: - /* result is closed by remote_result_elog in case it throws - * error */ - remote_result_elog(aresult->result, elevel); - break; - default: - { - PG_TRY(); - { - elog(elevel, "unexpected response status %u", status); - } - PG_CATCH(); - { - async_response_close(res); - PG_RE_THROW(); - } - PG_END_TRY(); - } - } - break; - } - case RESPONSE_COMMUNICATION_ERROR: - remote_connection_elog(((AsyncResponseCommunicationError *) res)->request->conn, - elevel); - break; - case RESPONSE_ERROR: - elog(elevel, "%s", ((AsyncResponseError *) res)->errmsg); - break; - case RESPONSE_TIMEOUT: - elog(elevel, "async operation timed out"); - } -} - -void -async_response_report_error_or_close(AsyncResponse *res, int elevel) -{ - async_response_report_error(res, elevel); - async_response_close(res); -} - -/* - * This is a convenience function to wait for a single result from a request. - * This function requires that the request is for a single SQL statement. - */ -AsyncResponseResult * -async_request_wait_any_result(AsyncRequest *req) -{ - AsyncRequestSet set = { 0 }; - AsyncResponseResult *result; - - async_request_set_add(&set, req); - result = async_request_set_wait_any_result(&set); - - /* Should expect exactly one response */ - if (NULL == result) - { - elog(ERROR, "expected response for the remote tuple request, but received none"); - } - - /* Make sure to drain the connection only if we've retrieved complete result set */ - if (result->base.type == RESPONSE_RESULT) - { - AsyncResponseResult *extra; - bool got_extra = false; - - /* Must drain any remaining result until NULL */ - while ((extra = async_request_set_wait_any_result(&set))) - { - async_response_result_close(extra); - got_extra = true; - } - - if (got_extra) - { - async_response_result_close(result); - elog(ERROR, "request must be for one sql statement"); - } - } - - return result; -} - -AsyncResponseResult * -async_request_wait_ok_result(AsyncRequest *req) -{ - AsyncResponseResult *res = async_request_wait_any_result(req); - - if (PQresultStatus(res->result) != PGRES_COMMAND_OK && - PQresultStatus(res->result) != PGRES_TUPLES_OK) - { - async_response_report_error(&res->base, ERROR); - Assert(false); - } - - return res; -} - -/* - * Get the result of an async request during cleanup. - * - * Cleanup is typically necessary for a query that is being interrupted by - * transaction abort, or a query that was initiated as part of transaction - * abort to get the remote side back to the appropriate state. - * - * endtime is the time at which we should give up and assume the remote - * side is dead. - * - * An AsyncReponse is always returned, indicating last PGresult received, - * a timeout, or error. - */ -AsyncResponse * -async_request_cleanup_result(AsyncRequest *req, TimestampTz endtime) -{ - TSConnection *conn = async_request_get_connection(req); - PGresult *last_res = NULL; - AsyncResponse *rsp = NULL; - - switch (req->state) - { - case DEFERRED: - if (remote_connection_is_processing(req->conn)) - return async_response_error_create( - psprintf("request already in progress on port %d", PostPortNumber)); - - req = async_request_send_internal(req, WARNING); - - if (req == NULL) - return async_response_error_create("failed to send deferred request"); - - Assert(req->state == EXECUTING); - break; - case EXECUTING: - break; - case COMPLETED: - return async_response_error_create("request already completed"); - } - - switch (remote_connection_drain(conn, endtime, &last_res)) - { - case CONN_TIMEOUT: - rsp = async_response_timeout_create(); - break; - case CONN_DISCONNECT: - rsp = &async_response_communication_error_create(req)->base; - break; - case CONN_NO_RESPONSE: - rsp = async_response_error_create("no response during cleanup"); - break; - case CONN_OK: - Assert(last_res != NULL); - rsp = &async_response_result_create(req, last_res)->base; - break; - } - - Assert(rsp != NULL); - - return rsp; -} - -void -async_request_wait_ok_command(AsyncRequest *req) -{ - AsyncResponseResult *res = async_request_wait_any_result(req); - - if (PQresultStatus(res->result) != PGRES_COMMAND_OK) - { - async_response_report_error(&res->base, ERROR); - Assert(false); - } - - async_response_result_close(res); -} - -PreparedStmt * -async_request_wait_prepared_statement(AsyncRequest *request) -{ - AsyncResponseResult *result; - PreparedStmt *prep; - - Assert(request->stmt_name != NULL); - - result = async_request_wait_ok_result(request); - prep = async_response_result_generate_prepared_stmt(result); - async_response_result_close(result); - - return prep; -} - -AsyncRequestSet * -async_request_set_create() -{ - return palloc0(sizeof(AsyncRequestSet)); -} - -void -async_request_set_add(AsyncRequestSet *set, AsyncRequest *req) -{ - set->requests = list_append_unique_ptr(set->requests, req); -} - -static AsyncResponse * -get_single_response_nonblocking(AsyncRequestSet *set) -{ - ListCell *lc; - - foreach (lc, set->requests) - { - AsyncRequest *req = lfirst(lc); - PGconn *pg_conn = remote_connection_get_pg_conn(req->conn); - - switch (req->state) - { - case DEFERRED: - if (remote_connection_is_processing(req->conn)) - { - return async_response_error_create( - psprintf("request already in progress on port %d", PostPortNumber)); - } - - req = async_request_send_internal(req, WARNING); - - if (req == NULL) - return async_response_error_create("failed to send deferred request"); - - Assert(req->state == EXECUTING); - TS_FALLTHROUGH; - case EXECUTING: - if (0 == PQisBusy(pg_conn)) - { - PGresult *res = PQgetResult(pg_conn); - - if (NULL == res) - { - /* - * NULL return means query is complete - */ - set->requests = list_delete_ptr(set->requests, req); - remote_connection_set_status(req->conn, CONN_IDLE); - async_request_set_state(req, COMPLETED); - - /* set changed so rerun function */ - return get_single_response_nonblocking(set); - } - return &async_response_result_create(req, res)->base; - } - break; - case COMPLETED: - return async_response_error_create("request already completed"); - } - } - - return NULL; -} - -/* - * wait_to_consume_data waits until data is recieved and put into buffers - * so that it can be recieved without blocking by `get_single_response_nonblocking` - * or similar. - * - * Returns NULL on success or an "error" AsyncResponse - */ -static AsyncResponse * -wait_to_consume_data(AsyncRequestSet *set, TimestampTz end_time) -{ - /* - * Looks like there is no good way to modify a WaitEventSet so we have to - * make a new one, otherwise we can't turn off wait events - */ - WaitEventSet *we_set; - ListCell *lc; - int rc; - WaitEvent event; - uint32 wait_event_info = PG_WAIT_EXTENSION; - AsyncRequest *wait_req; - AsyncResponse *result; - long timeout_ms = -1L; - - Assert(list_length(set->requests) > 0); - - if (end_time != TS_NO_TIMEOUT) - { - TimestampTz now = GetCurrentTimestamp(); - long secs; - int microsecs; - - if (now >= end_time) - return async_response_timeout_create(); - - TimestampDifference(now, end_time, &secs, µsecs); - timeout_ms = secs * 1000 + (microsecs / 1000); - } - - we_set = CreateWaitEventSet(CurrentMemoryContext, list_length(set->requests) + 1); - - /* always wait for my latch */ - AddWaitEventToSet(we_set, WL_LATCH_SET, PGINVALID_SOCKET, (Latch *) MyLatch, NULL); - - foreach (lc, set->requests) - { - AsyncRequest *req = lfirst(lc); - - AddWaitEventToSet(we_set, - WL_SOCKET_READABLE, - PQsocket(remote_connection_get_pg_conn(req->conn)), - NULL, - req); - } - - while (true) - { - wait_req = NULL; - rc = WaitEventSetWait(we_set, timeout_ms, &event, 1, wait_event_info); - - if (rc == 0) - { - result = async_response_timeout_create(); - break; - } - - CHECK_FOR_INTERRUPTS(); - - if (event.events & ~(WL_SOCKET_READABLE | WL_LATCH_SET)) - { - /* - * Sanity check on the wait result: we haven't requested anything - * other than my latch or the socket becoming readable. - */ - result = async_response_error_create( - psprintf("Unexpected event 0x%X while waiting for async request result", - event.events)); - break; - } - - if (event.events & WL_LATCH_SET) - { - ResetLatch(MyLatch); - } - - if (event.events & WL_SOCKET_READABLE) - { - wait_req = event.user_data; - Assert(wait_req != NULL); - PGconn *pg_conn = remote_connection_get_pg_conn(wait_req->conn); - - if (0 == PQconsumeInput(pg_conn)) - { - /* An error has occurred, remove connection from set. */ - set->requests = list_delete_ptr(set->requests, wait_req); - result = &async_response_communication_error_create(wait_req)->base; - break; - } - - /* - * From postgres docs on PQConsumeInput(): - * Note that the result does not say whether any input data was - * actually collected. After calling PQconsumeInput, the - * application can check PQisBusy and/or PQnotifies to see if their - * state has changed. - */ - if (PQisBusy(pg_conn) == 0) - { - result = NULL; - break; - } - } - } - - FreeWaitEventSet(we_set); - return result; -} - -/* Return NULL when nothing more to do in set */ -AsyncResponse * -async_request_set_wait_any_response_deadline(AsyncRequestSet *set, TimestampTz endtime) -{ - AsyncResponse *response; - - while (true) - { - response = get_single_response_nonblocking(set); - - if (response != NULL) - break; - - if (list_length(set->requests) == 0) - /* nothing to wait on anymore */ - return NULL; - - response = wait_to_consume_data(set, endtime); - - if (response != NULL) - break; - } - - /* Make sure callbacks are run when a response is received. For a timeout, - * we run the callbacks on all the requests the user has been waiting - * on. */ - if (NULL != response) - { - List *requests = NIL; - ListCell *lc; - - switch (response->type) - { - case RESPONSE_RESULT: - case RESPONSE_ROW: - requests = list_make1(((AsyncResponseResult *) response)->request); - break; - case RESPONSE_COMMUNICATION_ERROR: - requests = list_make1(((AsyncResponseCommunicationError *) response)->request); - break; - case RESPONSE_ERROR: - case RESPONSE_TIMEOUT: - requests = set->requests; - break; - } - - foreach (lc, requests) - { - AsyncRequest *req = lfirst(lc); - - if (NULL != req->response_cb) - req->response_cb(req, response, req->user_data); - } - } - - return response; -} - -AsyncResponseResult * -async_request_set_wait_any_result(AsyncRequestSet *set) -{ - AsyncResponse *res = async_request_set_wait_any_response(set); - - if (res == NULL) - return NULL; - - if (!(RESPONSE_RESULT == res->type || RESPONSE_ROW == res->type)) - async_response_report_error(res, ERROR); - - return (AsyncResponseResult *) res; -} - -AsyncResponseResult * -async_request_set_wait_ok_result(AsyncRequestSet *set) -{ - AsyncResponseResult *response_result = async_request_set_wait_any_result(set); - ExecStatusType status; - - if (response_result == NULL) - return NULL; - - status = PQresultStatus(response_result->result); - - if (status != PGRES_TUPLES_OK && status != PGRES_COMMAND_OK) - { - async_response_report_error(&response_result->base, ERROR); - Assert(false); - } - - return response_result; -} - -void -async_request_set_wait_all_ok_commands(AsyncRequestSet *set) -{ - AsyncResponse *rsp; - AsyncResponse *bad_rsp = NULL; - - /* Drain all responses and record the first error */ - while ((rsp = async_request_set_wait_any_response(set))) - { - switch (async_response_get_type(rsp)) - { - case RESPONSE_RESULT: - case RESPONSE_ROW: - { - AsyncResponseResult *ar = (AsyncResponseResult *) rsp; - ExecStatusType status = PQresultStatus(async_response_result_get_pg_result(ar)); - - if (status != PGRES_COMMAND_OK && bad_rsp == NULL) - bad_rsp = rsp; - else - async_response_result_close(ar); - break; - } - default: - if (bad_rsp == NULL) - bad_rsp = rsp; - break; - } - } - - /* Throw error once request set is drained */ - if (bad_rsp != NULL) - async_response_report_error(bad_rsp, ERROR); -} - -void -async_request_discard_response(AsyncRequest *req) -{ - Assert(req != NULL); - - AsyncRequestSet set = { 0 }; - async_request_set_add(&set, req); - - AsyncResponseResult *result = NULL; - do - { - /* for COPY fetching we need to loop until we consume the whole response */ - result = async_request_set_wait_any_result(&set); - if (result != NULL) - async_response_result_close(result); - } while (result != NULL && req->state != COMPLETED); -} - -void -prepared_stmt_close(PreparedStmt *stmt) -{ - char sql[64] = { '\0' }; - int ret; - - ret = snprintf(sql, sizeof(sql), "DEALLOCATE %s", stmt->stmt_name); - - if (ret < 0 || (size_t) ret >= sizeof(sql)) - elog(ERROR, "could not create deallocate statement"); - - async_request_wait_ok_command(async_request_send(stmt->conn, sql)); -} - -/* Request must have been generated by async_request_send_prepare() */ -PreparedStmt * -async_response_result_generate_prepared_stmt(AsyncResponseResult *result) -{ - PreparedStmt *prep; - - if (PQresultStatus(result->result) != PGRES_COMMAND_OK) - async_response_report_error(&result->base, ERROR); - - prep = palloc0(sizeof(PreparedStmt)); - - *prep = (PreparedStmt){ - .conn = result->request->conn, - .sql = result->request->sql, - .stmt_name = result->request->stmt_name, - .n_params = result->request->prep_stmt_params, - }; - - return prep; -} diff --git a/tsl/src/remote/async.h b/tsl/src/remote/async.h deleted file mode 100644 index ad447404ad1..00000000000 --- a/tsl/src/remote/async.h +++ /dev/null @@ -1,150 +0,0 @@ -/* - * 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. - */ -#pragma once - -#include -#include -#include - -#include "stmt_params.h" - -typedef struct AsyncRequest AsyncRequest; - -typedef struct TSConnection TSConnection; - -typedef enum AsyncResponseType -{ - /* We got an entire result */ - RESPONSE_RESULT = 0, - /* We got one row */ - RESPONSE_ROW, - /* There was some kind of communication error */ - RESPONSE_COMMUNICATION_ERROR, - /* Timeout while waiting for response */ - RESPONSE_TIMEOUT, - /* Unexpected event or other error */ - RESPONSE_ERROR, -} AsyncResponseType; - -/* Base type for all responses */ -typedef struct AsyncResponse AsyncResponse; -typedef struct AsyncResponseResult AsyncResponseResult; -typedef struct AsyncResponseCommunicationError AsyncResponseCommunicationError; -typedef struct AsyncResponseError AsyncResponseError; -typedef struct AsyncRequestSet AsyncRequestSet; - -typedef struct PreparedStmt PreparedStmt; - -/* Notes on naming conventions used in functions: - * - * Parameters named sql_statement take a string with single sql statement (as opposed to multiple - * statements) - * - * function endings: - * * _any_response return any responses including errors and timeouts - * * _any_result return a result which may be an error or success, but throw errors for - * communication errors and timeouts - * * _ok_result return successful results, throwing errors otherwise. - * * _ok_command returns void on successful commands, throwing errors otherwise. - * - */ - -/* Async Request */ - -typedef void (*async_response_callback)(AsyncRequest *m, AsyncResponse *, void *data); - -#define TS_NO_TIMEOUT DT_NOBEGIN - -extern AsyncRequest *async_request_send_with_stmt_params_elevel_res_format( - TSConnection *conn, const char *sql_statement, StmtParams *params, int elevel, int res_format); - -#define async_request_send_with_params(conn, sql_statement, params, res_format) \ - async_request_send_with_stmt_params_elevel_res_format(conn, \ - sql_statement, \ - params, \ - ERROR, \ - res_format) -#define async_request_send_with_params_elevel(conn, sql_statement, params, elevel) \ - async_request_send_with_stmt_params_elevel_res_format(conn, \ - sql_statement, \ - params, \ - elevel, \ - FORMAT_TEXT) -#define async_request_send_binary(conn, sql_statement) \ - async_request_send_with_stmt_params_elevel_res_format(conn, \ - sql_statement, \ - NULL, \ - ERROR, \ - FORMAT_BINARY) -#define async_request_send_with_error(conn, sql_statement, elevel) \ - async_request_send_with_stmt_params_elevel_res_format(conn, \ - sql_statement, \ - NULL, \ - elevel, \ - FORMAT_TEXT) -#define async_request_send(conn, sql_statement) \ - async_request_send_with_error(conn, sql_statement, ERROR) - -extern AsyncRequest *async_request_send_prepare(TSConnection *conn, const char *sql_statement, - int n_params); -extern AsyncRequest *async_request_send_prepared_stmt(PreparedStmt *stmt, - const char *const *param_values); -extern AsyncRequest *async_request_send_prepared_stmt_with_params(PreparedStmt *stmt, - StmtParams *params, - int res_format); - -extern void async_request_attach_user_data(AsyncRequest *req, void *user_data); -extern void async_request_set_response_callback(AsyncRequest *req, async_response_callback cb, - void *user_data); -extern bool async_request_set_single_row_mode(AsyncRequest *req); -extern TSConnection *async_request_get_connection(AsyncRequest *req); -extern AsyncResponseResult *async_request_wait_ok_result(AsyncRequest *request); -extern AsyncResponseResult *async_request_wait_any_result(AsyncRequest *request); -extern AsyncResponse *async_request_cleanup_result(AsyncRequest *req, TimestampTz endtime); - -/* Returns on successful commands, throwing errors otherwise */ -extern void async_request_wait_ok_command(AsyncRequest *req); -extern PreparedStmt *async_request_wait_prepared_statement(AsyncRequest *request); - -/* Async Response */ -extern void async_response_close(AsyncResponse *res); -extern void async_response_report_error(AsyncResponse *res, int elevel); -extern void async_response_report_error_or_close(AsyncResponse *res, int elevel); - -extern AsyncResponseType async_response_get_type(AsyncResponse *res); -extern void async_response_result_close(AsyncResponseResult *res); -extern PGresult *async_response_result_get_pg_result(AsyncResponseResult *res); -extern void *async_response_result_get_user_data(AsyncResponseResult *res); -extern AsyncRequest *async_response_result_get_request(AsyncResponseResult *res); -extern PreparedStmt *async_response_result_generate_prepared_stmt(AsyncResponseResult *res); - -/* Async Request Set */ -extern AsyncRequestSet *async_request_set_create(void); -extern void async_request_set_add(AsyncRequestSet *set, AsyncRequest *req); -#define async_request_set_add_sql(set, conn, sql) \ - async_request_set_add(set, async_request_send(conn, sql)) - -/* Return any response, including communication errors and timeouts */ -extern AsyncResponse *async_request_set_wait_any_response_deadline(AsyncRequestSet *set, - TimestampTz endtime); - -#define async_request_set_wait_any_response(set) \ - async_request_set_wait_any_response_deadline(set, TS_NO_TIMEOUT) - -/* Return only successful results, throwing errors otherwise */ -extern AsyncResponseResult *async_request_set_wait_ok_result(AsyncRequestSet *set); - -extern void async_request_set_wait_all_ok_commands(AsyncRequestSet *set); - -/* Return any results (success or failure), but throw errors on communication failures and timeouts - */ -extern AsyncResponseResult *async_request_set_wait_any_result(AsyncRequestSet *set); - -/* Consume any pending response and throw it away */ -extern void async_request_discard_response(AsyncRequest *req); - -/* Prepared Statements */ -extern void prepared_stmt_close(PreparedStmt *stmt); diff --git a/tsl/src/remote/connection.c b/tsl/src/remote/connection.c deleted file mode 100644 index 2417ba6daaa..00000000000 --- a/tsl/src/remote/connection.c +++ /dev/null @@ -1,2697 +0,0 @@ -/* - * This file and its contents are licensed under the Timescale License. - * Please see the included NOTICE for copyright information and - * LICENSE-TIMESCALE for a copy of the license. - */ - -/* - * This file contains source code that was copied and/or modified from the - * PostgreSQL database, which is licensed under the open-source PostgreSQL - * License. Please see the NOTICE at the top level directory for a copy of - * the PostgreSQL License. - */ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#ifdef USE_TELEMETRY -#include -#endif -#include "connection.h" -#include "data_node.h" -#include "debug_point.h" -#include "utils.h" -#include "ts_catalog/metadata.h" -#include "config.h" - -/* - * Connection library for TimescaleDB. - * - * This library file contains convenience functionality around the libpq - * API. The major additional functionality offered includes: - * - * - Lifecycle management: a connection is tied to the memory context it is - * created in and result objects are tied to the connection they are created - * from. The aim is to avoid memory leaks of libpq objects that aren't - * allocated on a PostgreSQL memory context. - * - * - Connection configuration suitable for TimescaleDB, ensuring the data - * nodes use the same relevant configurations as the access node (e.g., time - * zone). - * - * NOTE that it is strongly adviced that connection-related functions do not - * throw exceptions with, e.g., elog(ERROR). While exceptions can be caught - * with PG_TRY-CATCH for cleanup, it is not possible to safely continue the - * transaction that threw the exception as if no error occurred (see the - * following post if unconvinced: - * https://www.postgresql.org/message-id/27190.1508727890%40sss.pgh.pa.us). - * - * In some cases, we need to be able to continue a transaction even if a - * connection fails. One example is the removal of a data node, which must be - * able to proceed even if the node is no longer available to respond to a - * connection. Another example is performing a liveness check for node status. - * - * Therefore, it is best to defer throwing exceptions to high-level functions - * that know when it is appropriate. - */ - -/* for assigning cursor numbers and prepared statement numbers */ -static unsigned int cursor_number = 0; -static unsigned int prep_stmt_number = 0; -static RemoteConnectionStats connstats = { 0 }; - -static int eventproc(PGEventId eventid, void *eventinfo, void *data); - -TSConnectionId -remote_connection_id(const Oid server_oid, const Oid user_oid) -{ - TSConnectionId id = { .server_id = server_oid, .user_id = user_oid }; - return id; -} - -void -remote_connection_id_set(TSConnectionId *const id, Oid const server_oid, Oid const user_oid) -{ - id->server_id = server_oid; - id->user_id = user_oid; -} - -/* - * A simple circular list implementation for tracking libpq connection and - * result objects. We can't use pg_list here since it is bound to PostgreSQL's - * memory management system, while libpq is not. - */ -typedef struct ListNode -{ - struct ListNode *next; - struct ListNode *prev; -} ListNode; - -#define IS_DETACHED_ENTRY(entry) ((entry)->next == NULL && (entry)->prev == NULL) - -/* - * Detach a list node. - * - * Detaches a list node from the list, unless it is the anchor/head (which is - * a no-op). - */ -static inline void -list_detach(ListNode *entry) -{ - ListNode *prev = entry->prev; - ListNode *next = entry->next; - - next->prev = prev; - prev->next = next; - /* Clear entry fields */ - entry->prev = NULL; - entry->next = NULL; -} - -/* - * Insert a list node entry after the prev node. - */ -static inline void -list_insert_after(ListNode *entry, ListNode *prev) -{ - ListNode *next = prev->next; - - next->prev = entry; - entry->next = next; - entry->prev = prev; - prev->next = entry; -} - -/* - * List entry that holds a PGresult object. - */ -typedef struct ResultEntry -{ - struct ListNode ln; /* Must be first entry */ - TSConnection *conn; /* The connection the result was created on */ - SubTransactionId subtxid; /* The subtransaction ID that created this result, if any. */ - PGresult *result; -} ResultEntry; - -typedef struct TSConnection -{ - ListNode ln; /* Must be first entry */ - PGconn *pg_conn; /* PostgreSQL connection */ - TSConnectionStatus status; - NameData node_name; /* Associated data node name */ - char tz_name[TZ_STRLEN_MAX + 1]; /* Timezone name last sent over connection */ - int xact_depth; /* 0 => no transaction, 1 => main transaction, > 1 => - * levels of subtransactions */ - bool xact_transitioning; /* TRUE if connection is transitioning to - * another transaction state */ - ListNode results; /* Head of PGresult list */ - bool binary_copy; - MemoryContext mcxt; - MemoryContextCallback mcxt_cb; - bool mcxt_cb_invoked; - WaitEventSet *wes; - int sockeventpos; -} TSConnection; - -/* - * List of all connections we create. Used to auto-free connections and/or - * PGresults at transaction end. - */ -static ListNode connections = { &connections, &connections }; - -static bool -fill_simple_error(TSConnectionError *err, int errcode, const char *errmsg, const TSConnection *conn) -{ - if (NULL == err) - return false; - - MemSet(err, 0, sizeof(*err)); - - err->errcode = errcode; - err->msg = errmsg; - if (err->msg == NULL || strlen(err->msg) == 0) - { - char *connection_message = PQerrorMessage(conn->pg_conn); - if (connection_message) - { - err->msg = pstrdup(connection_message); - } - else - { - err->msg = "unknown error"; - } - } - err->host = pstrdup(PQhost(conn->pg_conn)); - err->nodename = pstrdup(remote_connection_node_name(conn)); - - return false; -} - -static bool -fill_connection_error(TSConnectionError *err, int errcode, const char *errmsg, - const TSConnection *conn) -{ - if (NULL == err) - return false; - - fill_simple_error(err, errcode, errmsg, conn); - /* The connection error from the remote side contains its own ERROR prefix - * and is ended by a newline. Make sure to strip that before emitting a - * local error. */ - err->connmsg = pchomp(PQerrorMessage(conn->pg_conn)); - - if (strncmp("ERROR: ", err->connmsg, 8) == 0) - err->connmsg += 8; - - return false; -} - -static char * -get_error_field_copy(const PGresult *res, int fieldcode) -{ - const char *msg = PQresultErrorField(res, fieldcode); - - if (NULL == msg) - return NULL; - return pchomp(msg); -} - -/* - * Convert libpq error severity to local error level. - */ -static int -severity_to_elevel(const char *severity) -{ - /* According to https://www.postgresql.org/docs/current/libpq-exec.html, - * libpq only returns the severity levels listed below. */ - static const struct - { - const char *severity; - int elevel; - } severity_levels[] = { { - .severity = "ERROR", - .elevel = ERROR, - }, - { - .severity = "FATAL", - .elevel = FATAL, - }, - { - .severity = "PANIC", - .elevel = PANIC, - }, - { - .severity = "WARNING", - .elevel = WARNING, - }, - { - .severity = "NOTICE", - .elevel = NOTICE, - }, - { - .severity = "DEBUG", - .elevel = DEBUG1, - }, - { - .severity = "INFO", - .elevel = INFO, - }, - { - .severity = "LOG", - .elevel = LOG, - }, - /* End marker */ - { - .severity = NULL, - .elevel = 0, - } }; - int i; - - if (NULL == severity) - return 0; - - i = 0; - - while (NULL != severity_levels[i].severity) - { - if (strcmp(severity_levels[i].severity, severity) == 0) - return severity_levels[i].elevel; - i++; - } - - pg_unreachable(); - - return ERROR; -} - -/* - * Fill a connection error based on the result of a remote query. - */ -static bool -fill_result_error(TSConnectionError *err, int errcode, const char *errmsg, const PGresult *res) -{ - const ResultEntry *entry = PQresultInstanceData(res, eventproc); - const char *sqlstate; - - if (NULL == err || NULL == res || NULL == entry) - { - if (err) - { - MemSet(err, 0, sizeof(*err)); - err->errcode = errcode; - err->msg = errmsg; - err->nodename = ""; - } - return false; - } - - Assert(entry->conn); - - fill_simple_error(err, errcode, errmsg, entry->conn); - err->remote.elevel = severity_to_elevel(PQresultErrorField(res, PG_DIAG_SEVERITY_NONLOCALIZED)); - err->remote.sqlstate = get_error_field_copy(res, PG_DIAG_SQLSTATE); - err->remote.msg = get_error_field_copy(res, PG_DIAG_MESSAGE_PRIMARY); - err->remote.detail = get_error_field_copy(res, PG_DIAG_MESSAGE_DETAIL); - err->remote.hint = get_error_field_copy(res, PG_DIAG_MESSAGE_HINT); - err->remote.context = get_error_field_copy(res, PG_DIAG_CONTEXT); - err->remote.stmtpos = get_error_field_copy(res, PG_DIAG_STATEMENT_POSITION); - /* - * Try to find at least some non-empty error message. The result error - * message may be not set if a node segfaults. - */ - if (err->remote.msg == NULL || strlen(err->remote.msg) == 0) - { - char *result_message = PQresultErrorMessage(res); - if (result_message && strlen(result_message)) - { - err->remote.msg = pstrdup(result_message); - } - } - - if (err->remote.msg == NULL || strlen(err->remote.msg) == 0) - { - char *connection_message = PQerrorMessage(entry->conn->pg_conn); - if (connection_message && strlen(connection_message)) - { - err->remote.msg = pstrdup(connection_message); - } - } - - if (err->remote.msg == NULL || strlen(err->remote.msg) == 0) - { - err->remote.msg = "unknown error"; - } - - sqlstate = err->remote.sqlstate; - - if (sqlstate && strlen(sqlstate) == 5) - err->remote.errcode = - MAKE_SQLSTATE(sqlstate[0], sqlstate[1], sqlstate[2], sqlstate[3], sqlstate[4]); - else - err->remote.errcode = ERRCODE_INTERNAL_ERROR; - - return false; -} - -/* - * The following event handlers make sure all PGresult are freed with - * PQClear() when its parent connection is closed. - * - * It is still recommended to explicitly call PGclear() or - * remote_connection_result_close(), however, especially when PGresults are - * created in a tight loop (e.g., when scanning many tuples on a remote - * table). - */ -#define EVENTPROC_FAILURE 0 -#define EVENTPROC_SUCCESS 1 - -/* - * Invoked on PQfinish(conn). Frees all PGresult objects created on the - * connection, apart from those already freed with PQclear(). - */ -static int -handle_conn_destroy(PGEventConnDestroy *event) -{ - TSConnection *conn = PQinstanceData(event->conn, eventproc); - unsigned int results_count = 0; - ListNode *curr; - - Assert(NULL != conn); - - curr = conn->results.next; - - while (curr != &conn->results) - { - ResultEntry *entry = (ResultEntry *) curr; - PGresult *result = entry->result; - - curr = curr->next; - PQclear(result); - /* No need to free curr here since PQclear will invoke - * handle_result_destroy() which will free it */ - results_count++; - } - - if (results_count > 0) - elog(DEBUG3, "cleared %u result objects on connection %p", results_count, conn); - - connstats.connections_closed++; - - conn->pg_conn = NULL; - list_detach(&conn->ln); - - FreeWaitEventSet(conn->wes); - - /* No need to delete the memory context here if handler was invoked by the - * MemoryContextDelete callback */ - if (!conn->mcxt_cb_invoked) - MemoryContextDelete(conn->mcxt); - - return EVENTPROC_SUCCESS; -} - -/* - * Invoked on PQgetResult(conn). Adds the PGresult to the list in the parent - * TSConnection. - */ -static int -handle_result_create(PGEventResultCreate *event) -{ - TSConnection *conn = PQinstanceData(event->conn, eventproc); - ResultEntry *entry; - - Assert(NULL != conn); - entry = MemoryContextAllocZero(conn->mcxt, sizeof(ResultEntry)); - - if (NULL == entry) - return EVENTPROC_FAILURE; - - entry->ln.next = entry->ln.prev = NULL; - entry->conn = conn; - entry->result = event->result; - entry->subtxid = GetCurrentSubTransactionId(); - - /* Add entry as new head and set instance data */ - list_insert_after(&entry->ln, &conn->results); - PQresultSetInstanceData(event->result, eventproc, entry); - - elog(DEBUG3, - "created result %p on connection %p subtxid %u", - event->result, - conn, - entry->subtxid); - - connstats.results_created++; - - return EVENTPROC_SUCCESS; -} - -/* - * Invoked on PQclear(result). Removes the PGresult from the list in the - * parent TSConnection. - */ -static int -handle_result_destroy(PGEventResultDestroy *event) -{ - ResultEntry *entry = PQresultInstanceData(event->result, eventproc); - - Assert(NULL != entry); - - /* Detach entry */ - list_detach(&entry->ln); - - elog(DEBUG3, "destroyed result %p for subtxnid %u", entry->result, entry->subtxid); - - pfree(entry); - connstats.results_cleared++; - - return EVENTPROC_SUCCESS; -} - -/* - * Main event handler invoked when events happen on a PGconn. - * - * According to the libpq API, the function should return a non-zero value if - * it succeeds and zero if it fails. We use EVENTPROC_SUCCESS and - * EVENTPROC_FAILURE in place of these two options. - */ -static int -eventproc(PGEventId eventid, void *eventinfo, void *data) -{ - int res = EVENTPROC_SUCCESS; - - switch (eventid) - { - case PGEVT_CONNDESTROY: - res = handle_conn_destroy((PGEventConnDestroy *) eventinfo); - break; - case PGEVT_RESULTCREATE: - res = handle_result_create((PGEventResultCreate *) eventinfo); - break; - case PGEVT_RESULTDESTROY: - res = handle_result_destroy((PGEventResultDestroy *) eventinfo); - break; - case PGEVT_RESULTCOPY: - /* Not used in the code, so not handled */ - Assert(false); - break; - default: - /* Not of interest, so return success */ - break; - } - - return res; -} - -static PQconninfoOption * -get_libpq_options() -{ - /* make static to fetch once per backend */ - static PQconninfoOption *libpq_options = NULL; - - if (libpq_options == NULL) - { - /* Note that the options array is Malloc'ed */ - libpq_options = PQconndefaults(); - } - - if (libpq_options == NULL) - { - /* probably OOM */ - elog(ERROR, "could not get default libpq options"); - } - - return libpq_options; -} - -static void -unset_libpq_envvar(void) -{ - PQconninfoOption *lopt; - PQconninfoOption *options = PQconndefaults(); - - TS_OOM_CHECK(options, "out of memory"); - - /* Explicitly unset all libpq environment variables. - * - * By default libpq uses environment variables as a fallback - * to specify connection options, potentially they could be in - * a conflict with PostgreSQL variables and introduce - * security risks. - */ - for (lopt = options; lopt->keyword; lopt++) - { - if (lopt->envvar) - unsetenv(lopt->envvar); - } - - PQconninfoFree(options); -} - -static bool -is_libpq_option(const char *keyword, char **display_option) -{ - PQconninfoOption *lopt; - - for (lopt = get_libpq_options(); lopt->keyword; lopt++) - { - if (strcmp(lopt->keyword, keyword) == 0) - { - if (display_option != NULL) - *display_option = lopt->dispchar; - return true; - } - } - return false; -} - -ConnOptionType -remote_connection_option_type(const char *keyword) -{ - char *display_option; - - if (!is_libpq_option(keyword, &display_option)) - return CONN_OPTION_TYPE_NONE; - - /* Hide debug options, as well as settings we override internally. */ - if (strchr(display_option, 'D') || strcmp(keyword, "fallback_application_name") == 0 || - strcmp(keyword, "client_encoding") == 0) - return CONN_OPTION_TYPE_NONE; - - /* - * "user" and any secret options are allowed only on user mappings. - * Everything else is a data node option. - */ - if (strchr(display_option, '*') || strcmp(keyword, "user") == 0) - return CONN_OPTION_TYPE_USER; - - return CONN_OPTION_TYPE_NODE; -} - -bool -remote_connection_valid_user_option(const char *keyword) -{ - return remote_connection_option_type(keyword) == CONN_OPTION_TYPE_USER; -} - -bool -remote_connection_valid_node_option(const char *keyword) -{ - return remote_connection_option_type(keyword) == CONN_OPTION_TYPE_NODE; -} - -static int -extract_connection_options(List *defelems, const char **keywords, const char **values, - const char **user) -{ - ListCell *lc; - int option_pos = 0; - - Assert(keywords != NULL); - Assert(values != NULL); - Assert(user != NULL); - - *user = NULL; - foreach (lc, defelems) - { - DefElem *d = (DefElem *) lfirst(lc); - - if (is_libpq_option(d->defname, NULL)) - { - keywords[option_pos] = d->defname; - values[option_pos] = defGetString(d); - if (strcmp(d->defname, "user") == 0) - { - Assert(*user == NULL); - *user = values[option_pos]; - } - option_pos++; - } - } - - return option_pos; -} - -static bool -prepend_enforced_conn_settings(TSConnection *conn, StringInfo cmdbuf) -{ - const char *local_tz_name = pg_get_timezone_name(session_timezone); - - /* - * We need to enforce the same timezone setting across nodes. Otherwise, - * we might get the wrong result when we push down things like - * date_trunc(text, timestamptz). To safely do that, we also need the - * timezone databases to be the same on all data nodes. - * - * We save away the timezone name so that we know what we last sent over - * the connection. If the time zone changed since last time we sent a - * command, we will send a SET TIMEZONE command with the new timezone - * first. - */ - if (conn->tz_name[0] == '\0' || - (local_tz_name && pg_strcasecmp(conn->tz_name, local_tz_name) != 0)) - { - StringInfo newcmd = makeStringInfo(); - - strncpy(conn->tz_name, local_tz_name, TZ_STRLEN_MAX); - appendStringInfo(newcmd, "SET TIMEZONE = '%s'", local_tz_name); - - if (cmdbuf->len > 0) - appendStringInfo(newcmd, ";%s", cmdbuf->data); - - *cmdbuf = *newcmd; - - return true; - } - - return false; -} - -/* - * Internal connection configure. - * - * This function will send internal configuration settings if they have - * changed. It is used to pass on configuration settings before executing a - * command requested by module users. - * - * Returns true if the current configuration is OK (no change) or was - * successfully applied, otherwise false. - */ -bool -remote_connection_configure_if_changed(TSConnection *conn) -{ - StringInfoData cmd = { - .data = NULL, - .len = 0, - .maxlen = 0, - }; - bool success = true; - - /* - * We need to enforce the same timezone setting across nodes. Otherwise, - * we might get the wrong result when we push down things like - * date_trunc(text, timestamptz). To safely do that, we also need the - * timezone databases to be the same on all data nodes. - * - * We save away the timezone name so that we know what we last sent over - * the connection. If the time zone changed since last time we sent a - * command, we will send a SET TIMEZONE command with the new timezone - * first. - */ - if (prepend_enforced_conn_settings(conn, &cmd)) - { - PGresult *result = remote_connection_exec(conn, cmd.data); - success = (PQresultStatus(result) == PGRES_COMMAND_OK); - PQclear(result); - } - - return success; -} - -/* - * Default options/commands to set on every new connection. - * - * Timezone is indirectly set with the first command executed. - */ -static const char *default_connection_options[] = { - /* - * Force the search path to contain only pg_catalog, which will force - * functions to output fully qualified identifier names (i.e., they will - * include the schema). - */ - "SET search_path = pg_catalog", - /* - * Set values needed to ensure unambiguous data output from remote. (This - * logic should match what pg_dump does. See also set_transmission_modes - * in fdw.c.) - */ - "SET datestyle = ISO", - "SET intervalstyle = postgres", - "SET extra_float_digits = 3", - /* - * Prepared statement data fetcher sets it to "force" which might be - * suboptimal for other kinds of queries. - */ - "RESET plan_cache_mode", - "SET statement_timeout = 0", - NULL, -}; - -/* - * Issue SET commands to make sure remote session is configured properly. - * - * We do this just once at connection, assuming nothing will change the - * values later. Since we'll never send volatile function calls to the - * remote, there shouldn't be any way to break this assumption from our end. - * It's possible to think of ways to break it at the remote end, eg making a - * foreign table point to a view that includes a set_config call --- but once - * you admit the possibility of a malicious view definition, there are any - * number of ways to break things. - */ -bool -remote_connection_configure(TSConnection *conn) -{ - const char *cmd; - StringInfoData sql; - PGresult *result; - bool success = true; - int i = 0; - - initStringInfo(&sql); - - while ((cmd = default_connection_options[i]) != NULL) - { - appendStringInfo(&sql, "%s;", cmd); - i++; - } - - result = remote_connection_exec(conn, sql.data); - success = PQresultStatus(result) == PGRES_COMMAND_OK; - PQclear(result); - pfree(sql.data); - - return success; -} - -static void -connection_memcxt_reset_cb(void *arg) -{ - TSConnection *conn = arg; - - conn->mcxt_cb_invoked = true; - - /* Close the connection and free all attached resources, unless already - * closed explicitly before being freed. */ - if (conn->pg_conn != NULL) - PQfinish(conn->pg_conn); -} - -/* - * Create a new connection. - * - * The returned connection object is allocated on the current memory context - * and is tied to its life-cycle. The connection object includes natively - * allocated memory from libpq (via malloc) which will be freed via callbacks - * when the main memory context is freed. - */ -static TSConnection * -remote_connection_create(PGconn *pg_conn, bool processing, const char *node_name) -{ - MemoryContext mcxt = - AllocSetContextCreate(CurrentMemoryContext, "TSConnection", ALLOCSET_SMALL_SIZES); - TSConnection *conn = MemoryContextAllocZero(mcxt, sizeof(TSConnection)); - int ret; - - /* Must register the event procedure before attaching any instance data */ - ret = PQregisterEventProc(pg_conn, eventproc, "remote connection", conn); - - if (ret == 0) - { - MemoryContextDelete(mcxt); - return NULL; - } - - ret = PQsetInstanceData(pg_conn, eventproc, conn); - Assert(ret != 0); - - conn->ln.next = conn->ln.prev = NULL; - conn->pg_conn = pg_conn; - remote_connection_set_status(conn, processing ? CONN_PROCESSING : CONN_IDLE); - namestrcpy(&conn->node_name, node_name); - conn->tz_name[0] = '\0'; - conn->xact_depth = 0; - conn->xact_transitioning = false; - /* Initialize results head */ - conn->results.next = &conn->results; - conn->results.prev = &conn->results; - conn->binary_copy = false; - conn->mcxt = mcxt; - conn->wes = CreateWaitEventSet(mcxt, 3); - AddWaitEventToSet(conn->wes, WL_LATCH_SET, PGINVALID_SOCKET, MyLatch, NULL); - AddWaitEventToSet(conn->wes, WL_EXIT_ON_PM_DEATH, PGINVALID_SOCKET, NULL, NULL); - /* Register the socket to get the position in the events array. The actual - * events used here does not matter, since it will be modified as - * appropriate when needed. */ - conn->sockeventpos = - AddWaitEventToSet(conn->wes, WL_SOCKET_READABLE, PQsocket(conn->pg_conn), NULL, NULL); - - /* Register a memory context callback that will ensure the connection is - * always closed and the resources are freed */ - conn->mcxt_cb.func = connection_memcxt_reset_cb; - conn->mcxt_cb.arg = conn; - MemoryContextRegisterResetCallback(mcxt, &conn->mcxt_cb); - list_insert_after(&conn->ln, &connections); - elog(DEBUG3, "created connection %p", conn); - connstats.connections_created++; - - return conn; -} - -int -remote_connection_xact_depth_get(const TSConnection *conn) -{ - Assert(conn->xact_depth >= 0); - return conn->xact_depth; -} - -int -remote_connection_xact_depth_inc(TSConnection *conn) -{ - Assert(conn->xact_depth >= 0); - return ++conn->xact_depth; -} - -int -remote_connection_xact_depth_dec(TSConnection *conn) -{ - Assert(conn->xact_depth > 0); - return --conn->xact_depth; -} - -void -remote_connection_xact_transition_begin(TSConnection *conn) -{ - Assert(!conn->xact_transitioning); - conn->xact_transitioning = true; -} - -void -remote_connection_xact_transition_end(TSConnection *conn) -{ - Assert(conn->xact_transitioning); - conn->xact_transitioning = false; -} - -bool -remote_connection_xact_is_transitioning(const TSConnection *conn) -{ - return conn->xact_transitioning; -} - -PGconn * -remote_connection_get_pg_conn(const TSConnection *conn) -{ - Assert(conn != NULL); - return conn->pg_conn; -} - -bool -remote_connection_is_processing(const TSConnection *conn) -{ - Assert(conn != NULL); - return conn->status != CONN_IDLE; -} - -void -remote_connection_set_status(TSConnection *conn, TSConnectionStatus status) -{ - Assert(conn != NULL); - conn->status = status; - - /* Should be blocking except when doing COPY. */ - Assert(PQisnonblocking(conn->pg_conn) == (conn->status == CONN_COPY_IN)); -} - -TSConnectionStatus -remote_connection_get_status(const TSConnection *conn) -{ - return conn->status; -} - -const char * -remote_connection_node_name(const TSConnection *conn) -{ -#ifndef NDEBUG - const char *hide_node_name = - GetConfigOption("timescaledb.hide_data_node_name_in_errors", true, false); - if (hide_node_name && strcmp(hide_node_name, "on") == 0) - { - return ""; - } -#endif - return NameStr(conn->node_name); -} - -void -remote_connection_get_error(const TSConnection *conn, TSConnectionError *err) -{ - fill_connection_error(err, ERRCODE_CONNECTION_FAILURE, NULL, conn); -} - -void -remote_connection_get_result_error(const PGresult *res, TSConnectionError *err) -{ - fill_result_error(err, ERRCODE_CONNECTION_EXCEPTION, NULL, res); -} - -static long -timeout_diff_ms(TimestampTz endtime) -{ - TimestampTz now; - long secs; - int microsecs; - - if (endtime == TS_NO_TIMEOUT) - return -1; - - now = GetCurrentTimestamp(); - if (now >= endtime) - return 0; - - TimestampDifference(now, endtime, &secs, µsecs); - return secs * 1000 + (microsecs / 1000); -} - -PGresult * -remote_connection_get_result(const TSConnection *conn, TimestampTz endtime) -{ - PGresult *pgres = NULL; - int busy = 1; - - do - { - CHECK_FOR_INTERRUPTS(); - - busy = PQisBusy(conn->pg_conn); - - if (busy == 1) - { - uint32 events; - WaitEvent event; - long timeout_ms; - int ret; - - events = WL_SOCKET_READABLE; - if (endtime != TS_NO_TIMEOUT) - events |= WL_TIMEOUT; - timeout_ms = timeout_diff_ms(endtime); - - /* Busy, wait for readable */ - ModifyWaitEvent(conn->wes, conn->sockeventpos, events, NULL); - ret = WaitEventSetWait(conn->wes, timeout_ms, &event, 1, PG_WAIT_EXTENSION); - - /* Timeout */ - if (ret == 0) - break; - - if (event.events & WL_LATCH_SET) - { - /* Check for interrupts at top of the loop */ - ResetLatch(MyLatch); - } - - if (event.events & WL_SOCKET_READABLE) - { - Assert(event.pos == conn->sockeventpos); - Assert(event.fd == PQsocket(conn->pg_conn)); - - if (PQconsumeInput(conn->pg_conn) == 0) - { - pgres = PQmakeEmptyPGresult(conn->pg_conn, PGRES_FATAL_ERROR); - TS_OOM_CHECK(pgres, "out of memory"); - PQfireResultCreateEvents(conn->pg_conn, pgres); - return pgres; - } - } - } - else if (busy == 0) - { - /* PQgetResult would not block */ - pgres = PQgetResult(conn->pg_conn); - } - else - { - pg_unreachable(); - Assert(false); - } - } while (busy == 1); - - return pgres; -} - -/* - * Execute a remote command. - * - * The execution blocks until a result is received or a failure occurs. Unlike - * PQexec() and PQexecParams(), however, this function observes PostgreSQL - * interrupts (e.g., a query is canceled). Like PQexecParams(), the PGresult - * returned describes only the last command executed in a multi-command - * string. - */ -PGresult * -remote_connection_exec_timeout(TSConnection *conn, const char *cmd, TimestampTz endtime) -{ - WaitEvent event; - PGresult *res = NULL; - int ret = 0; - size_t cmdlen = strlen(cmd); - StringInfoData cmd_buf = { - .data = (char *) cmd, - .len = cmdlen, - .maxlen = cmdlen + 1, - }; - - prepend_enforced_conn_settings(conn, &cmd_buf); - - do - { - uint32 events; - long timeout_ms; - - CHECK_FOR_INTERRUPTS(); - - events = WL_SOCKET_WRITEABLE; - if (endtime != TS_NO_TIMEOUT) - events |= WL_TIMEOUT; - timeout_ms = timeout_diff_ms(endtime); - - /* Wait for writable socket in outer loop */ - ModifyWaitEvent(conn->wes, conn->sockeventpos, events, NULL); - ret = WaitEventSetWait(conn->wes, timeout_ms, &event, 1, PG_WAIT_EXTENSION); - - /* Timeout */ - if (ret == 0) - break; - - if (event.events & WL_LATCH_SET) - { - ResetLatch(MyLatch); - CHECK_FOR_INTERRUPTS(); - } - if (event.events & WL_SOCKET_WRITEABLE) - { - PGresult *last_result; - - ret = PQsendQuery(conn->pg_conn, cmd_buf.data); - - if (ret == 0) - { - res = PQmakeEmptyPGresult(conn->pg_conn, PGRES_FATAL_ERROR); - TS_OOM_CHECK(res, "out of memory"); - PQfireResultCreateEvents(conn->pg_conn, res); - return res; - } - - /* Command sent, so now wait for readable result in inner loop */ - last_result = NULL; - - /* - * Read all results, but return only one in order to recreate the - * behavior of the blocking PQexec() call. We need to handle - * PG13 differently to be compatible across all versions of - * PostgreSQL. In libpq's PQexec() errors from all the results are - * concactinated, but that is not possible here due to lack of - * access to internals. PG14 handles that automatically, however. - */ - while ((res = remote_connection_get_result(conn, endtime)) != NULL) - { - if (last_result) - { -#if PG14_LT - if (PQresultStatus(last_result) == PGRES_FATAL_ERROR && - PQresultStatus(res) == PGRES_FATAL_ERROR) - { - PQclear(res); - res = last_result; - } - else - PQclear(last_result); -#else - PQclear(last_result); -#endif - } - - last_result = res; - - if (PQresultStatus(res) == PGRES_COPY_IN || PQresultStatus(res) == PGRES_COPY_OUT || - PQresultStatus(res) == PGRES_COPY_BOTH || - PQstatus(conn->pg_conn) == CONNECTION_BAD) - break; - } - - res = last_result; - } - } while (res == NULL); - - /* - * Workaround for the libpq disconnect case. - * - * libpq disconnect will create an empty result object without generating - * events, which is usually done for a regular errors. - * - * In order to be compatible with our error handling code, force - * create result event, if the result object does not have - * it already. - */ - if (res) - { - ResultEntry *entry = PQresultInstanceData(res, eventproc); - - if (entry == NULL) - PQfireResultCreateEvents(conn->pg_conn, res); - } - - return res; -} - -PGresult * -remote_connection_exec(TSConnection *conn, const char *cmd) -{ - return remote_connection_exec_timeout(conn, cmd, TS_NO_TIMEOUT); -} - -/* - * Must be a macro since va_start() must be called in the function that takes - * a variable number of arguments. - */ -#define stringinfo_va(fmt, sql) \ - do \ - { \ - initStringInfo((sql)); \ - for (;;) \ - { \ - va_list args; \ - int needed; \ - va_start(args, fmt); \ - needed = appendStringInfoVA((sql), fmt, args); \ - va_end(args); \ - if (needed == 0) \ - break; \ - /* Increase the buffer size and try again. */ \ - enlargeStringInfo((sql), needed); \ - } \ - } while (0); - -/* - * Execute a remote command. - * - * Like remote_connection_exec but takes a variable number of arguments. - */ -PGresult * -remote_connection_execf(TSConnection *conn, const char *fmt, ...) -{ - PGresult *res; - StringInfoData sql; - - stringinfo_va(fmt, &sql); - res = remote_connection_exec(conn, sql.data); - pfree(sql.data); - - return res; -} - -PGresult * -remote_connection_queryf_ok(TSConnection *conn, const char *fmt, ...) -{ - StringInfoData sql; - PGresult *res; - - stringinfo_va(fmt, &sql); - res = remote_result_query_ok(remote_connection_exec(conn, sql.data)); - pfree(sql.data); - return res; -} - -PGresult * -remote_connection_query_ok(TSConnection *conn, const char *query) -{ - return remote_result_query_ok(remote_connection_exec(conn, query)); -} - -void -remote_connection_cmd_ok(TSConnection *conn, const char *cmd) -{ - remote_result_cmd_ok(remote_connection_exec(conn, cmd)); -} - -void -remote_connection_cmdf_ok(TSConnection *conn, const char *fmt, ...) -{ - StringInfoData sql; - - stringinfo_va(fmt, &sql); - remote_result_cmd_ok(remote_connection_exec(conn, sql.data)); - pfree(sql.data); -} - -static PGresult * -remote_result_ok(PGresult *res, ExecStatusType expected) -{ - if (PQresultStatus(res) != expected) - remote_result_elog(res, ERROR); - - return res; -} - -void -remote_result_cmd_ok(PGresult *res) -{ - PQclear(remote_result_ok(res, PGRES_COMMAND_OK)); -} - -PGresult * -remote_result_query_ok(PGresult *res) -{ - return remote_result_ok(res, PGRES_TUPLES_OK); -} - -/** - * Validate extension version. - */ -void -remote_validate_extension_version(TSConnection *conn, const char *data_node_version) -{ - if (!dist_util_is_compatible_version(data_node_version, TIMESCALEDB_VERSION)) - ereport(ERROR, - (errcode(ERRCODE_TS_DATA_NODE_INVALID_CONFIG), - errmsg("remote PostgreSQL instance has an incompatible timescaledb extension " - "version"), - errdetail_internal("Access node version: %s, remote version: %s.", - TIMESCALEDB_VERSION_MOD, - data_node_version))); -} - -/* - * Check timescaledb extension version on a data node. - * - * Compare remote connection extension version with the one installed - * locally on the access node. - * - * Return false if extension is not found, true otherwise. - */ -bool -remote_connection_check_extension(TSConnection *conn) -{ - PGresult *res; - - res = remote_connection_execf(conn, - "SELECT extversion FROM pg_extension WHERE extname = %s", - quote_literal_cstr(EXTENSION_NAME)); - - /* Just to capture any bugs in the SELECT above */ - Assert(PQnfields(res) == 1); - - switch (PQntuples(res)) - { - case 0: /* extension does not exists */ - PQclear(res); - return false; - - case 1: - break; - - default: /* something strange happend */ - ereport(WARNING, - (errcode(ERRCODE_TS_DATA_NODE_INVALID_CONFIG), - errmsg("more than one TimescaleDB extension loaded"))); - break; - } - - /* validate extension version on data node and make sure that it is - * compatible */ - remote_validate_extension_version(conn, PQgetvalue(res, 0, 0)); - - PQclear(res); - return true; -} - -/* - * Configure remote connection using current instance UUID. - * - * This allows remote side to reason about whether this connection has been - * originated by access node. - * - * Returns true on success and false on error, in which case the optional - * errmsg parameter can be used to retrieve an error message. - */ -static bool -remote_connection_set_peer_dist_id(TSConnection *conn) -{ - bool isnull; - Datum uuid = ts_metadata_get_value(METADATA_UUID_KEY_NAME, UUIDOID, &isnull); - Datum id_string = DirectFunctionCall1(uuid_out, uuid); - PGresult *res; - bool success = true; - - res = remote_connection_execf(conn, - "SELECT * FROM _timescaledb_functions.set_peer_dist_id('%s')", - DatumGetCString(id_string)); - success = PQresultStatus(res) == PGRES_TUPLES_OK; - PQclear(res); - - return success; -} - -/* fallback_application_name, client_encoding, end marker */ -#define REMOTE_CONNECTION_SESSION_OPTIONS_N 3 - -/* passfile */ -#define REMOTE_CONNECTION_PASSWORD_OPTIONS_N 1 - -/* sslmode, sslrootcert, sslcert, sslkey */ -#define REMOTE_CONNECTION_SSL_OPTIONS_N 4 - -#define REMOTE_CONNECTION_OPTIONS_TOTAL_N \ - (REMOTE_CONNECTION_SESSION_OPTIONS_N + REMOTE_CONNECTION_PASSWORD_OPTIONS_N + \ - REMOTE_CONNECTION_SSL_OPTIONS_N) - -/* default password file basename */ -#define DEFAULT_PASSFILE_NAME "passfile" - -static void -set_password_options(const char **keywords, const char **values, int *option_start) -{ - int option_pos = *option_start; - - /* Set user specified password file path using timescaledb.passfile or - * use default path assuming that the file is stored in the - * data directory */ - keywords[option_pos] = "passfile"; - if (ts_guc_passfile) - values[option_pos] = ts_guc_passfile; - else - values[option_pos] = psprintf("%s/" DEFAULT_PASSFILE_NAME, DataDir); - option_pos++; - - *option_start = option_pos; -} - -typedef enum PathKind -{ - PATH_KIND_CRT, - PATH_KIND_KEY -} PathKind; - -/* Path description for human consumption */ -static const char *path_kind_text[PATH_KIND_KEY + 1] = { - [PATH_KIND_CRT] = "certificate", - [PATH_KIND_KEY] = "private key", -}; - -/* Path extension string for file system */ -static const char *path_kind_ext[PATH_KIND_KEY + 1] = { - [PATH_KIND_CRT] = "crt", - [PATH_KIND_KEY] = "key", -}; - -/* - * Helper function to report error. - * - * This is needed to avoid code coverage reporting low coverage for error - * cases in `make_user_path` that cannot be reached in normal situations. - */ -static void -report_path_error(PathKind path_kind, const char *user_name) -{ - elog(ERROR, - "cannot write %s for user \"%s\": path too long", - path_kind_text[path_kind], - user_name); -} - -/* - * Make a user path with the given extension and user name in a portable and - * safe manner. - * - * We use MD5 to compute a filename for the user name, which allows all forms - * of user names. It is not necessary for the function to be cryptographically - * secure, only to have a low risk of collisions, and MD5 is fast and with a - * low risk of collisions. - * - * Will return the resulting path, or abort with an error. - */ -static StringInfo -make_user_path(const char *user_name, PathKind path_kind) -{ - char ret_path[MAXPGPATH]; - char hexsum[33]; - StringInfo result; - const char *errstr; - - pg_md5_hash_compat(user_name, strlen(user_name), hexsum, &errstr); - - if (strlcpy(ret_path, ts_guc_ssl_dir ? ts_guc_ssl_dir : DataDir, MAXPGPATH) > MAXPGPATH) - report_path_error(path_kind, user_name); - canonicalize_path(ret_path); - - if (!ts_guc_ssl_dir) - { - join_path_components(ret_path, ret_path, EXTENSION_NAME); - join_path_components(ret_path, ret_path, "certs"); - } - - join_path_components(ret_path, ret_path, hexsum); - - result = makeStringInfo(); - appendStringInfo(result, "%s.%s", ret_path, path_kind_ext[path_kind]); - return result; -} - -static void -set_ssl_options(const char *user_name, const char **keywords, const char **values, - int *option_start) -{ - int option_pos = *option_start; - const char *ssl_enabled; - const char *ssl_ca_file; - - ssl_enabled = GetConfigOption("ssl", true, false); - - if (!ssl_enabled || strcmp(ssl_enabled, "on") != 0) - return; - -#ifndef NDEBUG - ssl_enabled = GetConfigOption("timescaledb.debug_enable_ssl", true, false); - if (ssl_enabled && strcmp(ssl_enabled, "on") != 0) - { - keywords[option_pos] = "sslmode"; - values[option_pos] = "disable"; - option_pos++; - *option_start = option_pos; - return; - } -#endif - - /* If SSL is enabled on AN then we assume it is also should be used for DN - * connections as well, otherwise we need to introduce some other way to - * control it */ - keywords[option_pos] = "sslmode"; - values[option_pos] = "require"; - option_pos++; - - ssl_ca_file = GetConfigOption("ssl_ca_file", true, false); - - /* Use ssl_ca_file as the root certificate when verifying the - * data node we connect to */ - if (ssl_ca_file) - { - keywords[option_pos] = "sslrootcert"; - values[option_pos] = ssl_ca_file; - option_pos++; - } - - /* Search for the user certificate in the user subdirectory of either - * timescaledb.ssl_dir or data directory. The user subdirectory is - * currently hardcoded. */ - - keywords[option_pos] = "sslcert"; - values[option_pos] = make_user_path(user_name, PATH_KIND_CRT)->data; - option_pos++; - - keywords[option_pos] = "sslkey"; - values[option_pos] = make_user_path(user_name, PATH_KIND_KEY)->data; - option_pos++; - - /* if ts_set_ssl_options_hook is enabled then invoke that hook */ - if (ts_set_ssl_options_hook) - ts_set_ssl_options_hook(user_name); - - *option_start = option_pos; -} - -/* - * Finish the connection and, optionally, save the connection error. - */ -static void -finish_connection(PGconn *conn, char **errmsg) -{ - if (NULL != errmsg) - { - if (NULL == conn) - *errmsg = "invalid connection"; - else - *errmsg = pchomp(PQerrorMessage(conn)); - } - - PQfinish(conn); -} - -/* - * Take options belonging to a foreign server and add additional default and - * other user/ssl related options as appropriate - */ -static void -setup_full_connection_options(List *connection_options, const char ***all_keywords, - const char ***all_values) -{ - const char *user_name = NULL; - const char **keywords; - const char **values; - int option_count; - int option_pos; - - /* - * Construct connection params from generic options of ForeignServer - * and user. (Some of them might not be libpq options, in - * which case we'll just waste a few array slots.) Add 3 extra slots - * for fallback_application_name, client_encoding, end marker. - * One additional slot to set passfile and 4 slots for ssl options. - */ - option_count = list_length(connection_options) + REMOTE_CONNECTION_OPTIONS_TOTAL_N; - keywords = (const char **) palloc(option_count * sizeof(char *)); - values = (const char **) palloc(option_count * sizeof(char *)); - - option_pos = extract_connection_options(connection_options, keywords, values, &user_name); - - if (NULL == user_name) - user_name = GetUserNameFromId(GetUserId(), false); - - /* Use the extension name as fallback_application_name. */ - keywords[option_pos] = "fallback_application_name"; - values[option_pos] = EXTENSION_NAME; - option_pos++; - - /* Set client_encoding so that libpq can convert encoding properly. */ - keywords[option_pos] = "client_encoding"; - values[option_pos] = GetDatabaseEncodingName(); - option_pos++; - - /* Set passfile options */ - set_password_options(keywords, values, &option_pos); - - /* Set client specific SSL connection options */ - set_ssl_options(user_name, keywords, values, &option_pos); - - /* Set end marker */ - keywords[option_pos] = values[option_pos] = NULL; - Assert(option_pos <= option_count); - - *all_keywords = keywords; - *all_values = values; -} - -/* - * Open a connection and assign it the given node name. - * - * This will only open a connection to a specific node, but not do any other - * session initialization. In particular, it will not perform any validation - * nor configure the connection since it cannot know that it connects to a - * data node database or not. For that, please use the - * `remote_connection_open_session` function. - * - * The connection's life-cycle is tied to the current memory context via its - * delete callback. As a result, the connection will be automatically closed - * and freed when the memory context is deleted. - * - * This function does not (and should not) throw (PostgreSQL) errors. Instead, - * an error message is optionally returned via the "errmsg" parameter. - */ -TSConnection * -remote_connection_open(const char *node_name, List *connection_options, TimestampTz endtime, - char **errmsg) -{ - PGconn *pg_conn = NULL; - TSConnection *ts_conn = NULL; - const char **keywords; - const char **values; - PostgresPollingStatusType status; - - if (NULL != errmsg) - *errmsg = NULL; - - setup_full_connection_options(connection_options, &keywords, &values); - pg_conn = PQconnectStartParams(keywords, values, 0 /* Do not expand dbname param */); - - /* Cast to (char **) to silence warning with MSVC compiler */ - pfree((char **) keywords); - pfree((char **) values); - - if (NULL == pg_conn) - return NULL; - - if (PQstatus(pg_conn) == CONNECTION_BAD) - { - finish_connection(pg_conn, errmsg); - return NULL; - } - - status = PGRES_POLLING_WRITING; - - do - { - long timeout_ms; - int events; - int io_flag; - int rc; - - if (status == PGRES_POLLING_READING) - io_flag = WL_SOCKET_READABLE; -#ifdef WIN32 - /* Windows needs a different test while waiting for connection-made */ - else if (PQstatus(pg_conn) == CONNECTION_STARTED) - io_flag = WL_SOCKET_CONNECTED; -#endif - else - io_flag = WL_SOCKET_WRITEABLE; - - if (endtime == TS_NO_TIMEOUT) - events = io_flag; - else - events = io_flag | WL_TIMEOUT; - - timeout_ms = timeout_diff_ms(endtime); - - /* - * Wait for latch or socket event. Note that it is not possible to - * reuse a WaitEventSet using the same socket file descriptor in each - * iteration of the loop since PQconnectPoll() might change the file - * descriptor across calls. Therefore, it is important to create a new - * WaitEventSet in every iteration of the loop and retreiving the - * correct file descriptor (socket) with PQsocket(). - */ - rc = WaitLatchOrSocket(MyLatch, - WL_EXIT_ON_PM_DEATH | WL_LATCH_SET | events, - PQsocket(pg_conn), - timeout_ms, - PG_WAIT_EXTENSION); - - if (rc & WL_TIMEOUT) - { - finish_connection(pg_conn, errmsg); - return NULL; - } - - if (rc & WL_LATCH_SET) - { - ResetLatch(MyLatch); - CHECK_FOR_INTERRUPTS(); - } - - if (rc & io_flag) - { - /* - * PQconnectPoll() is supposed to be non-blocking, but it - * isn't. PQconnectPoll() will internally try to send a startup - * packet and do DNS lookups (if necessary) and can therefore - * block. So, if there is a network issue (e.g., black hole - * routing) the connection attempt will hang on - * PQconnectPoll(). There's nothing that can be done about it, - * unless the blocking operations are moved out of PQconnectPoll() - * and integrated with the wait loop. - */ - status = PQconnectPoll(pg_conn); - } - } while (status != PGRES_POLLING_OK && status != PGRES_POLLING_FAILED); - - if (PQstatus(pg_conn) != CONNECTION_OK) - { - finish_connection(pg_conn, errmsg); - return NULL; - } - - ts_conn = remote_connection_create(pg_conn, false, node_name); - - if (NULL == ts_conn) - finish_connection(pg_conn, errmsg); - - return ts_conn; -} - -/* - * Open a connection to a data node and perform basic session initialization. - * - * This function will raise errors on failures. - */ -TSConnection * -remote_connection_open_session(const char *node_name, List *connection_options, bool set_dist_id) -{ - char *err = NULL; - TSConnection *conn = remote_connection_open(node_name, connection_options, TS_NO_TIMEOUT, &err); - - if (NULL == conn) - ereport(ERROR, - (errcode(ERRCODE_SQLCLIENT_UNABLE_TO_ESTABLISH_SQLCONNECTION), - errmsg("could not connect to \"%s\"", node_name), - err == NULL ? 0 : errdetail_internal("%s", err))); - - /* - * Use PG_TRY block to ensure closing connection on error. - */ - PG_TRY(); - { - Assert(NULL != conn->pg_conn); - - if (PQstatus(conn->pg_conn) != CONNECTION_OK) - ereport(ERROR, - (errcode(ERRCODE_SQLCLIENT_UNABLE_TO_ESTABLISH_SQLCONNECTION), - errmsg("could not connect to \"%s\"", node_name), - errdetail_internal("%s", pchomp(PQerrorMessage(conn->pg_conn))))); - - /* Prepare new session for use */ - if (!remote_connection_configure(conn)) - ereport(ERROR, - (errcode(ERRCODE_SQLCLIENT_UNABLE_TO_ESTABLISH_SQLCONNECTION), - errmsg("could not configure remote connection to \"%s\"", node_name), - errdetail_internal("%s", PQerrorMessage(conn->pg_conn)))); - - /* Check a data node extension version and show a warning - * message if it differs */ - remote_connection_check_extension(conn); - - if (set_dist_id) - { - /* Inform remote node about instance UUID */ - if (!remote_connection_set_peer_dist_id(conn)) - ereport(ERROR, - (errcode(ERRCODE_SQLCLIENT_UNABLE_TO_ESTABLISH_SQLCONNECTION), - errmsg("could not set distributed ID for \"%s\"", node_name), - errdetail_internal("%s", PQerrorMessage(conn->pg_conn)))); - } - } - PG_CATCH(); - { - /* Release PGconn data structure if we managed to create one */ - remote_connection_close(conn); - PG_RE_THROW(); - } - PG_END_TRY(); - - return conn; -} - -TSConnection * -remote_connection_open_session_by_id(TSConnectionId id) -{ - ForeignServer *server = GetForeignServer(id.server_id); - List *connection_options = remote_connection_prepare_auth_options(server, id.user_id); - - return remote_connection_open_session(server->servername, connection_options, true); -} - -/* - * Based on PG's GetUserMapping, but this version does not fail when a user - * mapping is not found. - */ -static UserMapping * -get_user_mapping(Oid userid, Oid serverid) -{ - Datum datum; - HeapTuple tp; - bool isnull; - UserMapping *um; - - tp = SearchSysCache2(USERMAPPINGUSERSERVER, - ObjectIdGetDatum(userid), - ObjectIdGetDatum(serverid)); - - if (!HeapTupleIsValid(tp)) - { - /* Not found for the specific user -- try PUBLIC */ - tp = SearchSysCache2(USERMAPPINGUSERSERVER, - ObjectIdGetDatum(InvalidOid), - ObjectIdGetDatum(serverid)); - } - - if (!HeapTupleIsValid(tp)) - return NULL; - - um = (UserMapping *) palloc(sizeof(UserMapping)); - um->umid = ((Form_pg_user_mapping) GETSTRUCT(tp))->oid; - um->userid = userid; - um->serverid = serverid; - - /* Extract the umoptions */ - datum = SysCacheGetAttr(USERMAPPINGUSERSERVER, tp, Anum_pg_user_mapping_umoptions, &isnull); - if (isnull) - um->options = NIL; - else - um->options = untransformRelOptions(datum); - - ReleaseSysCache(tp); - - return um; -} - -static bool -options_contain(List *options, const char *key) -{ - ListCell *lc; - - foreach (lc, options) - { - DefElem *d = (DefElem *) lfirst(lc); - - if (strcmp(d->defname, key) == 0) - return true; - } - - return false; -} - -/* - * Add athentication info (username and optionally password) to the connection - * options). - */ -List * -remote_connection_prepare_auth_options(const ForeignServer *server, Oid user_id) -{ - const UserMapping *um = get_user_mapping(user_id, server->serverid); - List *options = list_copy(server->options); - - /* If a user mapping exists, then use the "user" and "password" options - * from the user mapping (we assume that these options exist, or the - * connection will later fail). Otherwise, just add the "user" and rely on - * other authentication mechanisms. */ - if (NULL != um) - options = list_concat(options, um->options); - - if (!options_contain(options, "user")) - { - char *user_name = GetUserNameFromId(user_id, false); - options = lappend(options, makeDefElem("user", (Node *) makeString(user_name), -1)); - } - - return options; -} - -/* - * Append the given string to the buffer, with suitable quoting for passing - * the string as a value in a keyword/value pair in a libpq connection string. - * - * The implementation is based on libpq appendConnStrVal(). - */ -static void -remote_connection_append_connstr_value(StringInfo buf, const char *str) -{ - const char *s; - bool needquotes; - - /* - * If the string is one or more plain ASCII characters, no need to quote - * it. This is quite conservative, but better safe than sorry. - */ - needquotes = true; - for (s = str; *s; s++) - { - if (!((*s >= 'a' && *s <= 'z') || (*s >= 'A' && *s <= 'Z') || (*s >= '0' && *s <= '9') || - *s == '_' || *s == '.')) - { - needquotes = true; - break; - } - needquotes = false; - } - - if (needquotes) - { - appendStringInfoChar(buf, '\''); - while (*str) - { - /* ' and \ must be escaped by to \' and \\ */ - if (*str == '\'' || *str == '\\') - appendStringInfoChar(buf, '\\'); - - appendStringInfoChar(buf, *str); - str++; - } - appendStringInfoChar(buf, '\''); - } - else - appendStringInfoString(buf, str); -} - -char * -remote_connection_get_connstr(const char *node_name) -{ - ForeignServer *server; - List *connection_options; - const char **keywords; - const char **values; - StringInfoData connstr; - StringInfoData connstr_escape; - int i; - - server = data_node_get_foreign_server(node_name, ACL_NO_CHECK, false, false); - connection_options = remote_connection_prepare_auth_options(server, GetUserId()); - setup_full_connection_options(connection_options, &keywords, &values); - - /* Cycle through the options and create the connection string */ - initStringInfo(&connstr); - i = 0; - while (keywords[i] != NULL) - { - appendStringInfo(&connstr, " %s=", keywords[i]); - remote_connection_append_connstr_value(&connstr, values[i]); - i++; - } - Assert(keywords[i] == NULL && values[i] == NULL); - - initStringInfo(&connstr_escape); - enlargeStringInfo(&connstr_escape, connstr.len * 2 + 1); - connstr_escape.len += PQescapeString(connstr_escape.data, connstr.data, connstr.len); - - /* Cast to (char **) to silence warning with MSVC compiler */ - pfree((char **) keywords); - pfree((char **) values); - pfree(connstr.data); - - return connstr_escape.data; -} - -#define PING_QUERY "SELECT 1" - -bool -remote_connection_ping(const char *node_name, TimestampTz endtime) -{ - Oid server_id = get_foreign_server_oid(node_name, false); - ForeignServer *server = GetForeignServer(server_id); - Oid fdwid = get_foreign_data_wrapper_oid(EXTENSION_FDW_NAME, false); - List *connection_options; - TSConnection *conn; - bool success = false; - - if (server->fdwid != fdwid) - { - elog(WARNING, "invalid node type for \"%s\"", server->servername); - return false; - } - - connection_options = remote_connection_prepare_auth_options(server, GetUserId()); - conn = remote_connection_open(server->servername, connection_options, endtime, NULL); - - if (NULL == conn) - return false; - - if (PQstatus(conn->pg_conn) == CONNECTION_OK) - { - PGresult *res = remote_connection_exec_timeout(conn, PING_QUERY, endtime); - success = (PQresultStatus(res) == PGRES_TUPLES_OK); - } - - remote_connection_close(conn); - - return success; -} - -void -remote_connection_close(TSConnection *conn) -{ - /* The PQfinish callback handler will take care of freeing the resources, - * including the TSConnection object. */ - PQfinish(conn->pg_conn); -} - -/* - * Assign a "unique" number for a cursor. - * - * TODO should this be moved into the session? - * - * These really only need to be unique per connection within a transaction. - * For the moment we ignore the per-connection point and assign them across - * all connections in the transaction, but we ask for the connection to be - * supplied in case we want to refine that. - * - * Note that even if wraparound happens in a very long transaction, actual - * collisions are highly improbable; just be sure to use %u not %d to print. - */ -unsigned int -remote_connection_get_cursor_number() -{ - return ++cursor_number; -} - -void -remote_connection_reset_cursor_number() -{ - cursor_number = 0; -} - -/* - * Assign a "unique" number for a prepared statement. - * - * This works much like remote_connection_get_cursor_number, except that we never reset the counter - * within a session. That's because we can't be 100% sure we've gotten rid - * of all prepared statements on all connections, and it's not really worth - * increasing the risk of prepared-statement name collisions by resetting. - */ -unsigned int -remote_connection_get_prep_stmt_number() -{ - return ++prep_stmt_number; -} - -#define MAX_CONN_WAIT_TIMEOUT_MS 60000 - -/* - * Drain a connection of all data coming in and discard the results. Return - * CONN_OK if all data is drained before the deadline expires. - * - * This is mainly used in abort processing. This result being returned - * might be for a query that is being interrupted by transaction abort, or it might - * be a query that was initiated as part of transaction abort to get the remote - * side back to the appropriate state. - * - * It's not a huge problem if we throw an ERROR here, but if we get into error - * recursion trouble, we'll end up slamming the connection shut, which will - * necessitate failing the entire toplevel transaction even if subtransactions - * were used. Try to use WARNING where we can. - * - * end_time is the time at which we should give up and assume the remote - * side is dead. - */ -TSConnectionResult -remote_connection_drain(TSConnection *conn, TimestampTz endtime, PGresult **result) -{ - volatile TSConnectionResult connresult = CONN_OK; - PGresult *volatile last_res = NULL; - PGconn *pg_conn = remote_connection_get_pg_conn(conn); - - /* In what follows, do not leak any PGresults on an error. */ - PG_TRY(); - { - for (;;) - { - PGresult *res; - - while (PQisBusy(pg_conn)) - { - int wc; - TimestampTz now = GetCurrentTimestamp(); - long remaining_secs; - int remaining_usecs; - long cur_timeout_ms; - - /* If timeout has expired, give up, else get sleep time. */ - if (now >= endtime) - { - connresult = CONN_TIMEOUT; - goto exit; - } - - TimestampDifference(now, endtime, &remaining_secs, &remaining_usecs); - - /* To protect against clock skew, limit sleep to one minute. */ - cur_timeout_ms = - Min(MAX_CONN_WAIT_TIMEOUT_MS, remaining_secs * USECS_PER_SEC + remaining_usecs); - - /* Sleep until there's something to do */ - wc = WaitLatchOrSocket(MyLatch, - WL_LATCH_SET | WL_SOCKET_READABLE | WL_EXIT_ON_PM_DEATH | - WL_TIMEOUT, - PQsocket(pg_conn), - cur_timeout_ms, - PG_WAIT_EXTENSION); - ResetLatch(MyLatch); - - CHECK_FOR_INTERRUPTS(); - - /* Data available in socket? */ - if ((wc & WL_SOCKET_READABLE) && (0 == PQconsumeInput(pg_conn))) - { - connresult = CONN_DISCONNECT; - goto exit; - } - } - - res = PQgetResult(pg_conn); - - if (res == NULL) - { - /* query is complete */ - remote_connection_set_status(conn, CONN_IDLE); - connresult = CONN_OK; - break; - } - else if (PQresultStatus(res) == PGRES_COPY_OUT) - { - /* - * We are inside the COPY subprotocol, need to sychronize with - * the server. - */ - int end_res = PQendcopy(pg_conn); - if (end_res != 0) - { - TSConnectionError err; - remote_connection_get_error(conn, &err); - remote_connection_error_elog(&err, WARNING); - } - } - - PQclear(last_res); - last_res = res; - } - exit:; - } - PG_CATCH(); - { - PQclear(last_res); - PG_RE_THROW(); - } - PG_END_TRY(); - - switch (connresult) - { - case CONN_OK: - if (last_res == NULL) - connresult = CONN_NO_RESPONSE; - else if (result != NULL) - *result = last_res; - else - PQclear(last_res); - break; - case CONN_TIMEOUT: - case CONN_DISCONNECT: - PQclear(last_res); - break; - case CONN_NO_RESPONSE: - Assert(last_res == NULL); - break; - } - - return connresult; -} - -/* - * Cancel the currently-in-progress query and ignore the result. Returns true if we successfully - * cancel the query and discard any pending result, and false if not. - */ -bool -remote_connection_cancel_query(TSConnection *conn) -{ - PGcancel *cancel; - char errbuf[256]; - TimestampTz endtime; - TSConnectionError err; - bool success; - - if (!conn) - return true; - - memset(&err, 0, sizeof(TSConnectionError)); - - /* - * Catch exceptions so that we can ensure the status is IDLE after the - * cancel operation even in case of errors being thrown. Note that we - * cannot set the status before we drain, since the drain function needs - * to know the status (e.g., if the connection is in COPY_IN mode). - */ - PG_TRY(); - { - if (conn->status == CONN_COPY_IN && !remote_connection_end_copy(conn, &err)) - remote_connection_error_elog(&err, WARNING); - - /* - * If it takes too long to cancel the query and discard the result, assume - * the connection is dead. - */ - endtime = TimestampTzPlusMilliseconds(GetCurrentTimestamp(), 30000); - - /* - * Issue cancel request. Unfortunately, there's no good way to limit the - * amount of time that we might block inside PQcancel(). - */ - if ((cancel = PQgetCancel(conn->pg_conn))) - { - if (!PQcancel(cancel, errbuf, sizeof(errbuf))) - { - ereport(WARNING, - (errcode(ERRCODE_CONNECTION_FAILURE), - errmsg("could not send cancel request: %s", errbuf))); - PQfreeCancel(cancel); - remote_connection_set_status(conn, CONN_IDLE); - return false; - } - PQfreeCancel(cancel); - } - - switch (remote_connection_drain(conn, endtime, NULL)) - { - case CONN_OK: - /* Successfully, drained */ - case CONN_NO_RESPONSE: - /* No response, likely beceause there was nothing to cancel */ - success = true; - break; - default: - success = false; - break; - } - } - PG_CATCH(); - { - remote_connection_set_status(conn, CONN_IDLE); - PG_RE_THROW(); - } - PG_END_TRY(); - - remote_connection_set_status(conn, CONN_IDLE); - - return success; -} - -void -remote_result_close(PGresult *res) -{ - PQclear(res); -} - -bool -remote_connection_set_single_row_mode(TSConnection *conn) -{ - return PQsetSingleRowMode(conn->pg_conn); -} - -static bool -send_binary_copy_header(const TSConnection *conn, TSConnectionError *err) -{ - /* File header for binary format */ - static const char file_header[] = { - 'P', 'G', 'C', 'O', 'P', 'Y', '\n', '\377', '\r', '\n', '\0', /* Signature */ - 0, 0, 0, 0, /* 4 bytes flags */ - 0, 0, 0, 0 /* 4 bytes header extension length (unused) */ - }; - - int res = PQputCopyData(conn->pg_conn, file_header, sizeof(file_header)); - - if (res != 1) - return fill_connection_error(err, - ERRCODE_CONNECTION_FAILURE, - "could not set binary COPY mode", - conn); - return true; -} - -bool -remote_connection_begin_copy(TSConnection *conn, const char *copycmd, bool binary, - TSConnectionError *err) -{ - PGconn *pg_conn = remote_connection_get_pg_conn(conn); - PGresult *volatile res = NULL; - - if (PQisnonblocking(pg_conn)) - return fill_simple_error(err, - ERRCODE_FEATURE_NOT_SUPPORTED, - "distributed copy doesn't support non-blocking connections", - conn); - - if (conn->status != CONN_IDLE) - return fill_simple_error(err, - ERRCODE_INTERNAL_ERROR, - "connection not IDLE when beginning COPY", - conn); - -#ifndef NDEBUG - /* Set some variables for testing. */ - const char *error_after_option = - GetConfigOption("timescaledb.debug_broken_sendrecv_error_after", true, false); - if (error_after_option) - { - res = PQexec(pg_conn, - psprintf("set timescaledb.debug_broken_sendrecv_error_after = '%s';", - error_after_option)); - if (PQresultStatus(res) != PGRES_COMMAND_OK) - { - remote_connection_get_result_error(res, err); - PQclear(res); - return false; - } - PQclear(res); - } -#endif - - /* Run the COPY query. */ - res = PQexec(pg_conn, copycmd); - - if (PQresultStatus(res) != PGRES_COPY_IN) - { - fill_result_error(err, - ERRCODE_CONNECTION_FAILURE, - "unable to start remote COPY on data node", - res); - PQclear(res); - return false; - } - - PQclear(res); - - if (binary && !send_binary_copy_header(conn, err)) - goto err_end_copy; - - /* Switch the connection into nonblocking mode for the duration of COPY. */ - if (PQsetnonblocking(pg_conn, 1) != 0) - { - (void) fill_simple_error(err, - ERRCODE_CONNECTION_EXCEPTION, - "failed to set the connection into nonblocking mode", - conn); - goto err_end_copy; - } - - conn->binary_copy = binary; - remote_connection_set_status(conn, CONN_COPY_IN); - - return true; -err_end_copy: - PQputCopyEnd(pg_conn, err->msg); - - return false; -} - -int -remote_connection_put_copy_data(TSConnection *conn, const char *buffer, size_t len, - TSConnectionError *err) -{ - int res; - - res = PQputCopyData(remote_connection_get_pg_conn(conn), buffer, len); - - if (res == -1) - return fill_connection_error(err, - ERRCODE_CONNECTION_EXCEPTION, - "could not send COPY data", - conn); - - return res; -} - -static bool -send_end_binary_copy_data(const TSConnection *conn, TSConnectionError *err) -{ - const uint16 buf = pg_hton16((uint16) -1); - - if (PQputCopyData(conn->pg_conn, (char *) &buf, sizeof(buf)) != 1) - return fill_simple_error(err, ERRCODE_INTERNAL_ERROR, "could not end binary COPY", conn); - - return true; -} - -/* - * End COPY on the remote connection. - * This function is used to clean up after errors as well, so it works in a - * recovery fashion: it tries to bring the connection into predictable and - * usable state, even if there are some errors or discrepancies between its - * actual and expected state. - */ -bool -remote_connection_end_copy(TSConnection *conn, TSConnectionError *err) -{ - PGresult *res = NULL; - - /* - * In any case, try to switch the connection into the blocking mode, because - * that's what the non-COPY code expects. - */ - if (PQisnonblocking(conn->pg_conn)) - { - /* - * We have to flush the connection before we can switch it into blocking - * mode. - */ - for (;;) - { - CHECK_FOR_INTERRUPTS(); - - int flush_result = PQflush(conn->pg_conn); - - if (flush_result == 1) - { - /* - * In some rare cases, flush might report that it's busy, but - * actually there was an error and the socket became invalid. - * Check for it. This is something we have observed in COPY - * queries used for performance testing with tsbench, but not - * sure how it happens exactly, must be in the depths of - * pqReadData called by pqFlush. - */ - int socket = PQsocket(conn->pg_conn); - if (socket == PGINVALID_SOCKET) - { - return fill_connection_error(err, - ERRCODE_CONNECTION_EXCEPTION, - "failed to flush the COPY connection", - conn); - } - - /* - * The socket is busy, wait. We don't care about the wait result - * here, because whether it is a timeout or the socket became - * writeable, we just retry. - */ - (void) WaitLatchOrSocket(MyLatch, - WL_TIMEOUT | WL_SOCKET_WRITEABLE | WL_EXIT_ON_PM_DEATH, - socket, - /* timeout = */ 1000, - /* wait_event_info = */ 0); - } - else if (flush_result == 0) - { - /* Flushed all. */ - break; - } - else - { - /* Error. */ - return fill_connection_error(err, - ERRCODE_CONNECTION_EXCEPTION, - "failed to flush the COPY connection", - conn); - } - } - - /* Switch the connection into blocking mode. */ - if (PQsetnonblocking(conn->pg_conn, 0) != 0) - { - return fill_connection_error(err, - ERRCODE_CONNECTION_EXCEPTION, - "failed to set the connection into blocking mode", - conn); - } - } - - /* - * Shouldn't have been called for a connection we know is not in COPY mode. - */ - if (conn->status != CONN_COPY_IN) - return fill_simple_error(err, - ERRCODE_INTERNAL_ERROR, - "connection not in COPY_IN state when ending COPY", - conn); - - /* - * Check whether it's still in COPY mode. The dist_copy manages COPY - * protocol itself because it needs to work with multiple connections - * concurrently. The remote server might leave the COPY mode for own - * reasons, as well. If we discover this, update our info with the actual - * status, but still report the error. - */ - res = PQgetResult(conn->pg_conn); - if (res == NULL || PQresultStatus(res) != PGRES_COPY_IN) - { - remote_connection_set_status(conn, res == NULL ? CONN_IDLE : CONN_PROCESSING); - } - - /* - * Finish the COPY if needed. - */ - if (remote_connection_get_status(conn) == CONN_COPY_IN) - { - if (conn->binary_copy && !send_end_binary_copy_data(conn, err)) - return false; - - if (PQputCopyEnd(conn->pg_conn, NULL) != 1) - return fill_connection_error(err, - ERRCODE_CONNECTION_EXCEPTION, - "could not end remote COPY", - conn); - - remote_connection_set_status(conn, CONN_PROCESSING); - } - - bool success = true; - while ((res = PQgetResult(conn->pg_conn))) - { - ExecStatusType status = PQresultStatus(res); - if (status != PGRES_COMMAND_OK) - { - success = false; - remote_connection_get_result_error(res, err); - } - } - - remote_connection_set_status(conn, CONN_IDLE); - - return success; -} - -#ifdef TS_DEBUG -/* - * Reset the current connection stats. - */ -void -remote_connection_stats_reset(void) -{ - MemSet(&connstats, 0, sizeof(RemoteConnectionStats)); -} - -/* - * Get the current connection stats. - */ -RemoteConnectionStats * -remote_connection_stats_get(void) -{ - return &connstats; -} -#endif - -/* - * Cleanup connections and results at the end of a (sub-)transaction. - * - * This function is called at the end of transactions and sub-transactions to - * auto-cleanup connections and result objects. - */ -static void -remote_connections_xact_cleanup(SubTransactionId subtxid, bool isabort) -{ - ListNode *curr = connections.next; - unsigned int num_connections = 0; - unsigned int num_results = 0; - - while (curr != &connections) - { - TSConnection *conn = (TSConnection *) curr; - - /* Move to next connection since closing the current one might - * otherwise make the curr pointer invalid. */ - curr = curr->next; - - /* We're not closing the connection, but we should clean up any - * lingering results */ - ListNode *curr_result = conn->results.next; - - while (curr_result != &conn->results) - { - ResultEntry *entry = (ResultEntry *) curr_result; - - curr_result = curr_result->next; - - if (subtxid == InvalidSubTransactionId || subtxid == entry->subtxid) - { - PQclear(entry->result); - num_results++; - } - } - } - - if (subtxid == InvalidSubTransactionId) - elog(DEBUG3, - "cleaned up %u connections and %u results at %s of transaction", - num_connections, - num_results, - isabort ? "abort" : "commit"); - else - elog(DEBUG3, - "cleaned up %u connections and %u results at %s of sub-transaction %u", - num_connections, - num_results, - isabort ? "abort" : "commit", - subtxid); -} - -static void -remote_connection_xact_end(XactEvent event, void *unused_arg) -{ - /* - * We are deep down in CommitTransaction code path. We do not want our - * emit_log_hook_callback to interfere since it uses its own transaction - */ - emit_log_hook_type prev_emit_log_hook = emit_log_hook; - emit_log_hook = NULL; - - switch (event) - { - case XACT_EVENT_ABORT: - case XACT_EVENT_PARALLEL_ABORT: - /* - * We expect that the waitpoint will be retried and then we - * will return due to the process receiving a SIGTERM if - * the advisory lock is exclusively held by a user call - */ - DEBUG_RETRY_WAITPOINT("remote_conn_xact_end"); - remote_connections_xact_cleanup(InvalidSubTransactionId, true); - break; - case XACT_EVENT_COMMIT: - case XACT_EVENT_PARALLEL_COMMIT: - /* Same retry behavior as above */ - DEBUG_RETRY_WAITPOINT("remote_conn_xact_end"); - remote_connections_xact_cleanup(InvalidSubTransactionId, false); - break; - case XACT_EVENT_PREPARE: - /* - * We expect that the waitpoint will be retried and then we - * will return with a warning on crossing the retry count if - * the advisory lock is exclusively held by a user call - */ - DEBUG_RETRY_WAITPOINT("remote_conn_xact_end"); - break; - default: - /* other events are too early to use DEBUG_WAITPOINT.. */ - break; - } - - /* re-enable the emit_log_hook */ - emit_log_hook = prev_emit_log_hook; -} - -static void -remote_connection_subxact_end(SubXactEvent event, SubTransactionId subtxid, - SubTransactionId parent_subtxid, void *unused_arg) -{ - /* - * We are deep down in CommitTransaction code path. We do not want our - * emit_log_hook_callback to interfere since it uses its own transaction - */ - emit_log_hook_type prev_emit_log_hook = emit_log_hook; - emit_log_hook = NULL; - - switch (event) - { - case SUBXACT_EVENT_ABORT_SUB: - remote_connections_xact_cleanup(subtxid, true); - break; - case SUBXACT_EVENT_COMMIT_SUB: - remote_connections_xact_cleanup(subtxid, false); - break; - default: - break; - } - - /* re-enable the emit_log_hook */ - emit_log_hook = prev_emit_log_hook; -} - -void -_remote_connection_init(void) -{ - RegisterXactCallback(remote_connection_xact_end, NULL); - RegisterSubXactCallback(remote_connection_subxact_end, NULL); - unset_libpq_envvar(); -} - -void -_remote_connection_fini(void) -{ - UnregisterXactCallback(remote_connection_xact_end, NULL); - UnregisterSubXactCallback(remote_connection_subxact_end, NULL); -} diff --git a/tsl/src/remote/connection.h b/tsl/src/remote/connection.h deleted file mode 100644 index 42f01c52985..00000000000 --- a/tsl/src/remote/connection.h +++ /dev/null @@ -1,209 +0,0 @@ -/* - * 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. - */ -#pragma once - -#include -#include -#include - -#include "async.h" - -typedef struct TSConnection TSConnection; - -/* Associated with a connection foreign server and user id */ -typedef struct TSConnectionId -{ - Oid server_id; - Oid user_id; -} TSConnectionId; - -typedef enum ConnOptionType -{ - CONN_OPTION_TYPE_NONE, - CONN_OPTION_TYPE_USER, - CONN_OPTION_TYPE_NODE, -} ConnOptionType; - -typedef struct TSConnectionError -{ - /* Local error information */ - int errcode; - const char *msg; - const char *host; - const char *nodename; - const char *connmsg; - /* Remote error information, if available */ - struct - { - int elevel; - int errcode; - const char *sqlstate; - const char *msg; - const char *hint; - const char *detail; - const char *context; - const char *stmtpos; - const char *sqlcmd; - } remote; -} TSConnectionError; - -/* Open a connection with a remote endpoint. Note that this is a raw - * connection that does not obey txn semantics and is allocated using - * malloc. Most users should use `remote_dist_txn_get_connection` or - * `remote_connection_cache_get_connection` instead. Must be closed with - * `remote_connection_close` - */ -extern TSConnection *remote_connection_open(const char *node_name, List *connection_options, - TimestampTz endtime, char **errmsg); -extern TSConnection *remote_connection_open_session(const char *node_name, List *connection_options, - bool set_dist_id); -extern TSConnection *remote_connection_open_session_by_id(TSConnectionId id); -extern List *remote_connection_prepare_auth_options(const ForeignServer *server, Oid user_id); -extern int remote_connection_xact_depth_get(const TSConnection *conn); -extern int remote_connection_xact_depth_inc(TSConnection *conn); -extern int remote_connection_xact_depth_dec(TSConnection *conn); -extern void remote_connection_xact_transition_begin(TSConnection *conn); -extern void remote_connection_xact_transition_end(TSConnection *conn); -extern bool remote_connection_xact_is_transitioning(const TSConnection *conn); -extern bool remote_connection_ping(const char *node_name, TimestampTz endtime); -extern void remote_connection_close(TSConnection *conn); -extern PGresult *remote_connection_get_result(const TSConnection *conn, TimestampTz endtime); -extern PGresult *remote_connection_exec_timeout(TSConnection *conn, const char *cmd, - TimestampTz endtime); -extern PGresult *remote_connection_exec(TSConnection *conn, const char *cmd); -extern PGresult *remote_connection_execf(TSConnection *conn, const char *fmt, ...) - pg_attribute_printf(2, 3); -extern PGresult *remote_connection_query_ok(TSConnection *conn, const char *query); -extern PGresult *remote_connection_queryf_ok(TSConnection *conn, const char *fmt, ...) - pg_attribute_printf(2, 3); -extern void remote_connection_cmd_ok(TSConnection *conn, const char *cmd); -extern void remote_connection_cmdf_ok(TSConnection *conn, const char *fmt, ...) - pg_attribute_printf(2, 3); -extern ConnOptionType remote_connection_option_type(const char *keyword); -extern bool remote_connection_valid_user_option(const char *keyword); -extern bool remote_connection_valid_node_option(const char *keyword); -extern unsigned int remote_connection_get_cursor_number(void); -extern void remote_connection_reset_cursor_number(void); -extern unsigned int remote_connection_get_prep_stmt_number(void); -extern bool remote_connection_configure(TSConnection *conn); -extern bool remote_connection_check_extension(TSConnection *conn); -extern void remote_validate_extension_version(TSConnection *conn, const char *data_node_version); -extern char *remote_connection_get_connstr(const char *node_name); - -typedef enum TSConnectionResult -{ - CONN_OK, - CONN_TIMEOUT, - CONN_DISCONNECT, - CONN_NO_RESPONSE, -} TSConnectionResult; - -typedef enum TSConnectionStatus -{ - CONN_IDLE, /* No command being processed */ - CONN_PROCESSING, /* Command/query is being processed */ - CONN_COPY_IN, /* Connection is in COPY_IN mode */ -} TSConnectionStatus; - -TSConnectionResult remote_connection_drain(TSConnection *conn, TimestampTz endtime, - PGresult **result); -extern bool remote_connection_cancel_query(TSConnection *conn); -extern PGconn *remote_connection_get_pg_conn(const TSConnection *conn); -extern bool remote_connection_is_processing(const TSConnection *conn); -extern void remote_connection_set_status(TSConnection *conn, TSConnectionStatus status); -extern TSConnectionStatus remote_connection_get_status(const TSConnection *conn); -extern bool remote_connection_configure_if_changed(TSConnection *conn); -extern const char *remote_connection_node_name(const TSConnection *conn); -extern bool remote_connection_set_single_row_mode(TSConnection *conn); - -/* Functions operating on PGresult objects */ -extern void remote_result_cmd_ok(PGresult *res); -extern PGresult *remote_result_query_ok(PGresult *res); -extern void remote_result_close(PGresult *res); - -/* wrappers around async stuff to emulate sync communication */ - -extern TSConnectionId remote_connection_id(const Oid server_oid, const Oid user_oid); -extern void remote_connection_id_set(TSConnectionId *const id, const Oid server_oid, - const Oid user_oid); - -typedef struct RemoteConnectionStats -{ - unsigned int connections_created; - unsigned int connections_closed; - unsigned int results_created; - unsigned int results_cleared; -} RemoteConnectionStats; - -#ifdef TS_DEBUG -extern void remote_connection_stats_reset(void); -extern RemoteConnectionStats *remote_connection_stats_get(void); -#endif - -/* - * Connection functions for COPY mode. - */ -extern bool remote_connection_begin_copy(TSConnection *conn, const char *copycmd, bool binary, - TSConnectionError *err); -extern bool remote_connection_end_copy(TSConnection *conn, TSConnectionError *err); -extern int remote_connection_put_copy_data(TSConnection *conn, const char *buffer, size_t len, - TSConnectionError *err); - -/* Error handling functions for connections */ -extern void remote_connection_get_error(const TSConnection *conn, TSConnectionError *err); -extern void remote_connection_get_result_error(const PGresult *res, TSConnectionError *err); - -/* - * The following are macros for emitting errors related to connections or - * remote command execution. They need to be macros to preserve the error - * context of where they are called (line number, statement, etc.). - */ -#define remote_connection_error_elog(err, elevel) \ - ereport(elevel, \ - ((err)->remote.errcode != 0 ? errcode((err)->remote.errcode) : \ - errcode((err)->errcode), \ - (err)->remote.msg ? \ - errmsg_internal("[%s]: %s", (err)->nodename, (err)->remote.msg) : \ - ((err)->connmsg ? errmsg_internal("[%s]: %s", (err)->nodename, (err)->connmsg) : \ - errmsg_internal("[%s]: %s", (err)->nodename, (err)->msg)), \ - (err)->remote.detail ? errdetail_internal("%s", (err)->remote.detail) : 0, \ - (err)->remote.hint ? errhint("%s", (err)->remote.hint) : 0, \ - (err)->remote.sqlcmd ? errcontext("Remote SQL command: %s", (err)->remote.sqlcmd) : \ - 0)) - -/* - * Report an error we got from the remote host. - * - * elevel: error level to use (typically ERROR, but might be less) - * res: PGresult containing the error - */ -#define remote_result_elog(pgres, elevel) \ - do \ - { \ - PG_TRY(); \ - { \ - TSConnectionError err; \ - remote_connection_get_result_error(pgres, &err); \ - remote_connection_error_elog(&err, elevel); \ - } \ - PG_CATCH(); \ - { \ - PQclear(pgres); \ - PG_RE_THROW(); \ - } \ - PG_END_TRY(); \ - } while (0) - -#define remote_connection_elog(conn, elevel) \ - do \ - { \ - TSConnectionError err; \ - remote_connection_get_error(conn, &err); \ - remote_connection_error_elog(&err, elevel); \ - } while (0) - -extern void _remote_connection_init(void); -extern void _remote_connection_fini(void); diff --git a/tsl/src/remote/connection_cache.c b/tsl/src/remote/connection_cache.c deleted file mode 100644 index 313d73ba353..00000000000 --- a/tsl/src/remote/connection_cache.c +++ /dev/null @@ -1,518 +0,0 @@ -/* - * 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. - */ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include "connection_cache.h" - -static Cache *connection_cache = NULL; -static bool ignore_connection_invalidation = false; - -typedef struct ConnectionCacheEntry -{ - TSConnectionId id; - TSConnection *conn; - uint32 foreign_server_hashvalue; /* Hash of server OID for cache invalidation */ - uint32 role_hashvalue; /* Hash of role OID for cache invalidation */ - bool invalidated; -} ConnectionCacheEntry; - -static void -connection_cache_entry_free(void *gen_entry) -{ - ConnectionCacheEntry *entry = gen_entry; - - if (entry->conn != NULL) - { - /* Cannot directly read the Log_connections boolean since it is not - * exported on Windows */ - const char *log_conns = GetConfigOption("log_connections", true, false); - - if (NULL != log_conns && strcmp(log_conns, "on") == 0) - { - /* Log the connection closing if requested. Only log the - * associated user ID since cannot lookup the user name here due - * to, potentially, not being in a proper transaction state where - * it is possible to use the syscache. */ - elog(LOG, - "closing cached connection to \"%s\" [UserId: %d]", - remote_connection_node_name(entry->conn), - entry->id.user_id); - } - - remote_connection_close(entry->conn); - entry->conn = NULL; - } -} - -static void -connection_cache_pre_destroy_hook(Cache *cache) -{ - HASH_SEQ_STATUS scan; - ConnectionCacheEntry *entry; - - hash_seq_init(&scan, cache->htab); - while ((entry = hash_seq_search(&scan)) != NULL) - { - /* - * If we don't do this we will have a memory leak because connections - * are allocated using malloc - */ - connection_cache_entry_free(entry); - } -} - -static bool -connection_cache_valid_result(const void *result) -{ - if (result == NULL) - return false; - return ((ConnectionCacheEntry *) result)->conn != NULL; -} - -static void * -connection_cache_get_key(CacheQuery *query) -{ - return (TSConnectionId *) query->data; -} - -void -remote_connection_cache_invalidation_ignore(bool value) -{ - ignore_connection_invalidation = value; -} - -/* - * Check if a connection needs to be remade. - * - * A connection can be in a bad state, in which case we need to - * fail. Otherwise, the connection could be invalidated and needs to be remade - * to apply new options. But a connection can only be remade if we are not - * currently processing a transaction on the connection. - */ -static bool -connection_should_be_remade(const ConnectionCacheEntry *entry) -{ - bool invalidated; - - if (NULL == entry->conn) - return true; - - if (remote_connection_xact_is_transitioning(entry->conn)) - { - NameData nodename; - - namestrcpy(&nodename, remote_connection_node_name(entry->conn)); - - /* The connection is marked as being in the middle of transaction - * state change, which means the transaction was aborted in the middle - * of a transition and now we don't know the state of the remote - * endpoint. It is not safe to continue on such a connection, so we - * remove (and close) the connection and raise an error. */ - remote_connection_cache_remove(entry->id); - ereport(ERROR, - (errcode(ERRCODE_CONNECTION_EXCEPTION), - errmsg("connection to data node \"%s\" was lost", NameStr(nodename)))); - } - - /* Check if the connection has to be refreshed. If processing is set, then - * an async call was aborted. The connection could also have been - * invalidated, but we only care if we aren't still processing a - * transaction. */ - - invalidated = !ignore_connection_invalidation && entry->invalidated && - remote_connection_xact_depth_get(entry->conn) == 0; - if (remote_connection_get_status(entry->conn) == CONN_PROCESSING || invalidated) - return true; - - return false; -} - -static void * -connection_cache_create_entry(Cache *cache, CacheQuery *query) -{ - TSConnectionId *id = (TSConnectionId *) query->data; - ConnectionCacheEntry *entry = query->result; - MemoryContext old; - - /* - * Protects against errors in remote_connection_open, necessary since this - * entry is already in hashtable. - */ - entry->conn = NULL; - - /* - * Note: we do not use the cache memory context here to allocate a PGconn - * because PGconn allocation happens using malloc. Which is why calling - * remote_connection_close at cleanup is critical. - */ - old = MemoryContextSwitchTo(ts_cache_memory_ctx(cache)); - entry->conn = remote_connection_open_session_by_id(*id); - MemoryContextSwitchTo(old); - - /* Set the hash values of the foreign server and role for cache - * invalidation purposes */ - entry->foreign_server_hashvalue = - GetSysCacheHashValue1(FOREIGNSERVEROID, ObjectIdGetDatum(id->server_id)); - entry->role_hashvalue = GetSysCacheHashValue1(AUTHOID, ObjectIdGetDatum(id->user_id)); - entry->invalidated = false; - - return entry; -} - -/* - * This is called when the connection cache entry is found in the cache and - * before it is returned. The connection can either be bad, in which case it - * needs to be recreated, or the settings for the local session might have - * changed since it was last used. In the latter case, we need to configure - * the remote session again to ensure that it has the same configuration as - * the local session. - */ -static void * -connection_cache_update_entry(Cache *cache, CacheQuery *query) -{ - ConnectionCacheEntry *entry = query->result; - TSConnectionStatus status; - - if (connection_should_be_remade(entry)) - { - remote_connection_close(entry->conn); - return connection_cache_create_entry(cache, query); - } - - status = remote_connection_get_status(entry->conn); - Assert(status == CONN_IDLE || status == CONN_COPY_IN); - - if (status == CONN_IDLE) - remote_connection_configure_if_changed(entry->conn); - - return entry; -} - -static Cache * -connection_cache_create(void) -{ - MemoryContext ctx = - AllocSetContextCreate(CacheMemoryContext, "Connection cache", ALLOCSET_DEFAULT_SIZES); - Cache *cache = MemoryContextAlloc(ctx, sizeof(Cache)); - - *cache = (Cache) - { - .hctl = { - .keysize = sizeof(TSConnectionId), - .entrysize = sizeof(ConnectionCacheEntry), - .hcxt = ctx, - }, - .name = "connection_cache", - .numelements = 16, - .flags = HASH_ELEM | HASH_CONTEXT | HASH_BLOBS, - .valid_result = connection_cache_valid_result, - .get_key = connection_cache_get_key, - .create_entry = connection_cache_create_entry, - .remove_entry = connection_cache_entry_free, - .update_entry = connection_cache_update_entry, - .pre_destroy_hook = connection_cache_pre_destroy_hook, - }; - - ts_cache_init(cache); - cache->handle_txn_callbacks = false; - - return cache; -} - -TSConnection * -remote_connection_cache_get_connection(TSConnectionId id) -{ - CacheQuery query = { .data = &id }; - ConnectionCacheEntry *entry = ts_cache_fetch(connection_cache, &query); - - return entry->conn; -} - -bool -remote_connection_cache_remove(TSConnectionId id) -{ - return ts_cache_remove(connection_cache, &id); -} - -/* - * Connection invalidation callback function - * - * After a change to a pg_foreign_server catalog entry, - * mark the cache entry as invalid. - */ -void -remote_connection_cache_invalidate_callback(Datum arg, int cacheid, uint32 hashvalue) -{ - HASH_SEQ_STATUS scan; - ConnectionCacheEntry *entry; - - Assert(cacheid == FOREIGNSERVEROID || cacheid == AUTHOID); - hash_seq_init(&scan, connection_cache->htab); - - while ((entry = hash_seq_search(&scan)) != NULL) - { - /* hashvalue == 0 means cache reset, so invalidate entire cache */ - if (hashvalue == 0 || (cacheid == AUTHOID && hashvalue == entry->role_hashvalue) || - (cacheid == FOREIGNSERVEROID && entry->foreign_server_hashvalue == hashvalue)) - entry->invalidated = true; - } -} - -static bool -is_loopback_host_or_addr(const char *hostaddr) -{ - /* Use strncmp with length to succesfully compare against host address - * strings like "127.0.0.1/32" */ - return strcmp("localhost", hostaddr) == 0 || strncmp("127.0.0.1", hostaddr, 9) == 0 || - strncmp("::1", hostaddr, 3) == 0; -} - -/* - * Check if a connection is local. - * - * This is an imperfect check, but should work for common cases. - * - * It currently doesn't capture being connected on a local network interface - * address. That would require a platform-independent way to lookup local - * interface addresses (on UNIX one could use getifaddrs, for instance). - */ -static bool -is_local_connection(const PGconn *conn) -{ - const char *host = PQhost(conn); - int32 port; - - if (host[0] == '/' /* unix domain socket starts with a slash */) - return true; - - /* A TCP connection must match both the port and localhost address */ - port = pg_strtoint32(PQport(conn)); - - return (port == PostPortNumber) && is_loopback_host_or_addr(host); -} - -/* - * Remove connections that connect to a local DB, which is being dropped. - * - * This function is called when a database is dropped on the local instance - * and is needed to prevent errors when the database serves as a - * "same-instance" data node (common in tests). When a data node exists on the - * local instance (i.e., in a separate local database), we need to close all - * connections to the data node database in order to drop it. Otherwise, the - * drop database will fail since the database is being used by the connections - * in the cache. - */ -void -remote_connection_cache_dropped_db_callback(const char *dbname) -{ - HASH_SEQ_STATUS scan; - ConnectionCacheEntry *entry; - - hash_seq_init(&scan, connection_cache->htab); - - while ((entry = hash_seq_search(&scan)) != NULL) - { - PGconn *pgconn = remote_connection_get_pg_conn(entry->conn); - - /* Remove the connection if it is local and connects to a DB with the - * same name as the one being dropped. */ - if (strcmp(dbname, PQdb(pgconn)) == 0 && is_local_connection(pgconn)) - remote_connection_cache_remove(entry->id); - } -} - -/* - * Remove and close connections that belong to roles that are dropped. - * - * Immediately purging such connections should be safe since the DROP command - * must be executed by different user than the one being dropped. - */ -void -remote_connection_cache_dropped_role_callback(const char *rolename) -{ - HASH_SEQ_STATUS scan; - ConnectionCacheEntry *entry; - Oid roleid = get_role_oid(rolename, true); - - if (!OidIsValid(roleid)) - return; - - hash_seq_init(&scan, connection_cache->htab); - - while ((entry = hash_seq_search(&scan)) != NULL) - { - if (entry->id.user_id == roleid) - remote_connection_cache_remove(entry->id); - } -} -/* - * Functions and data structures for printing the connection cache. - */ -enum Anum_show_conn -{ - Anum_show_conn_node_name = 1, - Anum_show_conn_user_name, - Anum_show_conn_host, - Anum_show_conn_port, - Anum_show_conn_db, - Anum_show_conn_backend_pid, - Anum_show_conn_status, - Anum_show_conn_txn_status, - Anum_show_conn_txn_depth, - Anum_show_conn_processing, - Anum_show_conn_invalidated, - _Anum_show_conn_max, -}; - -#define Natts_show_conn (_Anum_show_conn_max - 1) - -static const char *conn_status_str[] = { - [CONNECTION_OK] = "OK", - [CONNECTION_BAD] = "BAD", - [CONNECTION_STARTED] = "STARTED", - [CONNECTION_MADE] = "MADE", - [CONNECTION_AWAITING_RESPONSE] = "AWAITING RESPONSE", - [CONNECTION_AUTH_OK] = "AUTH_OK", - [CONNECTION_SETENV] = "SETENV", - [CONNECTION_SSL_STARTUP] = "SSL STARTUP", - [CONNECTION_NEEDED] = "CONNECTION NEEDED", - [CONNECTION_CHECK_WRITABLE] = "CHECK WRITABLE", - [CONNECTION_CONSUME] = "CONSUME", - [CONNECTION_GSS_STARTUP] = "GSS STARTUP", - [CONNECTION_CHECK_TARGET] = "CHECK TARGET", -}; - -static const char *conn_txn_status_str[] = { - [PQTRANS_IDLE] = "IDLE", [PQTRANS_ACTIVE] = "ACTIVE", [PQTRANS_INTRANS] = "INTRANS", - [PQTRANS_INERROR] = "INERROR", [PQTRANS_UNKNOWN] = "UNKNOWN", -}; - -static HeapTuple -create_tuple_from_conn_entry(const ConnectionCacheEntry *entry, const TupleDesc tupdesc) -{ - Datum values[Natts_show_conn]; - bool nulls[Natts_show_conn] = { false }; - PGconn *pgconn = remote_connection_get_pg_conn(entry->conn); - NameData conn_node_name, conn_user_name, conn_db; - const char *username = GetUserNameFromId(entry->id.user_id, true); - - namestrcpy(&conn_node_name, remote_connection_node_name(entry->conn)); - - if (NULL == username) - pg_snprintf(NameStr(conn_user_name), NAMEDATALEN, "%u", entry->id.user_id); - else - namestrcpy(&conn_user_name, username); - - namestrcpy(&conn_db, PQdb(pgconn)); - - values[AttrNumberGetAttrOffset(Anum_show_conn_node_name)] = NameGetDatum(&conn_node_name); - values[AttrNumberGetAttrOffset(Anum_show_conn_user_name)] = NameGetDatum(&conn_user_name); - values[AttrNumberGetAttrOffset(Anum_show_conn_host)] = CStringGetTextDatum(PQhost(pgconn)); - values[AttrNumberGetAttrOffset(Anum_show_conn_port)] = - Int32GetDatum(pg_strtoint32(PQport(pgconn))); - values[AttrNumberGetAttrOffset(Anum_show_conn_db)] = NameGetDatum(&conn_db); - values[AttrNumberGetAttrOffset(Anum_show_conn_backend_pid)] = - Int32GetDatum(PQbackendPID(pgconn)); - values[AttrNumberGetAttrOffset(Anum_show_conn_status)] = - CStringGetTextDatum(conn_status_str[PQstatus(pgconn)]); - values[AttrNumberGetAttrOffset(Anum_show_conn_txn_status)] = - CStringGetTextDatum(conn_txn_status_str[PQtransactionStatus(pgconn)]); - values[AttrNumberGetAttrOffset(Anum_show_conn_txn_depth)] = - Int32GetDatum(remote_connection_xact_depth_get(entry->conn)); - values[AttrNumberGetAttrOffset(Anum_show_conn_processing)] = - BoolGetDatum(remote_connection_is_processing(entry->conn)); - values[AttrNumberGetAttrOffset(Anum_show_conn_invalidated)] = BoolGetDatum(entry->invalidated); - - return heap_form_tuple(tupdesc, values, nulls); -} - -typedef struct ConnCacheShowState -{ - HASH_SEQ_STATUS scan; - Cache *cache; -} ConnCacheShowState; - -Datum -remote_connection_cache_show(PG_FUNCTION_ARGS) -{ - FuncCallContext *funcctx; - ConnCacheShowState *info; - const ConnectionCacheEntry *entry; - HeapTuple tuple; - - if (SRF_IS_FIRSTCALL()) - { - MemoryContext oldcontext; - TupleDesc tupdesc; - - funcctx = SRF_FIRSTCALL_INIT(); - oldcontext = MemoryContextSwitchTo(funcctx->multi_call_memory_ctx); - - if (get_call_result_type(fcinfo, NULL, &tupdesc) != TYPEFUNC_COMPOSITE) - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("function returning record called in context " - "that cannot accept type record"))); - - info = palloc0(sizeof(ConnCacheShowState)); - info->cache = ts_cache_pin(connection_cache); - hash_seq_init(&info->scan, info->cache->htab); - funcctx->user_fctx = info; - funcctx->tuple_desc = BlessTupleDesc(tupdesc); - MemoryContextSwitchTo(oldcontext); - } - - funcctx = SRF_PERCALL_SETUP(); - info = funcctx->user_fctx; - - entry = hash_seq_search(&info->scan); - - if (entry == NULL) - { - ts_cache_release(info->cache); - SRF_RETURN_DONE(funcctx); - } - - tuple = create_tuple_from_conn_entry(entry, funcctx->tuple_desc); - - SRF_RETURN_NEXT(funcctx, HeapTupleGetDatum(tuple)); -} - -static void -connection_cache_xact_callback(XactEvent event, void *arg) -{ - /* Reset ignore_connection_invalidation to default value */ - remote_connection_cache_invalidation_ignore(false); -} - -void -_remote_connection_cache_init(void) -{ - connection_cache = connection_cache_create(); - RegisterXactCallback(connection_cache_xact_callback, NULL); -} - -void -_remote_connection_cache_fini(void) -{ - ts_cache_invalidate(connection_cache); - connection_cache = NULL; -} diff --git a/tsl/src/remote/connection_cache.h b/tsl/src/remote/connection_cache.h deleted file mode 100644 index ad08f9ab073..00000000000 --- a/tsl/src/remote/connection_cache.h +++ /dev/null @@ -1,30 +0,0 @@ -/* - * 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. - */ -#pragma once - -#include - -#include "connection.h" -#include "cache.h" - -/* - * This is a cross-transaction connection cache that allows you to reuse - * connections across transactions on a per-data node basis. - * - * Note that automatic cache unpinning on txn callbacks have been disabled on this cache. - * That means unpinning on aborts has to be done by the caller. This is because this cache - * is used by xact callbacks. - */ - -extern TSConnection *remote_connection_cache_get_connection(TSConnectionId id); -extern bool remote_connection_cache_remove(TSConnectionId id); -extern void remote_connection_cache_invalidation_ignore(bool value); -extern void remote_connection_cache_invalidate_callback(Datum arg, int cacheid, uint32 hashvalue); -extern void remote_connection_cache_dropped_db_callback(const char *dbname); -extern void remote_connection_cache_dropped_role_callback(const char *rolename); -extern Datum remote_connection_cache_show(PG_FUNCTION_ARGS); -extern void _remote_connection_cache_init(void); -extern void _remote_connection_cache_fini(void); diff --git a/tsl/src/remote/copy_fetcher.c b/tsl/src/remote/copy_fetcher.c deleted file mode 100644 index 04a303df45e..00000000000 --- a/tsl/src/remote/copy_fetcher.c +++ /dev/null @@ -1,692 +0,0 @@ -/* - * 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. - */ - -#include -#include -#include - -#include "copy_fetcher.h" -#include "tuplefactory.h" -#include "async.h" - -typedef struct CopyFetcher -{ - DataFetcher state; - - /* Data for virtual tuples of the current retrieved batch. */ - Datum *batch_values; - bool *batch_nulls; - AsyncRequest *req; -#ifdef USE_ASSERT_CHECKING - bool file_trailer_received; -#endif -} CopyFetcher; - -static void copy_fetcher_send_fetch_request(DataFetcher *df); -static void copy_fetcher_reset(CopyFetcher *fetcher); -static int copy_fetcher_fetch_data(DataFetcher *df); -static void copy_fetcher_set_fetch_size(DataFetcher *df, int fetch_size); -static void copy_fetcher_set_tuple_memcontext(DataFetcher *df, MemoryContext mctx); -static void copy_fetcher_store_next_tuple(DataFetcher *df, TupleTableSlot *slot); -static void copy_fetcher_rewind(DataFetcher *df); -static void copy_fetcher_close(DataFetcher *df); - -static DataFetcherFuncs funcs = { - .close = copy_fetcher_close, - .fetch_data = copy_fetcher_fetch_data, - .rescan = data_fetcher_rescan, - .rewind = copy_fetcher_rewind, - .send_fetch_request = copy_fetcher_send_fetch_request, - .set_fetch_size = copy_fetcher_set_fetch_size, - .set_tuple_mctx = copy_fetcher_set_tuple_memcontext, - .store_next_tuple = copy_fetcher_store_next_tuple, -}; - -static void -copy_fetcher_set_fetch_size(DataFetcher *df, int fetch_size) -{ - CopyFetcher *fetcher = cast_fetcher(CopyFetcher, df); - - data_fetcher_set_fetch_size(&fetcher->state, fetch_size); -} - -static void -copy_fetcher_set_tuple_memcontext(DataFetcher *df, MemoryContext mctx) -{ - CopyFetcher *fetcher = cast_fetcher(CopyFetcher, df); - data_fetcher_set_tuple_mctx(&fetcher->state, mctx); -} - -static void -copy_fetcher_reset(CopyFetcher *fetcher) -{ - fetcher->state.open = false; - -#ifdef USE_ASSERT_CHECKING - fetcher->file_trailer_received = false; -#endif - - if (fetcher->req != NULL) - { - pfree(fetcher->req); - fetcher->req = NULL; - } - - data_fetcher_reset(&fetcher->state); -} - -static void -copy_fetcher_send_fetch_request(DataFetcher *df) -{ - AsyncRequest *volatile req = NULL; - MemoryContext oldcontext; - CopyFetcher *fetcher = cast_fetcher(CopyFetcher, df); - - if (fetcher->state.open) - { - /* data request has already been sent */ - return; - } - - /* make sure to have a clean state */ - copy_fetcher_reset(fetcher); - - StringInfoData copy_query; - initStringInfo(©_query); - appendStringInfo(©_query, "copy (%s) to stdout with (format binary)", fetcher->state.stmt); - - PG_TRY(); - { - oldcontext = MemoryContextSwitchTo(fetcher->state.req_mctx); - - Assert(tuplefactory_is_binary(fetcher->state.tf)); - req = async_request_send_with_stmt_params_elevel_res_format(fetcher->state.conn, - copy_query.data, - fetcher->state.stmt_params, - ERROR, - FORMAT_BINARY); - Assert(NULL != req); - - /* - * Single-row mode doesn't really influence the COPY queries, but setting - * it here is a convenient way to prevent concurrent COPY requests on the - * same connection. This can happen if we have multiple tables on the same - * data node and still use the COPY fetcher. - */ - if (!async_request_set_single_row_mode(req)) - { - ereport(ERROR, - (errcode(ERRCODE_CONNECTION_FAILURE), - errmsg("could not set single-row mode on connection to \"%s\"", - remote_connection_node_name(fetcher->state.conn)), - errdetail("The aborted statement is: %s.", fetcher->state.stmt), - errhint("COPY fetcher is not supported together with sub-queries." - " Use cursor fetcher instead."))); - } - - fetcher->state.open = true; - fetcher->req = req; - } - PG_CATCH(); - { - if (NULL != req) - pfree(req); - - PG_RE_THROW(); - } - PG_END_TRY(); - - MemoryContextSwitchTo(oldcontext); -} - -static int -copy_data_consume_bytes(StringInfo copy_data, int bytes_to_read) -{ - const int bytes_read = Min(bytes_to_read, copy_data->len - copy_data->cursor); - copy_data->cursor += bytes_read; - Assert(copy_data->cursor <= copy_data->len); - return bytes_read; -} - -static char * -copy_data_read_bytes(StringInfo copy_data, int bytes_to_read) -{ - char *result = ©_data->data[copy_data->cursor]; - const int bytes_read = copy_data_consume_bytes(copy_data, bytes_to_read); - - if (bytes_read != bytes_to_read) - { - elog(ERROR, - "could not read the requested %d bytes of COPY data, read %d instead", - bytes_to_read, - bytes_read); - } - - return result; -} - -static int16 -copy_data_read_int16(StringInfo copy_data) -{ - char *buf = ©_data->data[copy_data->cursor]; - char aligned_buf[2]; - if (copy_data_consume_bytes(copy_data, 2) != 2) - { - elog(ERROR, "failed to read int16 from COPY data: not enough bytes left"); - } - if (buf != (const char *) TYPEALIGN(2, buf)) - { - memcpy(aligned_buf, buf, 2); - buf = aligned_buf; - } - AssertPointerAlignment(buf, 2); - return (int16) pg_ntoh16(*(uint16 *) buf); -} - -static int32 -copy_data_read_int32(StringInfo copy_data) -{ - char *buf = ©_data->data[copy_data->cursor]; - char aligned_buf[4]; - if (copy_data_consume_bytes(copy_data, 4) != 4) - { - elog(ERROR, "failed to read int32 from COPY data: not enough bytes left"); - } - if (buf != (const char *) TYPEALIGN(4, buf)) - { - memcpy(aligned_buf, buf, 4); - buf = aligned_buf; - } - AssertPointerAlignment(buf, 4); - return (int32) pg_ntoh32(*(uint32 *) buf); -} - -static void -copy_data_check_header(StringInfo copy_data) -{ - static const char required_signature[11] = "PGCOPY\n\377\r\n\0"; - char *actual_signature = copy_data_read_bytes(copy_data, sizeof(required_signature)); - if (memcmp(required_signature, actual_signature, sizeof(required_signature)) != 0) - { - elog(ERROR, "wrong COPY data signature"); - } - - int32 flags = copy_data_read_int32(copy_data); - if (flags != 0) - { - elog(ERROR, "wrong COPY flags: %d, should be 0", flags); - } - - /* - * Header extension area length - * 32-bit integer, length in bytes of remainder of header, not including - * self. Currently, this is zero, and the first tuple follows - * immediately. Future changes to the format might allow additional data - * to be present in the header. A reader should silently skip over any - * header extension data it does not know what to do with. - */ - int32 header_extension_length = copy_data_read_int32(copy_data); - int bytes_read = copy_data_consume_bytes(copy_data, header_extension_length); - if (bytes_read != header_extension_length) - { - elog(ERROR, - "failed to read COPY header extension: expected %d bytes, read %d", - header_extension_length, - bytes_read); - } -} - -/* - * End the COPY after receiving EOF or canceling a query (e.g., due to a LIMIT - * being reached). - * - * This should be called after canceling a query, or, after reading all data, - * the file trailer, and getting an EOF return value. - */ -static void -end_copy(CopyFetcher *fetcher, bool canceled) -{ - PGconn *conn = remote_connection_get_pg_conn(fetcher->state.conn); - PGresult *final_pgres = NULL; - PGresult *pgres = NULL; - ExecStatusType received_status; - - Assert(fetcher->state.open); - - /* Read results until NULL */ - while ((pgres = PQgetResult(conn))) - { - if (final_pgres == NULL) - final_pgres = pgres; - else - remote_result_close(pgres); - } - - received_status = PQresultStatus(final_pgres); - remote_result_close(final_pgres); - - if (canceled) - { - /* If the query was canceled during query execution by the access node - * (e.g., due to reaching a LIMIT), expect either PGRES_COMMAND_OK - * (query completed before cancel happened) or PGRES_FATAL_ERROR - * (query abandoned before completion) */ - if (received_status != PGRES_COMMAND_OK && received_status != PGRES_FATAL_ERROR) - remote_connection_elog(fetcher->state.conn, ERROR); - } - else if (received_status != PGRES_COMMAND_OK) - { - Assert(received_status == PGRES_FATAL_ERROR || received_status == PGRES_NONFATAL_ERROR); - remote_connection_elog(fetcher->state.conn, ERROR); - } - - fetcher->state.open = false; - remote_connection_set_status(fetcher->state.conn, CONN_IDLE); -} - -/* - * Prematurely end the COPY before EOF is received. - * - * This happens for queries that are abandoned before reaching EOF, e.g., when - * using LIMITs. - */ -static void -end_copy_before_eof(CopyFetcher *fetcher) -{ - /* - * The fetcher state might not be open if the fetcher got initialized but - * never executed due to executor constraints. - */ - if (!fetcher->state.open) - return; - - Assert(!fetcher->state.eof); - remote_connection_cancel_query(fetcher->state.conn); - end_copy(fetcher, true); -} - -static void -copy_fetcher_read_fetch_response(CopyFetcher *fetcher) -{ - PGconn *conn = remote_connection_get_pg_conn(fetcher->state.conn); - PGresult *res; - - if (fetcher->req == NULL) - return; - - res = PQgetResult(conn); - pfree(fetcher->req); - fetcher->req = NULL; - - if (res == NULL) - { - /* Shouldn't really happen but technically possible. */ - TSConnectionError err; - remote_connection_get_error(fetcher->state.conn, &err); - remote_connection_error_elog(&err, ERROR); - } - if (PQresultStatus(res) != PGRES_COPY_OUT) - { - TSConnectionError err; - remote_connection_get_result_error(res, &err); - PQclear(res); - remote_connection_error_elog(&err, ERROR); - } - - PQclear(res); -} - -/* - * Read the next data from the connection and store the data in copy_data. - * If no data can be read return false, or throw an error, otherwise - * return true. - */ -static bool -copy_fetcher_read_data(CopyFetcher *fetcher, PGconn *conn, char *volatile *dataptr, - StringInfoData *copy_data) -{ - copy_data->len = PQgetCopyData(conn, - ©_data->data, - /* async = */ false); - - /* Set dataptr to ensure data is freed with PQfreemem() in - * PG_CATCH() clause in case error is thrown. */ - *dataptr = copy_data->data; - - if (copy_data->len == -1) - { - /* Note: it is possible to get EOF without having received the - * file trailer in case there's e.g., a remote error. */ - fetcher->state.eof = true; - - /* Should read final result with PQgetResult() until it - * returns NULL. This happens later in end_copy. */ - return false; - } - else if (copy_data->len == -2) - { - /* - * Error. The docs say: consult PQerrorMessage() for the reason. - * remote_connection_elog() will do this for us. - */ - remote_connection_elog(fetcher->state.conn, ERROR); - - /* remote_connection_elog should raise an ERROR */ - pg_unreachable(); - } - - copy_data->maxlen = copy_data->len; - - return true; -} - -/* - * Process response for ongoing async request - */ -static int -copy_fetcher_complete(CopyFetcher *fetcher) -{ - /* Marked as volatile since it's modified in PG_TRY used in PG_CATCH */ - AsyncResponseResult *volatile response = NULL; - char *volatile dataptr = NULL; - MemoryContext oldcontext; - PGconn *conn = remote_connection_get_pg_conn(fetcher->state.conn); - - Assert(fetcher->state.open); - data_fetcher_validate(&fetcher->state); - - if (fetcher->req != NULL) - copy_fetcher_read_fetch_response(fetcher); - - /* - * We'll store the tuples in the batch_mctx. First, flush the previous - * batch. - */ - MemoryContextReset(fetcher->state.batch_mctx); - oldcontext = MemoryContextSwitchTo(fetcher->state.batch_mctx); - const TupleDesc tupdesc = tuplefactory_get_tupdesc(fetcher->state.tf); - const List *retrieved_attrs = tuplefactory_get_retrieved_attrs(fetcher->state.tf); - const int tupdesc_natts = tupdesc->natts; - const int retrieved_natts = list_length(retrieved_attrs); - const int total = tupdesc_natts * fetcher->state.fetch_size; - fetcher->batch_nulls = palloc(sizeof(bool) * total); - for (int i = 0; i < total; i++) - { - fetcher->batch_nulls[i] = true; - } - fetcher->batch_values = palloc0(sizeof(Datum) * total); - - PG_TRY(); - { - int row; - - for (row = 0; row < fetcher->state.fetch_size; row++) - { - MemoryContextSwitchTo(fetcher->state.req_mctx); - - StringInfoData copy_data = { 0 }; - bool tuple_read = copy_fetcher_read_data(fetcher, conn, &dataptr, ©_data); - - /* Were we able to fetch new data? */ - if (!tuple_read) - break; - - Assert(copy_data.cursor == 0); - - if (fetcher->state.batch_count == 0 && row == 0) - { - copy_data_check_header(©_data); - } - - const AttConvInMetadata *attconv = tuplefactory_get_attconv(fetcher->state.tf); - Assert(attconv->binary); - const int16 natts = copy_data_read_int16(©_data); - if (natts == -1) - { - Assert(!fetcher->file_trailer_received && !fetcher->state.eof); - /* - * From the PostgreSQL (libpq) docs: The file trailer consists - * of a 16-bit integer word containing -1. This is easily - * distinguished from a tuple's field-count word. A reader - * should report an error if a field-count word is neither -1 - * nor the expected number of columns. This provides an extra - * check against somehow getting out of sync with the data. - */ -#ifdef USE_ASSERT_CHECKING - fetcher->file_trailer_received = true; -#endif - - /* Next PQgetCopyData() should return -1, indicating EOF and - * that the remote side ended the copy. The final result - * (PGRES_COMMAND_OK) should then be read with - * PQgetResult(). - * - * Execute PQgetCopyData() (invoked in copy_fetcher_read_data) - * directly here, because if row = state.fetch_size - 1 - * (i.e., file_trailer is the last tuple of the batch), the - * for loop will not be executed again and PQgetCopyData() - * will never be called. If it is not called, the EOF state - * is not updated and a new batch would be requested. - */ - tuple_read = copy_fetcher_read_data(fetcher, conn, &dataptr, ©_data); - Assert(tuple_read == false); - break; - } - else - { - Assert(!fetcher->file_trailer_received && !fetcher->state.eof); - /* - * There is also one case where no tupdesc attributes are retrieved. - * This is when we do `select count(*) from t`, and - * `enable_partitionwise_aggregate` is 0, so the data node queries - * become `select null from ...` and we should get 1 NULL attribute - * from COPY. - */ - int16 expected_natts = Max(1, retrieved_natts); - if (natts != expected_natts) - { - elog(ERROR, - "wrong number of attributes for a COPY tuple: expected %d, got %d", - expected_natts, - natts); - } - - Datum *values = &fetcher->batch_values[tupdesc_natts * row]; - bool *nulls = &fetcher->batch_nulls[tupdesc_natts * row]; - for (int i = 0; i < tupdesc_natts; i++) - { - nulls[i] = true; - } - - MemoryContextSwitchTo(fetcher->state.tuple_mctx); - for (int i = 0; i < retrieved_natts; i++) - { - const int att = list_nth_int(retrieved_attrs, i) - 1; - Assert(att >= 0); - Assert(att < tupdesc_natts); - const int32 att_bytes = copy_data_read_int32(©_data); - if (att_bytes == -1) - { - /* - * NULL. From the Postgres docs: - * Usually, a receive function should be declared STRICT; if - * it is not, it will be called with a NULL first parameter - * when reading a NULL input value. The function must still - * return NULL in this case, unless it raises an error. - * (This case is mainly meant to support domain receive - * functions, which might need to reject NULL inputs.) - * https://www.postgresql.org/docs/current/sql-createtype.html - */ - if (!attconv->conv_funcs[att].fn_strict) - { - values[att] = ReceiveFunctionCall(&attconv->conv_funcs[att], - NULL, - attconv->ioparams[att], - attconv->typmods[att]); - } - else - { - values[att] = PointerGetDatum(NULL); - } - nulls[att] = true; - continue; - } - - StringInfoData att_data = { 0 }; - att_data.data = copy_data_read_bytes(©_data, att_bytes); - att_data.len = att_bytes; - - values[att] = ReceiveFunctionCall(&attconv->conv_funcs[att], - &att_data, - attconv->ioparams[att], - attconv->typmods[att]); - nulls[att] = false; - } - - /* - * We expect one row per message here, check that no data is - * left. - */ - Assert(copy_data.cursor = copy_data.len); - } - MemoryContextSwitchTo(fetcher->state.batch_mctx); - PQfreemem(copy_data.data); - dataptr = NULL; - } - - fetcher->state.num_tuples = row; - fetcher->state.next_tuple_idx = 0; - - /* Must be EOF if we didn't get as many tuples as we asked for. */ -#ifdef USE_ASSERT_CHECKING - if (fetcher->state.num_tuples < fetcher->state.fetch_size) - Assert(fetcher->state.eof); -#endif - - fetcher->state.batch_count++; - - /* Finish the COPY here instead of at scan end (fetcher close) in - * order to not leave the connection in COPY_OUT mode. This is - * necessary to handle, e.g., remote EXPLAINs (together with ANALYZE) - * where tuples are first fetched in COPY mode, then a remote explain - * is performed on the same connection within the same scan. */ - if (fetcher->state.eof) - end_copy(fetcher, false); - } - PG_CATCH(); - { - if (NULL != response) - async_response_result_close(response); - - if (NULL != dataptr) - PQfreemem(dataptr); - - PG_RE_THROW(); - } - PG_END_TRY(); - - MemoryContextSwitchTo(oldcontext); - - return fetcher->state.num_tuples; -} - -static int -copy_fetcher_fetch_data(DataFetcher *df) -{ - CopyFetcher *fetcher = cast_fetcher(CopyFetcher, df); - - if (fetcher->state.eof) - return 0; - - if (!fetcher->state.open) - copy_fetcher_send_fetch_request(df); - - return copy_fetcher_complete(fetcher); -} - -static void -copy_fetcher_store_tuple(DataFetcher *df, int row, TupleTableSlot *slot) -{ - CopyFetcher *fetcher = cast_fetcher(CopyFetcher, df); - - ExecClearTuple(slot); - - if (row >= df->num_tuples) - { - if (df->eof || df->funcs->fetch_data(df) == 0) - { - return; - } - - row = 0; - Assert(row == df->next_tuple_idx); - } - - Assert(fetcher->batch_values != NULL); - Assert(fetcher->batch_nulls != NULL); - Assert(row >= 0 && row < df->num_tuples); - - const int nattrs = tuplefactory_get_nattrs(fetcher->state.tf); - slot->tts_values = &fetcher->batch_values[nattrs * row]; - slot->tts_isnull = &fetcher->batch_nulls[nattrs * row]; - ExecStoreVirtualTuple(slot); -} - -static void -copy_fetcher_store_next_tuple(DataFetcher *df, TupleTableSlot *slot) -{ - copy_fetcher_store_tuple(df, df->next_tuple_idx, slot); - - if (!TupIsNull(slot)) - df->next_tuple_idx++; - - Assert(df->next_tuple_idx <= df->num_tuples); -} - -DataFetcher * -copy_fetcher_create_for_scan(TSConnection *conn, const char *stmt, StmtParams *params, - TupleFactory *tf) -{ - CopyFetcher *fetcher = palloc0(sizeof(CopyFetcher)); - - data_fetcher_init(&fetcher->state, conn, stmt, params, tf); - fetcher->state.type = CopyFetcherType; - fetcher->state.funcs = &funcs; - - return &fetcher->state; -} - -static void -copy_fetcher_close(DataFetcher *df) -{ - CopyFetcher *fetcher = cast_fetcher(CopyFetcher, df); - - Assert((df->eof && fetcher->file_trailer_received) || !df->eof); - - /* Check for premature ending of COPY (i.e., before reaching EOF). Note, - * if EOF was reached, we closed already in the main processing loop so no - * need to end_copy() here in that case. */ - if (!fetcher->state.eof) - end_copy_before_eof(fetcher); - - Assert(!fetcher->state.open); - copy_fetcher_reset(fetcher); -} - -static void -copy_fetcher_rewind(DataFetcher *df) -{ - CopyFetcher *fetcher = cast_fetcher(CopyFetcher, df); - - if (fetcher->state.batch_count > 1) - { - /* we're over the first batch so we need to close fetcher and restart from clean state */ - copy_fetcher_close(df); - } - else - /* we can reuse current batch of results */ - fetcher->state.next_tuple_idx = 0; -} diff --git a/tsl/src/remote/copy_fetcher.h b/tsl/src/remote/copy_fetcher.h deleted file mode 100644 index 16b49926504..00000000000 --- a/tsl/src/remote/copy_fetcher.h +++ /dev/null @@ -1,13 +0,0 @@ -/* - * 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. - */ -#pragma once - -#include - -#include "data_fetcher.h" - -extern DataFetcher *copy_fetcher_create_for_scan(TSConnection *conn, const char *stmt, - StmtParams *params, TupleFactory *tf); diff --git a/tsl/src/remote/cursor_fetcher.c b/tsl/src/remote/cursor_fetcher.c deleted file mode 100644 index 9c8ba731b56..00000000000 --- a/tsl/src/remote/cursor_fetcher.c +++ /dev/null @@ -1,415 +0,0 @@ -/* - * This file and its contents are licensed under the Timescale License. - * Please see the included NOTICE for copyright information and - * LICENSE-TIMESCALE for a copy of the license. - */ -/* - * This file contains source code that was copied and/or modified from - * the PostgreSQL database, which is licensed under the open-source - * PostgreSQL License. Please see the NOTICE at the top level - * directory for a copy of the PostgreSQL License. - */ -#include -#include -#include -#include - -#include "utils.h" -#include "async.h" -#include "stmt_params.h" -#include "tuplefactory.h" -#include "cursor_fetcher.h" -#include "data_fetcher.h" - -/* - * Cursor for fetching data from a data node. - * - * The cursor fetcher splits the query result into multiple fetches, which - * allows multiplexing on-going sub-queries on the same connection without - * having to fetch the full result for each sub-query in one go (thus not - * over-running memory). - * - * When a query consists of multiple subqueries that fetch data from the same - * data nodes, and the sub-queries are joined using, e.g., a nested loop, then - * a CURSOR is necessary to run the two sub-queries over the same connection. - * - * The downside of using a CURSOR, however, is that the plan on the remote - * node cannot execute in parallel. - * - * https://www.postgresql.org/docs/current/when-can-parallel-query-be-used.html - */ -typedef struct CursorFetcher -{ - DataFetcher state; - unsigned int id; - char fetch_stmt[64]; /* cursor fetch statement */ - AsyncRequest *create_req; /* a request to create cursor */ -} CursorFetcher; - -static void cursor_fetcher_send_fetch_request(DataFetcher *df); -static int cursor_fetcher_fetch_data(DataFetcher *df); -static void cursor_fetcher_set_fetch_size(DataFetcher *df, int fetch_size); -static void cursor_fetcher_set_tuple_memcontext(DataFetcher *df, MemoryContext mctx); -static void cursor_fetcher_store_next_tuple(DataFetcher *df, TupleTableSlot *slot); -static void cursor_fetcher_rewind(DataFetcher *df); -static void cursor_fetcher_close(DataFetcher *df); - -static DataFetcherFuncs funcs = { - .close = cursor_fetcher_close, - .fetch_data = cursor_fetcher_fetch_data, - .rescan = data_fetcher_rescan, - .rewind = cursor_fetcher_rewind, - .send_fetch_request = cursor_fetcher_send_fetch_request, - .set_fetch_size = cursor_fetcher_set_fetch_size, - .set_tuple_mctx = cursor_fetcher_set_tuple_memcontext, - .store_next_tuple = cursor_fetcher_store_next_tuple, -}; - -static void -cursor_create_req(CursorFetcher *cursor) -{ - AsyncRequest *volatile req = NULL; - StringInfoData buf; - MemoryContext oldcontext; - - initStringInfo(&buf); - appendStringInfo(&buf, "DECLARE c%u CURSOR FOR\n%s", cursor->id, cursor->state.stmt); - oldcontext = MemoryContextSwitchTo(cursor->state.req_mctx); - - PG_TRY(); - { - if (NULL == cursor->state.stmt_params) - req = async_request_send(cursor->state.conn, buf.data); - else - req = async_request_send_with_params(cursor->state.conn, - buf.data, - cursor->state.stmt_params, - FORMAT_TEXT); - - Assert(NULL != req); - - cursor->create_req = req; - pfree(buf.data); - } - PG_CATCH(); - { - if (NULL != req) - pfree(req); - - PG_RE_THROW(); - } - PG_END_TRY(); - MemoryContextSwitchTo(oldcontext); -} - -/* - * Complete ongoing cursor create request if needed and return cursor. - * If cursor is in async mode then we will dispatch a request to fetch data. - */ -static void -cursor_fetcher_wait_until_open(DataFetcher *df) -{ - CursorFetcher *cursor = cast_fetcher(CursorFetcher, df); - - if (cursor->state.open) - { - Assert(cursor->create_req == NULL); - /* nothing to do */ - return; - } - - if (cursor->create_req == NULL) - ereport(ERROR, - (errcode(ERRCODE_INVALID_CURSOR_STATE), - errmsg("invalid cursor state"), - errdetail("Cannot wait on unsent cursor request."))); - - async_request_wait_ok_command(cursor->create_req); - cursor->state.open = true; - pfree(cursor->create_req); - cursor->create_req = NULL; -} - -DataFetcher * -cursor_fetcher_create_for_scan(TSConnection *conn, const char *stmt, StmtParams *params, - TupleFactory *tf) -{ - CursorFetcher *fetcher = palloc0(sizeof(CursorFetcher)); - - data_fetcher_init(&fetcher->state, conn, stmt, params, tf); - - fetcher->state.type = CursorFetcherType; - /* Assign a unique ID for my cursor */ - fetcher->id = remote_connection_get_cursor_number(); - fetcher->create_req = NULL; - /* send a request to DECLARE cursor */ - cursor_create_req(fetcher); - fetcher->state.funcs = &funcs; - cursor_fetcher_wait_until_open(&fetcher->state); - - return &fetcher->state; -} - -static void -cursor_fetcher_set_fetch_size(DataFetcher *df, int fetch_size) -{ - CursorFetcher *cursor = cast_fetcher(CursorFetcher, df); - - data_fetcher_set_fetch_size(&cursor->state, fetch_size); - snprintf(cursor->fetch_stmt, - sizeof(cursor->fetch_stmt), - "FETCH %u FROM c%u", - fetch_size, - cursor->id); -} - -static void -cursor_fetcher_set_tuple_memcontext(DataFetcher *df, MemoryContext mctx) -{ - CursorFetcher *cursor = cast_fetcher(CursorFetcher, df); - - data_fetcher_set_tuple_mctx(&cursor->state, mctx); -} - -/* - * Send async req to fetch data from cursor. - */ -static void -cursor_fetcher_send_fetch_request(DataFetcher *df) -{ - AsyncRequest *volatile req = NULL; - MemoryContext oldcontext; - CursorFetcher *cursor = cast_fetcher(CursorFetcher, df); - - Assert(cursor->state.open); - - if (cursor->state.data_req != NULL) - ereport(ERROR, - (errcode(ERRCODE_INVALID_CURSOR_STATE), - errmsg("invalid cursor state"), - errdetail("Cannot fetch new data while previous request is ongoing."))); - - PG_TRY(); - { - TSConnection *conn = cursor->state.conn; - - /* We use a separate mem context because batch mem context is getting reset once we fetch - * new batch and here we need our async request to survive */ - oldcontext = MemoryContextSwitchTo(cursor->state.req_mctx); - - if (tuplefactory_is_binary(cursor->state.tf)) - req = async_request_send_binary(conn, cursor->fetch_stmt); - else - req = async_request_send(conn, cursor->fetch_stmt); - - Assert(NULL != req); - cursor->state.data_req = req; - } - PG_CATCH(); - { - if (NULL != req) - pfree(req); - - PG_RE_THROW(); - } - PG_END_TRY(); - - MemoryContextSwitchTo(oldcontext); -} - -/* - * Retrieve data from ongoing async fetch request - */ -static int -cursor_fetcher_fetch_data_complete(CursorFetcher *cursor) -{ - AsyncResponseResult *volatile response = NULL; - MemoryContext oldcontext; - int numrows = 0; - int format = 0; - - Assert(cursor != NULL); - Assert(cursor->state.data_req != NULL); - - Assert(cursor->state.open); - data_fetcher_validate(&cursor->state); - - /* - * We'll store the tuples in the batch_mctx. First, flush the previous - * batch. - */ - cursor->state.tuples = NULL; - MemoryContextReset(cursor->state.batch_mctx); - - PG_TRY(); - { - PGresult *res; - int i; - - oldcontext = MemoryContextSwitchTo(cursor->state.req_mctx); - - response = async_request_wait_any_result(cursor->state.data_req); - Assert(NULL != response); - - res = async_response_result_get_pg_result(response); - format = PQbinaryTuples(res); - - MemoryContextSwitchTo(cursor->state.batch_mctx); - - /* On error, report the original query, not the FETCH. */ - if (PQresultStatus(res) != PGRES_TUPLES_OK) - { - /* remote_result_elog will call PQclear() on the result, so need - * to mark the response as NULL to avoid double PQclear() */ - pfree(response); - response = NULL; - remote_result_elog(res, ERROR); - } - - /* Convert the data into HeapTuples */ - numrows = PQntuples(res); - cursor->state.tuples = (HeapTuple *) palloc0(numrows * sizeof(HeapTuple)); - cursor->state.num_tuples = numrows; - cursor->state.next_tuple_idx = 0; - - /* Allow creating tuples in alternative memory context if user has set - * it explicitly, otherwise same as batch_mctx */ - MemoryContextSwitchTo(cursor->state.tuple_mctx); - - for (i = 0; i < numrows; i++) - cursor->state.tuples[i] = tuplefactory_make_tuple(cursor->state.tf, res, i, format); - - tuplefactory_reset_mctx(cursor->state.tf); - MemoryContextSwitchTo(cursor->state.batch_mctx); - - /* Update batch count to indicate we are no longer in the first - * batch. When we are on the second batch or greater, a rewind of the - * cursor needs to refetch the first batch. If we are still in the - * first batch, however, a rewind can be done by simply resetting the - * tuple index to 0 within the batch. */ - if (cursor->state.batch_count < 2) - cursor->state.batch_count++; - - /* Must be EOF if we didn't get as many tuples as we asked for. */ - cursor->state.eof = (numrows < cursor->state.fetch_size); - - pfree(cursor->state.data_req); - cursor->state.data_req = NULL; - - async_response_result_close(response); - response = NULL; - } - PG_CATCH(); - { - if (NULL != cursor->state.data_req) - { - pfree(cursor->state.data_req); - cursor->state.data_req = NULL; - } - - if (NULL != response) - async_response_result_close(response); - - PG_RE_THROW(); - } - PG_END_TRY(); - - MemoryContextSwitchTo(oldcontext); - - return numrows; -} - -static int -cursor_fetcher_fetch_data(DataFetcher *df) -{ - CursorFetcher *cursor = cast_fetcher(CursorFetcher, df); - - if (cursor->state.eof) - return 0; - - if (!cursor->state.open) - { - if (cursor->create_req == NULL) - { - cursor_create_req(cursor); - } - - cursor_fetcher_wait_until_open(df); - } - - if (cursor->state.data_req == NULL) - cursor_fetcher_send_fetch_request(df); - - return cursor_fetcher_fetch_data_complete(cursor); -} - -static void -cursor_fetcher_store_next_tuple(DataFetcher *df, TupleTableSlot *slot) -{ - CursorFetcher *cursor = cast_fetcher(CursorFetcher, df); - - data_fetcher_store_next_tuple(&cursor->state, slot); -} - -static void -remote_cursor_exec_cmd(CursorFetcher *cursor, const char *sql) -{ - AsyncRequest *req; - - /* - * We don't use a PG_TRY block here, so be careful not to throw error - * without releasing the PGresult. - */ - req = async_request_send(cursor->state.conn, sql); - Assert(NULL != req); - async_request_wait_ok_command(req); - pfree(req); - - data_fetcher_reset(&cursor->state); -} - -static void -cursor_fetcher_rewind(DataFetcher *df) -{ - CursorFetcher *cursor = cast_fetcher(CursorFetcher, df); - /* We need to make sure that cursor is opened */ - cursor_fetcher_wait_until_open(df); - - if (cursor->state.batch_count > 1) - { - char sql[64]; - - if (!cursor->state.eof && cursor->state.data_req != NULL) - async_request_discard_response(cursor->state.data_req); - - /* We are beyond the first fetch, so need to rewind the remote end */ - snprintf(sql, sizeof(sql), "MOVE BACKWARD ALL IN c%u", cursor->id); - remote_cursor_exec_cmd(cursor, sql); - } - else - { - /* We have done zero or one fetch, so we can simply re-read what we - * have in memory, if anything */ - cursor->state.next_tuple_idx = 0; - } -} - -static void -cursor_fetcher_close(DataFetcher *df) -{ - CursorFetcher *cursor = cast_fetcher(CursorFetcher, df); - char sql[64]; - - if (!cursor->state.open && cursor->create_req != NULL) - { - async_request_discard_response(cursor->create_req); - return; - } - - if (!cursor->state.eof && cursor->state.data_req != NULL) - async_request_discard_response(cursor->state.data_req); - - snprintf(sql, sizeof(sql), "CLOSE c%u", cursor->id); - cursor->state.open = false; - remote_cursor_exec_cmd(cursor, sql); -} diff --git a/tsl/src/remote/cursor_fetcher.h b/tsl/src/remote/cursor_fetcher.h deleted file mode 100644 index cc2c1d950e8..00000000000 --- a/tsl/src/remote/cursor_fetcher.h +++ /dev/null @@ -1,13 +0,0 @@ -/* - * 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. - */ -#pragma once - -#include - -#include "data_fetcher.h" - -extern DataFetcher *cursor_fetcher_create_for_scan(TSConnection *conn, const char *stmt, - StmtParams *params, TupleFactory *tf); diff --git a/tsl/src/remote/data_fetcher.c b/tsl/src/remote/data_fetcher.c deleted file mode 100644 index 84cb2a3cc61..00000000000 --- a/tsl/src/remote/data_fetcher.c +++ /dev/null @@ -1,135 +0,0 @@ -/* - * 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. - */ -#include - -#include "data_fetcher.h" -#include "cursor_fetcher.h" -#include "copy_fetcher.h" -#include "guc.h" -#include "errors.h" - -#define DEFAULT_FETCH_SIZE 100 - -void -data_fetcher_init(DataFetcher *df, TSConnection *conn, const char *stmt, StmtParams *params, - TupleFactory *tf) -{ - Assert(df != NULL); - Assert(stmt != NULL); - - memset(df, 0, sizeof(DataFetcher)); - df->tuples = NULL; - df->conn = conn; - df->stmt = pstrdup(stmt); - df->stmt_params = params; - df->tf = tf; - - tuplefactory_set_per_tuple_mctx_reset(df->tf, false); - df->batch_mctx = AllocSetContextCreate(CurrentMemoryContext, - "data fetcher tuple batch data", - ALLOCSET_DEFAULT_SIZES); - df->tuple_mctx = df->batch_mctx; - df->req_mctx = AllocSetContextCreate(CurrentMemoryContext, - "data fetcher async request/response", - ALLOCSET_DEFAULT_SIZES); - df->fetch_size = DEFAULT_FETCH_SIZE; -} - -void -data_fetcher_validate(DataFetcher *df) -{ - /* ANALYZE command is accessing random tuples so we should never fail here when running ANALYZE - */ - if (df->next_tuple_idx != 0 && df->next_tuple_idx < df->num_tuples) - ereport(ERROR, - (errcode(ERRCODE_TS_INTERNAL_ERROR), - errmsg("invalid data fetcher state. sql: %s", df->stmt), - errhint("Shouldn't fetch new data before consuming existing."))); -} - -void -data_fetcher_store_tuple(DataFetcher *df, int row, TupleTableSlot *slot) -{ - if (row >= df->num_tuples) - { - /* No point in another fetch if we already detected EOF, though. */ - if (df->eof || df->funcs->fetch_data(df) == 0) - { - ExecClearTuple(slot); - return; - } - - /* More data was fetched so need to reset row index */ - row = 0; - Assert(row == df->next_tuple_idx); - } - - Assert(df->tuples != NULL); - Assert(row >= 0 && row < df->num_tuples); - - /* - * Return the next tuple. Must force the tuple into the slot since - * CustomScan initializes ss_ScanTupleSlot to a VirtualTupleTableSlot - * while we're storing a HeapTuple. - */ - ExecForceStoreHeapTuple(df->tuples[row], slot, /* shouldFree = */ false); -} - -void -data_fetcher_store_next_tuple(DataFetcher *df, TupleTableSlot *slot) -{ - data_fetcher_store_tuple(df, df->next_tuple_idx, slot); - - if (!TupIsNull(slot)) - df->next_tuple_idx++; - - Assert(df->next_tuple_idx <= df->num_tuples); -} - -void -data_fetcher_set_fetch_size(DataFetcher *df, int fetch_size) -{ - df->fetch_size = fetch_size; -} - -void -data_fetcher_set_tuple_mctx(DataFetcher *df, MemoryContext mctx) -{ - Assert(mctx != NULL); - df->tuple_mctx = mctx; -} - -void -data_fetcher_reset(DataFetcher *df) -{ - df->tuples = NULL; - df->num_tuples = 0; - df->next_tuple_idx = 0; - df->batch_count = 0; - df->eof = false; - MemoryContextReset(df->req_mctx); - MemoryContextReset(df->batch_mctx); -} - -/* - * This is the default implementation of starting the scan with the new - * parameters. It just closes the current scan and updates the parameter - * values, and the next scan is initialized from scratch. The prepared statement - * fetcher is more efficient than that, and reuses the prepared statement. - */ -void -data_fetcher_rescan(DataFetcher *df, StmtParams *params) -{ - df->funcs->close(df); - df->stmt_params = params; -} - -void -data_fetcher_free(DataFetcher *df) -{ - df->funcs->close(df); - pfree(df); -} diff --git a/tsl/src/remote/data_fetcher.h b/tsl/src/remote/data_fetcher.h deleted file mode 100644 index 22bb5457dca..00000000000 --- a/tsl/src/remote/data_fetcher.h +++ /dev/null @@ -1,101 +0,0 @@ -/* - * 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. - */ -#pragma once - -#include -#include -#include -#include - -#include "connection.h" -#include "stmt_params.h" -#include "guc.h" -#include "tuplefactory.h" - -typedef struct DataFetcher DataFetcher; - -typedef struct DataFetcherFuncs -{ - void (*close)(DataFetcher *data_fetcher); - - /* - * Read data in response to a fetch request. If no request has been sent, - * send it first. - */ - int (*fetch_data)(DataFetcher *data_fetcher); - - /* - * Restart the parameterized remote scan with the new parameter values. - */ - void (*rescan)(DataFetcher *data_fetcher, StmtParams *params); - - /* - * Restart the non-parameterized remote scan. This happens in some nested - * loop-type plans. Ideally we should materialize the data locally in this - * case, probably on plan level by putting a Materialize node above it. - */ - void (*rewind)(DataFetcher *data_fetcher); - - /* Send a request for new data. This doesn't read the data itself */ - void (*send_fetch_request)(DataFetcher *data_fetcher); - - /* Set the fetch (batch) size */ - void (*set_fetch_size)(DataFetcher *data_fetcher, int fetch_size); - void (*set_tuple_mctx)(DataFetcher *data_fetcher, MemoryContext mctx); - void (*store_next_tuple)(DataFetcher *data_fetcher, TupleTableSlot *slot); -} DataFetcherFuncs; - -typedef struct DataFetcher -{ - DataFetcherType type; - DataFetcherFuncs *funcs; - - TSConnection *conn; - TupleFactory *tf; - - MemoryContext req_mctx; /* Stores async request and response */ - MemoryContext batch_mctx; /* Stores batches of fetched tuples */ - MemoryContext tuple_mctx; - - const char *stmt; /* sql statement */ - StmtParams *stmt_params; /* sql statement params */ - - HeapTuple *tuples; /* array of currently-retrieved tuples */ - int num_tuples; /* # of tuples in array */ - int next_tuple_idx; /* index of next one to return */ - int fetch_size; /* # of tuples to fetch */ - int batch_count; /* how many batches (parts of result set) we've done */ - - bool open; - bool eof; - - AsyncRequest *data_req; /* a request to fetch data */ -} DataFetcher; - -void data_fetcher_free(DataFetcher *df); - -extern void data_fetcher_init(DataFetcher *df, TSConnection *conn, const char *stmt, - StmtParams *params, TupleFactory *tf); - -extern void data_fetcher_store_tuple(DataFetcher *df, int row, TupleTableSlot *slot); -extern void data_fetcher_store_next_tuple(DataFetcher *df, TupleTableSlot *slot); -extern void data_fetcher_set_fetch_size(DataFetcher *df, int fetch_size); -extern void data_fetcher_set_tuple_mctx(DataFetcher *df, MemoryContext mctx); -extern void data_fetcher_validate(DataFetcher *df); -extern void data_fetcher_reset(DataFetcher *df); -extern void data_fetcher_rescan(DataFetcher *df, StmtParams *params); - -#ifdef USE_ASSERT_CHECKING -static inline DataFetcher * -assert_df_type(DataFetcherType type, DataFetcher *df) -{ - Assert(df == NULL || df->type == type); - return df; -} -#define cast_fetcher(type, dfptr) ((type *) assert_df_type(type##Type, dfptr)) -#else -#define cast_fetcher(type, dfptr) ((type *) dfptr) -#endif /* USE_ASSERT_CHECKING */ diff --git a/tsl/src/remote/data_format.c b/tsl/src/remote/data_format.c deleted file mode 100644 index 6fa5161a423..00000000000 --- a/tsl/src/remote/data_format.c +++ /dev/null @@ -1,126 +0,0 @@ -/* - * 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. - */ -#include -#include -#include -#include -#include -#include -#include -#include - -#include "guc.h" -#include "data_format.h" - -static Oid -get_type_in_out_func(Oid type, bool *is_binary, bool force_text, Oid *type_io_param, bool out) -{ - HeapTuple type_tuple; - Form_pg_type pg_type; - Oid func; - - type_tuple = SearchSysCache1(TYPEOID, ObjectIdGetDatum(type)); - if (!HeapTupleIsValid(type_tuple)) - elog(ERROR, "cache lookup failed for type %u", type); - pg_type = (Form_pg_type) GETSTRUCT(type_tuple); - - if (!pg_type->typisdefined) - ereport(ERROR, - (errcode(ERRCODE_UNDEFINED_OBJECT), - errmsg("type %s is only a shell", format_type_be(type)))); - - if (out) - if (OidIsValid(pg_type->typsend) && !force_text) - { - func = pg_type->typsend; - *is_binary = true; - } - else - { - func = pg_type->typoutput; - *is_binary = false; - } - else - { - if (OidIsValid(pg_type->typreceive) && !force_text) - { - func = pg_type->typreceive; - *is_binary = true; - } - else - { - func = pg_type->typinput; - *is_binary = false; - } - *type_io_param = getTypeIOParam(type_tuple); - } - - ReleaseSysCache(type_tuple); - if (!OidIsValid(func)) - ereport(ERROR, - (errcode(ERRCODE_UNDEFINED_FUNCTION), - errmsg("no binary or text in/out function available for type %s", - format_type_be(type)))); - return func; -} - -/** - * Returns either type binary or text output function (text output is secondary) - **/ -Oid -data_format_get_type_output_func(Oid type, bool *is_binary, bool force_text) -{ - return get_type_in_out_func(type, is_binary, force_text, NULL, true); -} - -Oid -data_format_get_type_input_func(Oid type, bool *is_binary, bool force_text, Oid *type_io_param) -{ - return get_type_in_out_func(type, is_binary, force_text, type_io_param, false); -} - -AttConvInMetadata * -data_format_create_att_conv_in_metadata(TupleDesc tupdesc, bool force_text) -{ - AttConvInMetadata *att_conv_metadata; - int i = 0; - bool prev = true, isbinary = true; - - att_conv_metadata = palloc(sizeof(AttConvInMetadata)); - - BlessTupleDesc(tupdesc); - - att_conv_metadata->conv_funcs = palloc(tupdesc->natts * sizeof(FmgrInfo)); - att_conv_metadata->ioparams = palloc(tupdesc->natts * sizeof(Oid)); - att_conv_metadata->typmods = palloc(tupdesc->natts * sizeof(int32)); - - while (i < tupdesc->natts) - { - Oid funcoid; - - if (!TupleDescAttr(tupdesc, i)->attisdropped) - { - funcoid = data_format_get_type_input_func(TupleDescAttr(tupdesc, i)->atttypid, - &isbinary, - force_text || !isbinary, - &att_conv_metadata->ioparams[i]); - if (prev == !isbinary) - { - i = 0; /* in/out functions has to be eiher all binary or all text (PostgreSQL - limitation). Let's restart function discovery process */ - prev = false; - continue; - } - - fmgr_info(funcoid, &att_conv_metadata->conv_funcs[i]); - att_conv_metadata->typmods[i] = TupleDescAttr(tupdesc, i)->atttypmod; - } - i++; - } - - att_conv_metadata->binary = isbinary; - return att_conv_metadata; -} diff --git a/tsl/src/remote/data_format.h b/tsl/src/remote/data_format.h deleted file mode 100644 index 8fd95802b7b..00000000000 --- a/tsl/src/remote/data_format.h +++ /dev/null @@ -1,29 +0,0 @@ -/* - * 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. - */ -#pragma once - -#include -#include -#include - -#define FORMAT_TEXT 0 -#define FORMAT_BINARY 1 - -/* Metadata to convert PG result into tuples */ -typedef struct AttConvInMetadata -{ - FmgrInfo *conv_funcs; /* in functions for converting */ - Oid *ioparams; - int32 *typmods; - bool binary; /* if we use function with binary input */ -} AttConvInMetadata; - -extern AttConvInMetadata *data_format_create_att_conv_in_metadata(TupleDesc tupdesc, - bool force_text); - -extern Oid data_format_get_type_output_func(Oid type, bool *is_binary, bool force_text); -extern Oid data_format_get_type_input_func(Oid type, bool *is_binary, bool force_text, - Oid *type_io_param); diff --git a/tsl/src/remote/dist_commands.c b/tsl/src/remote/dist_commands.c deleted file mode 100644 index 233323d66f3..00000000000 --- a/tsl/src/remote/dist_commands.c +++ /dev/null @@ -1,600 +0,0 @@ -/* - * 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. - */ -#include -#include -#include -#include -#include -#include -#include -#include - -#include - -#include "dist_commands.h" -#include "dist_txn.h" -#include "connection_cache.h" -#include "data_node.h" -#include "dist_util.h" -#include "miscadmin.h" -#include "errors.h" -#include "deparse.h" -#include "debug_point.h" -#include "ts_catalog/array_utils.h" - -typedef struct DistPreparedStmt -{ - const char *data_node_name; - PreparedStmt *prepared_stmt; -} DistPreparedStmt; - -typedef struct DistCmdResponse -{ - const char *data_node; - AsyncResponseResult *result; -} DistCmdResponse; - -typedef struct DistCmdResult -{ - Size num_responses; - TypeFuncClass funcclass; /* Function class of invoked function, if any */ - Oid typeid; /* Expected result type, or InvalidOid */ - TupleDesc tupdesc; /* Tuple descriptor of invoked function - * result. Set if typeid is valid and has a - * composite return value */ - DistCmdResponse responses[FLEXIBLE_ARRAY_MEMBER]; -} DistCmdResult; - -static DistCmdResult * -ts_dist_cmd_collect_responses(List *requests) -{ - AsyncRequestSet *rs; - AsyncResponseResult *ar; - ListCell *lc; - DistCmdResult *results; - int i = 0; - - rs = async_request_set_create(); - results = palloc0(sizeof(DistCmdResult) + list_length(requests) * sizeof(DistCmdResponse)); - - foreach (lc, requests) - async_request_set_add(rs, lfirst(lc)); - - while ((ar = async_request_set_wait_ok_result(rs))) - { - DistCmdResponse *response = &results->responses[i]; - - response->result = ar; - response->data_node = pstrdup(async_response_result_get_user_data(ar)); - ++i; - } - - Assert(i == list_length(requests)); - results->num_responses = i; - return results; -} - -/* - * Invoke multiple SQL statements (commands) on the given data nodes. - * - * The list of data nodes can either be a list of data node names, or foreign - * server OIDs. - * - * If "transactional" is false then it means that the SQL should be executed - * in autocommit (implicit statement level commit) mode without the need for - * an explicit 2PC from the access node. - */ -DistCmdResult * -ts_dist_multi_cmds_params_invoke_on_data_nodes(List *cmd_descriptors, List *data_nodes, - bool transactional) -{ - ListCell *lc_data_node, *lc_cmd_descr; - List *requests = NIL; - DistCmdResult *results; - - if (data_nodes == NIL) - ereport(ERROR, - (errcode(ERRCODE_INVALID_PARAMETER_VALUE), - errmsg("no data nodes to execute command on"), - errhint("Add data nodes before executing a distributed command."))); - - Assert(list_length(data_nodes) == list_length(cmd_descriptors)); - switch (nodeTag(data_nodes)) - { - case T_OidList: - data_nodes = data_node_oids_to_node_name_list(data_nodes, ACL_NO_CHECK); - break; - case T_List: - /* Already in the format we want */ - data_node_name_list_check_acl(data_nodes, ACL_NO_CHECK); - break; - default: - elog(ERROR, "invalid list type %u", nodeTag(data_nodes)); - break; - } - - forboth (lc_data_node, data_nodes, lc_cmd_descr, cmd_descriptors) - { - const char *node_name = lfirst(lc_data_node); - AsyncRequest *req; - TSConnection *connection = - data_node_get_connection(node_name, REMOTE_TXN_NO_PREP_STMT, transactional); - DistCmdDescr *cmd_descr = lfirst(lc_cmd_descr); - const char *sql = cmd_descr->sql; - StmtParams *params = cmd_descr->params; - - ereport(DEBUG2, (errmsg_internal("sending \"%s\" to data node \"%s\"", sql, node_name))); - - if (params == NULL) - req = async_request_send(connection, sql); - else - req = async_request_send_with_params(connection, sql, params, FORMAT_TEXT); - - async_request_attach_user_data(req, (char *) node_name); - requests = lappend(requests, req); - } - - results = ts_dist_cmd_collect_responses(requests); - list_free(requests); - Assert(ts_dist_cmd_response_count(results) == (Size) list_length(data_nodes)); - - return results; -} - -DistCmdResult * -ts_dist_cmd_params_invoke_on_data_nodes(const char *sql, StmtParams *params, List *data_nodes, - bool transactional) -{ - List *cmd_descriptors = NIL; - DistCmdDescr cmd_descr = { .sql = sql, .params = params }; - DistCmdResult *results; - - for (int i = 0; i < list_length(data_nodes); ++i) - { - cmd_descriptors = lappend(cmd_descriptors, &cmd_descr); - } - results = - ts_dist_multi_cmds_params_invoke_on_data_nodes(cmd_descriptors, data_nodes, transactional); - list_free(cmd_descriptors); - return results; -} - -DistCmdResult * -ts_dist_cmd_invoke_on_data_nodes(const char *sql, List *data_nodes, bool transactional) -{ - return ts_dist_cmd_params_invoke_on_data_nodes(sql, NULL, data_nodes, transactional); -} - -DistCmdResult * -ts_dist_cmd_invoke_on_data_nodes_using_search_path(const char *sql, const char *search_path, - List *node_names, bool transactional) -{ - DistCmdResult *set_result; - DistCmdResult *results; - bool set_search_path = search_path != NULL; - - DEBUG_WAITPOINT("dist_cmd_using_search_path_1"); - - /* - * As a workaround for non-transactional execution, we expect the same connection - * to be present after we set search_path. - */ - remote_connection_cache_invalidation_ignore(true); - - if (set_search_path) - { - char *set_request = psprintf("SET search_path = %s, pg_catalog", search_path); - - set_result = ts_dist_cmd_invoke_on_data_nodes(set_request, node_names, transactional); - if (set_result) - ts_dist_cmd_close_response(set_result); - - pfree(set_request); - } - - DEBUG_WAITPOINT("dist_cmd_using_search_path_2"); - - results = ts_dist_cmd_invoke_on_data_nodes(sql, node_names, transactional); - - if (set_search_path) - { - set_result = ts_dist_cmd_invoke_on_data_nodes("SET search_path = pg_catalog", - node_names, - transactional); - if (set_result) - ts_dist_cmd_close_response(set_result); - } - - remote_connection_cache_invalidation_ignore(false); - return results; -} - -DistCmdResult * -ts_dist_multi_cmds_invoke_on_data_nodes_using_search_path(List *cmd_descriptors, - const char *search_path, List *node_names, - bool transactional) -{ - DistCmdResult *set_result; - DistCmdResult *results; - bool set_search_path = search_path != NULL; - - remote_connection_cache_invalidation_ignore(true); - - if (set_search_path) - { - char *set_request = psprintf("SET search_path = %s, pg_catalog", search_path); - - set_result = ts_dist_cmd_invoke_on_data_nodes(set_request, node_names, transactional); - if (set_result) - ts_dist_cmd_close_response(set_result); - - pfree(set_request); - } - - results = - ts_dist_multi_cmds_params_invoke_on_data_nodes(cmd_descriptors, node_names, transactional); - - if (set_search_path) - { - set_result = ts_dist_cmd_invoke_on_data_nodes("SET search_path = pg_catalog", - node_names, - transactional); - if (set_result) - ts_dist_cmd_close_response(set_result); - } - - remote_connection_cache_invalidation_ignore(false); - return results; -} - -DistCmdResult * -ts_dist_cmd_invoke_on_all_data_nodes(const char *sql) -{ - return ts_dist_cmd_invoke_on_data_nodes(sql, data_node_get_node_name_list(), true); -} - -/* - * Relay a function call to data nodes. - * - * A NIL list of data nodes means invoke on ALL data nodes. - */ -DistCmdResult * -ts_dist_cmd_invoke_func_call_on_data_nodes(FunctionCallInfo fcinfo, List *data_nodes) -{ - DistCmdResult *result; - - if (NIL == data_nodes) - data_nodes = data_node_get_node_name_list(); - - result = ts_dist_cmd_invoke_on_data_nodes(deparse_func_call(fcinfo), data_nodes, true); - - /* Initialize result conversion info in case caller wants to convert the - * result to datums. */ - result->funcclass = get_call_result_type(fcinfo, &result->typeid, &result->tupdesc); - - return result; -} - -DistCmdResult * -ts_dist_cmd_invoke_func_call_on_all_data_nodes(FunctionCallInfo fcinfo) -{ - return ts_dist_cmd_invoke_on_data_nodes(deparse_func_call(fcinfo), - data_node_get_node_name_list(), - true); -} - -/* - * Relay a function call to data nodes. - * - * This version throws away the result. - */ -void -ts_dist_cmd_func_call_on_data_nodes(FunctionCallInfo fcinfo, List *data_nodes) -{ - DistCmdResult *result = ts_dist_cmd_invoke_func_call_on_data_nodes(fcinfo, data_nodes); - - ts_dist_cmd_close_response(result); -} - -PGresult * -ts_dist_cmd_get_result_by_node_name(DistCmdResult *response, const char *node_name) -{ - for (size_t i = 0; i < response->num_responses; ++i) - { - DistCmdResponse *resp = &response->responses[i]; - - if (strcmp(node_name, resp->data_node) == 0) - return async_response_result_get_pg_result(resp->result); - } - return NULL; -} - -/* - * Get the n:th command result. - * - * Returns the n:th command result as given by the index, or NULL if no such - * result. - * - * Optionally get the name of the node that the result was from via the - * node_name parameter. - */ -PGresult * -ts_dist_cmd_get_result_by_index(DistCmdResult *response, Size index, const char **node_name) -{ - DistCmdResponse *rsp; - - if (index >= response->num_responses) - return NULL; - - rsp = &response->responses[index]; - - if (NULL != node_name) - *node_name = rsp->data_node; - - return async_response_result_get_pg_result(rsp->result); -} - -/* - * Get the number of responses in a distributed command result. - */ -Size -ts_dist_cmd_response_count(DistCmdResult *result) -{ - return result->num_responses; -} - -long -ts_dist_cmd_total_row_count(DistCmdResult *result) -{ - long num_rows = 0; - - for (size_t i = 0; i < result->num_responses; ++i) - { - DistCmdResponse *resp = &result->responses[i]; - - num_rows += PQntuples(async_response_result_get_pg_result(resp->result)); - } - - return num_rows; -} - -/* - * Convert an expected scalar return value. - * - * Convert the result of a remote function invokation returning a single - * scalar value. For example, a function returning a bool. - */ -Datum -ts_dist_cmd_get_single_scalar_result_by_index(DistCmdResult *result, Size index, bool *isnull, - const char **node_name_out) -{ - PGresult *pgres; - Oid typioparam; - Oid typinfunc; - const char *node_name; - - if (!OidIsValid(result->typeid)) - elog(ERROR, "invalid result type of distributed command"); - - if (result->funcclass != TYPEFUNC_SCALAR) - elog(ERROR, "distributed command result is not scalar"); - - pgres = ts_dist_cmd_get_result_by_index(result, index, &node_name); - - if (NULL == pgres) - elog(ERROR, "invalid index for distributed command result"); - - if (node_name_out) - *node_name_out = node_name; - - if (PQresultStatus(pgres) != PGRES_TUPLES_OK || PQntuples(pgres) != 1 || PQnfields(pgres) != 1) - ereport(ERROR, - (errcode(ERRCODE_TS_UNEXPECTED), - errmsg("unexpected response from data node \"%s\"", node_name))); - - if (PQgetisnull(pgres, 0, 0)) - { - if (isnull) - *isnull = true; - - return (Datum) 0; - } - - if (isnull) - *isnull = false; - - getTypeInputInfo(result->typeid, &typinfunc, &typioparam); - Assert(OidIsValid(typinfunc)); - - return OidInputFunctionCall(typinfunc, PQgetvalue(pgres, 0, 0), typioparam, -1); -} - -void -ts_dist_cmd_clear_result_by_index(DistCmdResult *response, Size index) -{ - DistCmdResponse *resp; - - if (index >= response->num_responses) - elog(ERROR, "no response for index %zu", index); - - resp = &response->responses[index]; - - if (resp->result != NULL) - { - async_response_result_close(resp->result); - resp->result = NULL; - } - - if (resp->data_node != NULL) - { - pfree((char *) resp->data_node); - resp->data_node = NULL; - } -} - -void -ts_dist_cmd_close_response(DistCmdResult *response) -{ - Size i; - - for (i = 0; i < response->num_responses; ++i) - ts_dist_cmd_clear_result_by_index(response, i); - - pfree(response); -} - -extern PreparedDistCmd * -ts_dist_cmd_prepare_command(const char *sql, size_t n_params, List *node_names) -{ - List *result = NIL; - ListCell *lc; - AsyncRequestSet *prep_requests = async_request_set_create(); - AsyncResponseResult *async_resp; - - if (node_names == NIL) - ereport(ERROR, - (errcode(ERRCODE_INVALID_PARAMETER_VALUE), - errmsg("invalid data nodes list"), - errdetail("Must specify a non-empty list of data nodes."))); - - foreach (lc, node_names) - { - const char *name = lfirst(lc); - TSConnection *connection = data_node_get_connection(name, REMOTE_TXN_USE_PREP_STMT, true); - DistPreparedStmt *cmd = palloc(sizeof(DistPreparedStmt)); - AsyncRequest *ar = async_request_send_prepare(connection, sql, n_params); - - cmd->data_node_name = pstrdup(name); - async_request_attach_user_data(ar, &cmd->prepared_stmt); - result = lappend(result, cmd); - async_request_set_add(prep_requests, ar); - } - - while ((async_resp = async_request_set_wait_ok_result(prep_requests))) - { - *(PreparedStmt **) async_response_result_get_user_data(async_resp) = - async_response_result_generate_prepared_stmt(async_resp); - async_response_result_close(async_resp); - } - - return result; -} - -PreparedDistCmd * -ts_dist_cmd_prepare_command_on_all_data_nodes(const char *sql, size_t n_params) -{ - return ts_dist_cmd_prepare_command(sql, n_params, data_node_get_node_name_list()); -} - -extern DistCmdResult * -ts_dist_cmd_invoke_prepared_command(PreparedDistCmd *command, const char *const *param_values) -{ - List *reqs = NIL; - ListCell *lc; - DistCmdResult *results; - - foreach (lc, command) - { - DistPreparedStmt *stmt = lfirst(lc); - AsyncRequest *req = async_request_send_prepared_stmt(stmt->prepared_stmt, param_values); - - async_request_attach_user_data(req, (char *) stmt->data_node_name); - reqs = lappend(reqs, req); - } - - results = ts_dist_cmd_collect_responses(reqs); - list_free(reqs); - return results; -} - -void -ts_dist_cmd_close_prepared_command(PreparedDistCmd *command) -{ - ListCell *lc; - - foreach (lc, command) - prepared_stmt_close(((DistPreparedStmt *) lfirst(lc))->prepared_stmt); - - list_free_deep(command); -} - -Datum -ts_dist_cmd_exec(PG_FUNCTION_ARGS) -{ -#if PG16_GE - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("distributed command execution is not supported"), - errdetail("Multi-node is not supported anymore on PostgreSQL >= 16."))); -#endif - - const char *query = PG_ARGISNULL(0) ? NULL : TextDatumGetCString(PG_GETARG_DATUM(0)); - ArrayType *data_nodes = PG_ARGISNULL(1) ? NULL : PG_GETARG_ARRAYTYPE_P(1); - bool transactional = PG_ARGISNULL(2) ? true : PG_GETARG_BOOL(2); - DistCmdResult *result; - List *data_node_list; - const char *search_path; - - if (!transactional) - TS_PREVENT_IN_TRANSACTION_BLOCK(true); - - if (NULL == query) - ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE), errmsg("empty command string"))); - - if (dist_util_membership() != DIST_MEMBER_ACCESS_NODE) - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("function must be run on the access node only"))); - - if (data_nodes == NULL) - data_node_list = data_node_get_node_name_list(); - else - { - int ndatanodes; - - if (ARR_NDIM(data_nodes) > 1) - ereport(ERROR, - (errcode(ERRCODE_INVALID_PARAMETER_VALUE), - errmsg("invalid data nodes list"), - errdetail("The array of data nodes cannot be multi-dimensional."))); - - if (ARR_HASNULL(data_nodes)) - ereport(ERROR, - (errcode(ERRCODE_INVALID_PARAMETER_VALUE), - errmsg("invalid data nodes list"), - errdetail("The array of data nodes cannot contain null values."))); - - ndatanodes = ArrayGetNItems(ARR_NDIM(data_nodes), ARR_DIMS(data_nodes)); - - if (ndatanodes == 0) - ereport(ERROR, - (errcode(ERRCODE_INVALID_PARAMETER_VALUE), - errmsg("invalid data nodes list"), - errdetail("The array of data nodes cannot be empty."))); - - data_node_list = data_node_array_to_node_name_list(data_nodes); - } - - /* Assert that the data node list is not empty. Since we checked that the - * function is run on an access node above, the list of data nodes must - * per definition be non-empty for the case when not specifying an - * explicit list of data nodes. For the case of explicitly specifying data - * nodes, we already checked for a non-empty array, and then validated all - * the specified data nodes. If there was a node in the list that is not a - * data node, we would already have thrown an error. */ - Assert(data_node_list != NIL); - search_path = GetConfigOption("search_path", false, false); - result = ts_dist_cmd_invoke_on_data_nodes_using_search_path(query, - search_path, - data_node_list, - transactional); - if (result) - ts_dist_cmd_close_response(result); - - list_free(data_node_list); - - PG_RETURN_VOID(); -} diff --git a/tsl/src/remote/dist_commands.h b/tsl/src/remote/dist_commands.h deleted file mode 100644 index b7d123774aa..00000000000 --- a/tsl/src/remote/dist_commands.h +++ /dev/null @@ -1,63 +0,0 @@ -/* - * 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. - */ -#pragma once - -#include "ts_catalog/catalog.h" - -#include "async.h" - -typedef struct DistCmdResult DistCmdResult; -typedef struct List PreparedDistCmd; -typedef struct DistCmdDescr -{ - const char *sql; - StmtParams *params; - -} DistCmdDescr; - -extern DistCmdResult *ts_dist_multi_cmds_params_invoke_on_data_nodes(List *cmd_descriptors, - List *data_nodes, - bool transactional); -extern DistCmdResult *ts_dist_cmd_invoke_on_data_nodes(const char *sql, List *data_nodes, - bool transactional); -extern DistCmdResult *ts_dist_cmd_params_invoke_on_data_nodes(const char *sql, StmtParams *params, - List *data_nodes, bool transactional); -extern DistCmdResult *ts_dist_multi_cmds_invoke_on_data_nodes_using_search_path( - List *cmd_descriptors, const char *search_path, List *node_names, bool transactional); -extern DistCmdResult *ts_dist_cmd_invoke_on_data_nodes_using_search_path(const char *sql, - const char *search_path, - List *node_names, - bool transactional); -extern DistCmdResult *ts_dist_cmd_invoke_on_all_data_nodes(const char *sql); -extern DistCmdResult *ts_dist_cmd_invoke_func_call_on_all_data_nodes(FunctionCallInfo fcinfo); -extern DistCmdResult *ts_dist_cmd_invoke_func_call_on_data_nodes(FunctionCallInfo fcinfo, - List *data_nodes); -extern Datum ts_dist_cmd_get_single_scalar_result_by_index(DistCmdResult *result, Size index, - bool *isnull, const char **node_name); -extern void ts_dist_cmd_func_call_on_data_nodes(FunctionCallInfo fcinfo, List *data_nodes); -extern PGresult *ts_dist_cmd_get_result_by_node_name(DistCmdResult *response, - const char *node_name); -extern PGresult *ts_dist_cmd_get_result_by_index(DistCmdResult *response, Size index, - const char **node_name); -extern void ts_dist_cmd_clear_result_by_index(DistCmdResult *response, Size index); -extern Size ts_dist_cmd_response_count(DistCmdResult *result); -extern long ts_dist_cmd_total_row_count(DistCmdResult *result); -extern void ts_dist_cmd_close_response(DistCmdResult *response); - -#define ts_dist_cmd_run_on_data_nodes(command, nodes, transactional) \ - ts_dist_cmd_close_response(ts_dist_cmd_invoke_on_data_nodes(command, nodes, transactional)); - -extern PreparedDistCmd *ts_dist_cmd_prepare_command(const char *sql, size_t n_params, - List *node_names); -extern PreparedDistCmd *ts_dist_cmd_prepare_command_on_all_data_nodes(const char *sql, - size_t n_params); - -extern DistCmdResult *ts_dist_cmd_invoke_prepared_command(PreparedDistCmd *command, - const char *const *param_values); - -extern void ts_dist_cmd_close_prepared_command(PreparedDistCmd *command); - -extern Datum ts_dist_cmd_exec(PG_FUNCTION_ARGS); diff --git a/tsl/src/remote/dist_txn.c b/tsl/src/remote/dist_txn.c deleted file mode 100644 index ef74bfa66f4..00000000000 --- a/tsl/src/remote/dist_txn.c +++ /dev/null @@ -1,601 +0,0 @@ -/* - * 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. - */ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include "dist_txn.h" -#include "ts_catalog/catalog.h" -#include "connection.h" -#include "async.h" -#include "errors.h" -#include "txn.h" -#include "txn_store.h" -#include "guc.h" - -#ifdef DEBUG - -static const DistTransactionEventHandler *event_handler = NULL; -static const char *eventnames[MAX_DTXN_EVENT] = { - [DTXN_EVENT_ANY] = "any", - [DTXN_EVENT_PRE_COMMIT] = "pre-commit", - [DTXN_EVENT_WAIT_COMMIT] = "waiting-commit", - [DTXN_EVENT_PRE_ABORT] = "pre-abort", - [DTXN_EVENT_PRE_PREPARE] = "pre-prepare-transaction", - [DTXN_EVENT_WAIT_PREPARE] = "waiting-prepare-transaction", - [DTXN_EVENT_POST_PREPARE] = "post-prepare-transaction", - [DTXN_EVENT_PRE_COMMIT_PREPARED] = "pre-commit-prepared", - [DTXN_EVENT_WAIT_COMMIT_PREPARED] = "waiting-commit-prepared", - [DTXN_EVENT_SUB_XACT_ABORT] = "subxact-abort", -}; - -void -remote_dist_txn_set_event_handler(const DistTransactionEventHandler *handler) -{ - event_handler = handler; -} - -static inline void -eventcallback(const DistTransactionEvent event) -{ - if (NULL != event_handler && NULL != event_handler->handler) - event_handler->handler(event, event_handler->data); -} - -DistTransactionEvent -remote_dist_txn_event_from_name(const char *eventname) -{ - int i; - - for (i = 0; i < MAX_DTXN_EVENT; i++) - { - if (strcmp(eventname, eventnames[i]) == 0) - return i; - } - - elog(ERROR, "invalid event name"); - pg_unreachable(); -} - -const char * -remote_dist_txn_event_name(const DistTransactionEvent event) -{ - return eventnames[event]; -} - -#else -#define eventcallback(event) \ - do \ - { \ - } while (0) -#endif - -static RemoteTxnStore *store = NULL; - -/* - * Get a connection which can be used to execute queries on the remote PostgreSQL - * data node with the user's authorization. A new connection is established - * if we don't already have a suitable one, and a transaction is opened at - * the right subtransaction nesting depth if we didn't do that already. - * - * will_prep_stmt must be true if caller intends to create any prepared - * statements. Since those don't go away automatically at transaction end - * (not even on error), we need this flag to cue manual cleanup. - */ -TSConnection * -remote_dist_txn_get_connection(TSConnectionId id, RemoteTxnPrepStmtOption prep_stmt_opt) -{ - bool found; - RemoteTxn *remote_txn; - - /* First time through, initialize the remote_txn_store */ - if (store == NULL) - store = remote_txn_store_create(TopTransactionContext); - - remote_txn = remote_txn_store_get(store, id, &found); - remote_txn_begin(remote_txn, GetCurrentTransactionNestLevel()); - remote_txn_set_will_prep_statement(remote_txn, prep_stmt_opt); - - return remote_txn_get_connection(remote_txn); -} - -/* This potentially deallocates prepared statements that were created in a subtxn - * that aborted before it deallocated the statement. - */ -static void -dist_txn_deallocate_prepared_stmts_if_needed() -{ - RemoteTxn *remote_txn; - - /* below deallocate only happens on error so not worth making async */ - remote_txn_store_foreach(store, remote_txn) - { - remote_txn_deallocate_prepared_stmts_if_needed(remote_txn); - } -} - -/* Perform actions on one-phase pre-commit. - * Mainly just send a COMMIT to all remote nodes and wait for successes. - */ -static void -dist_txn_xact_callback_1pc_pre_commit() -{ - RemoteTxn *remote_txn; - Catalog *catalog = ts_catalog_get(); - AsyncRequestSet *ars = async_request_set_create(); - - eventcallback(DTXN_EVENT_PRE_COMMIT); - - /* - * In 1PC, we don't need to add entries to the remote_txn table. However - * we do need to take a SHARE lock on it to interlock with any distributed - * restore point activity that might be happening in parallel. - * - * The catalog table lock is kept until the transaction completes in order to - * synchronize with distributed restore point creation - */ - LockRelationOid(catalog->tables[REMOTE_TXN].id, AccessShareLock); - - /* send a commit to all connections */ - remote_txn_store_foreach(store, remote_txn) - { - Assert(remote_connection_xact_depth_get(remote_txn_get_connection(remote_txn)) > 0); - - /* Commit all remote transactions during pre-commit */ - async_request_set_add(ars, remote_txn_async_send_commit(remote_txn)); - } - - eventcallback(DTXN_EVENT_WAIT_COMMIT); - - /* async collect all the replies */ - async_request_set_wait_all_ok_commands(ars); - dist_txn_deallocate_prepared_stmts_if_needed(); -} - -/* - * Abort on the access node. - * - * The access node needs to send aborts to all of the remote endpoints. This - * code should not throw errors itself, since we are already in abort due to a - * previous error. Instead, we try to emit errors as warnings. For safety, we - * should probaby try-catch and swallow any potential lower-layer errors given - * that we're doing remote calls over the network. But the semantics for - * capturing and proceeding after such recursive errors are unclear. - */ -static void -dist_txn_xact_callback_abort() -{ - RemoteTxn *remote_txn; - - eventcallback(DTXN_EVENT_PRE_ABORT); - - remote_txn_store_foreach(store, remote_txn) - { - if (remote_txn_is_ongoing(remote_txn) && !remote_txn_abort(remote_txn)) - elog(WARNING, - "transaction rollback on data node \"%s\" failed", - remote_connection_node_name(remote_txn_get_connection(remote_txn))); - } -} - -/* - * Reject transactions that didn't successfully complete a transaction - * transition at some point. - */ -static void -reject_transaction_with_incomplete_transition(RemoteTxn *remote_txn) -{ - const TSConnection *conn = remote_txn_get_connection(remote_txn); - - if (remote_connection_xact_is_transitioning(conn)) - { - NameData nodename; - - namestrcpy(&nodename, remote_connection_node_name(conn)); - remote_txn_store_remove(store, remote_txn_get_connection_id(remote_txn)); - - ereport(ERROR, - (errcode(ERRCODE_CONNECTION_EXCEPTION), - errmsg("connection to data node \"%s\" was lost", NameStr(nodename)))); - } -} - -static void -reject_transactions_with_incomplete_transitions(void) -{ - RemoteTxn *remote_txn; - - remote_txn_store_foreach(store, remote_txn) - { - reject_transaction_with_incomplete_transition(remote_txn); - } -} - -static void -cleanup_at_end_of_transaction(void) -{ - RemoteTxn *remote_txn; - - remote_txn_store_foreach(store, remote_txn) - { - TSConnection *conn = remote_txn_get_connection(remote_txn); - - /* The connection could have failed at START TRANSACTION, in which - * case the depth is 0. Otherwise, we'd expect depth 1. */ - if (remote_connection_xact_depth_get(conn) > 0) - { - PGconn *pgconn = remote_connection_get_pg_conn(conn); - - /* Indicate we're out of the transaction */ - Assert(remote_connection_xact_depth_get(conn) == 1); - remote_connection_xact_depth_dec(conn); - - /* Cleanup connections with failed transactions */ - if (PQstatus(pgconn) != CONNECTION_OK || PQtransactionStatus(pgconn) != PQTRANS_IDLE || - remote_connection_xact_is_transitioning(conn)) - { - elog(DEBUG3, "discarding connection %p", conn); - remote_txn_store_remove(store, remote_txn_get_connection_id(remote_txn)); - } - } - } - - remote_txn_store_destroy(store); - store = NULL; - - /* - * cursor are per-connection and txn so it's safe to reset at the end of - * the txn. - */ - remote_connection_reset_cursor_number(); -} - -/* - * Transaction callback for one-phase commits. - * - * With one-phase commits, we send a remote commit during local pre-commit or - * a remote abort during local abort. - */ -static void -dist_txn_xact_callback_1pc(XactEvent event, void *arg) -{ - switch (event) - { - case XACT_EVENT_PRE_COMMIT: - case XACT_EVENT_PARALLEL_PRE_COMMIT: - reject_transactions_with_incomplete_transitions(); - dist_txn_xact_callback_1pc_pre_commit(); - break; - case XACT_EVENT_PRE_PREPARE: - - /* - * Cannot prepare stuff on the access node. - */ - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("cannot prepare a transaction that modified " - "remote tables"))); - break; - case XACT_EVENT_PARALLEL_COMMIT: - case XACT_EVENT_COMMIT: - case XACT_EVENT_PREPARE: - /* Pre-commit should have closed the open transaction in 1pc */ - elog(ERROR, "missed cleaning up connection during pre-commit"); - break; - case XACT_EVENT_PARALLEL_ABORT: - case XACT_EVENT_ABORT: - dist_txn_xact_callback_abort(); - break; - } - - /* In one-phase commit, we're done irrespective of event */ - cleanup_at_end_of_transaction(); -} - -static void -dist_txn_send_prepare_transaction() -{ - RemoteTxn *remote_txn; - AsyncRequestSet *ars = async_request_set_create(); - AsyncResponse *error_response = NULL; - AsyncResponse *res; - - eventcallback(DTXN_EVENT_PRE_PREPARE); - - /* send a prepare transaction to all connections */ - remote_txn_store_foreach(store, remote_txn) - { - AsyncRequest *req; - - remote_txn_write_persistent_record(remote_txn); - req = remote_txn_async_send_prepare_transaction(remote_txn); - async_request_set_add(ars, req); - } - - eventcallback(DTXN_EVENT_WAIT_PREPARE); - - /* - * async collect the replies. Since errors in PREPARE TRANSACTION are not - * uncommon, handle them gracefully: delay throwing errors in results - * until all responses collected since you need to mark - * changing_xact_state correctly. So throw errors on connection errors but - * not errors in results. - */ - error_response = NULL; - while ((res = async_request_set_wait_any_response(ars))) - { - switch (async_response_get_type(res)) - { - case RESPONSE_COMMUNICATION_ERROR: - case RESPONSE_ERROR: - case RESPONSE_ROW: - case RESPONSE_TIMEOUT: - elog(DEBUG3, "error during second phase of two-phase commit"); - async_response_report_error(res, ERROR); - continue; - case RESPONSE_RESULT: - { - AsyncResponseResult *response_result = (AsyncResponseResult *) res; - bool success = - PQresultStatus(async_response_result_get_pg_result(response_result)) == - PGRES_COMMAND_OK; - - if (!success) - { - /* save first error, warn about subsequent errors */ - if (error_response == NULL) - error_response = (AsyncResponse *) response_result; - else - async_response_report_error((AsyncResponse *) response_result, WARNING); - } - else - async_response_close(res); - break; - } - } - } - - if (error_response != NULL) - async_response_report_error(error_response, ERROR); - - eventcallback(DTXN_EVENT_POST_PREPARE); -} - -static void -dist_txn_send_commit_prepared_transaction() -{ - RemoteTxn *remote_txn; - AsyncRequestSet *ars = async_request_set_create(); - AsyncResponse *res; - - /* - * send a commit transaction to all connections and asynchronously collect - * the replies - */ - remote_txn_store_foreach(store, remote_txn) - { - AsyncRequest *req; - - req = remote_txn_async_send_commit_prepared(remote_txn); - - if (req == NULL) - { - elog(DEBUG3, "error during second phase of two-phase commit"); - continue; - } - - async_request_set_add(ars, req); - } - - eventcallback(DTXN_EVENT_WAIT_COMMIT_PREPARED); - - /* async collect the replies */ - while ((res = async_request_set_wait_any_response(ars))) - { - /* throw WARNINGS not ERRORS here */ - /* - * NOTE: warnings make sure that all data nodes get a commit prepared. - * But, there is arguably some weirdness here in terms of RYOW if - * there is an error. - */ - AsyncResponseResult *response_result; - - switch (async_response_get_type(res)) - { - case RESPONSE_COMMUNICATION_ERROR: - case RESPONSE_ERROR: - case RESPONSE_ROW: - case RESPONSE_TIMEOUT: - elog(DEBUG3, "error during second phase of two-phase commit"); - async_response_report_error(res, WARNING); - continue; - case RESPONSE_RESULT: - response_result = (AsyncResponseResult *) res; - if (PQresultStatus(async_response_result_get_pg_result(response_result)) != - PGRES_COMMAND_OK) - async_response_report_error(res, WARNING); - else - async_response_close(res); - break; - } - } -} - -/* - * Transaction callback for two-phase commit. - * - * With two-phase commits, we write a persistent record and send a remote - * PREPARE TRANSACTION during local pre-commit. After commit we send a remote - * COMMIT TRANSACTION. - */ -static void -dist_txn_xact_callback_2pc(XactEvent event, void *arg) -{ - switch (event) - { - case XACT_EVENT_PARALLEL_PRE_COMMIT: - case XACT_EVENT_PRE_COMMIT: - reject_transactions_with_incomplete_transitions(); - dist_txn_send_prepare_transaction(); - dist_txn_deallocate_prepared_stmts_if_needed(); - break; - case XACT_EVENT_PRE_PREPARE: - case XACT_EVENT_PREPARE: - - /* - * Cannot prepare stuff on the access node. - */ - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("cannot prepare a transaction that modified " - "remote tables"))); - break; - case XACT_EVENT_PARALLEL_COMMIT: - case XACT_EVENT_COMMIT: - eventcallback(DTXN_EVENT_PRE_COMMIT_PREPARED); - - /* - * We send a commit here so that future commands on this - * connection get read-your-own-writes semantics. Later, we can - * optimize latency on connections by doing this in a background - * process and using IPC to assure RYOW - */ - dist_txn_send_commit_prepared_transaction(); - - /* - * NOTE: You cannot delete the remote_txn_persistent_record here - * because you are out of transaction. Therefore cleanup of those - * entries has to happen in a background process or manually. - */ - cleanup_at_end_of_transaction(); - break; - case XACT_EVENT_PARALLEL_ABORT: - case XACT_EVENT_ABORT: - dist_txn_xact_callback_abort(); - cleanup_at_end_of_transaction(); - break; - } -} - -static void -dist_txn_xact_callback(XactEvent event, void *arg) -{ - bool use_2pc; - char *xactReadOnly; - - /* Quick exit if no connections were touched in this transaction. */ - if (store == NULL) - return; - - /* - * Windows MSVC builds have linking issues for GUC variables from postgres for - * use inside this extension. So we use GetConfigOptionByName - */ - xactReadOnly = GetConfigOptionByName("transaction_read_only", NULL, false); - - /* - * The decision to use 2PC rests on multiple factors: - * - * 1) if ts_guc_enable_2pc is enabled and it's a regular backend use it - * - * 2) if ts_guc_enable_2pc is enabled but we are running a read only txn, don't use it - * - * We might be tempted to use 1PC if just one DN is involved in the transaction. - * However, it's possible that a transaction which involves data on AN and the one DN could get - * a failure at the end of the COMMIT processing on the AN due to issues in local AN data. In - * such a case since we send a COMMIT at "XACT_EVENT_PRE_COMMIT" event time to the DN, we might - * end up with a COMMITTED DN but an aborted AN! Hence this optimization is not possible to - * guarantee transactional semantics. - */ - use_2pc = (ts_guc_enable_2pc && strncmp(xactReadOnly, "on", sizeof("on")) != 0); -#ifdef TS_DEBUG - ereport(DEBUG3, (errmsg("use 2PC: %s", use_2pc ? "true" : "false"))); -#endif - - if (use_2pc) - dist_txn_xact_callback_2pc(event, arg); - else - dist_txn_xact_callback_1pc(event, arg); -} - -/* - * Subtransaction callback handler. - * - * If the subtxn was committed, send a RELEASE SAVEPOINT to the remote nodes. - * If the subtxn was aborted, send a ROLLBACK SAVEPOINT and set a deferred - * error if that fails. - */ -static void -dist_txn_subxact_callback(SubXactEvent event, SubTransactionId mySubid, - SubTransactionId parentSubid, void *arg) -{ - RemoteTxn *remote_txn; - int curlevel; - - /* Quick exit if no connections were touched in this transaction. */ - if (store == NULL) - return; - - switch (event) - { - case SUBXACT_EVENT_START_SUB: - case SUBXACT_EVENT_COMMIT_SUB: - /* Nothing to do at subxact start, nor after commit. */ - return; - case SUBXACT_EVENT_PRE_COMMIT_SUB: - reject_transactions_with_incomplete_transitions(); - break; - case SUBXACT_EVENT_ABORT_SUB: - eventcallback(DTXN_EVENT_SUB_XACT_ABORT); - break; - } - - curlevel = GetCurrentTransactionNestLevel(); - - remote_txn_store_foreach(store, remote_txn) - { - TSConnection *conn = remote_txn_get_connection(remote_txn); - - if (!remote_txn_is_at_sub_txn_level(remote_txn, curlevel)) - continue; - - if (event == SUBXACT_EVENT_PRE_COMMIT_SUB) - { - reject_transaction_with_incomplete_transition(remote_txn); - remote_txn_sub_txn_pre_commit(remote_txn, curlevel); - } - else - { - Assert(event == SUBXACT_EVENT_ABORT_SUB); - remote_txn_sub_txn_abort(remote_txn, curlevel); - } - - remote_connection_xact_depth_dec(conn); - } -} - -void -_remote_dist_txn_init() -{ - RegisterXactCallback(dist_txn_xact_callback, NULL); - RegisterSubXactCallback(dist_txn_subxact_callback, NULL); -} - -void -_remote_dist_txn_fini() -{ - /* can't unregister callbacks */ - if (NULL != store) - { - remote_txn_store_destroy(store); - store = NULL; - } -} diff --git a/tsl/src/remote/dist_txn.h b/tsl/src/remote/dist_txn.h deleted file mode 100644 index 44b2e06a17a..00000000000 --- a/tsl/src/remote/dist_txn.h +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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. - */ -#pragma once - -#include -#include -#include - -#include "txn.h" - -/* Get a remote connection for a distributed txn corresponding to the current local txn. */ - -extern TSConnection *remote_dist_txn_get_connection(TSConnectionId id, - RemoteTxnPrepStmtOption prep_stmt); - -#ifdef DEBUG - -typedef enum DistTransactionEvent -{ - DTXN_EVENT_ANY, - DTXN_EVENT_PRE_COMMIT, - DTXN_EVENT_WAIT_COMMIT, - DTXN_EVENT_PRE_ABORT, - DTXN_EVENT_PRE_PREPARE, - DTXN_EVENT_WAIT_PREPARE, - DTXN_EVENT_POST_PREPARE, - DTXN_EVENT_PRE_COMMIT_PREPARED, - DTXN_EVENT_WAIT_COMMIT_PREPARED, - DTXN_EVENT_SUB_XACT_ABORT, -} DistTransactionEvent; - -#define MAX_DTXN_EVENT (DTXN_EVENT_SUB_XACT_ABORT + 1) - -typedef struct DistTransactionEventHandler -{ - void (*handler)(const DistTransactionEvent event, void *data); - void *data; -} DistTransactionEventHandler; - -extern void remote_dist_txn_set_event_handler(const DistTransactionEventHandler *handler); -extern DistTransactionEvent remote_dist_txn_event_from_name(const char *eventname); -extern const char *remote_dist_txn_event_name(const DistTransactionEvent event); - -#endif /* DEBUG */ - -void _remote_dist_txn_init(void); -void _remote_dist_txn_fini(void); diff --git a/tsl/src/remote/prepared_statement_fetcher.c b/tsl/src/remote/prepared_statement_fetcher.c deleted file mode 100644 index 74362714dcc..00000000000 --- a/tsl/src/remote/prepared_statement_fetcher.c +++ /dev/null @@ -1,416 +0,0 @@ -/* - * 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. - */ - -#include - -#include "prepared_statement_fetcher.h" -#include "tuplefactory.h" -#include "async.h" - -typedef struct PreparedStatementFetcher -{ - DataFetcher state; - - /* Data for virtual tuples of the current retrieved batch. */ - Datum *batch_values; - bool *batch_nulls; -} PreparedStatementFetcher; - -static void prepared_statement_fetcher_send_fetch_request(DataFetcher *df); -static void prepared_statement_fetcher_reset(PreparedStatementFetcher *fetcher); -static int prepared_statement_fetcher_fetch_data(DataFetcher *df); -static void prepared_statement_fetcher_set_fetch_size(DataFetcher *df, int fetch_size); -static void prepared_statement_fetcher_set_tuple_memcontext(DataFetcher *df, MemoryContext mctx); -static void prepared_statement_fetcher_store_next_tuple(DataFetcher *df, TupleTableSlot *slot); -static void prepared_statement_fetcher_rewind(DataFetcher *df); -static void prepared_statement_fetcher_rescan(DataFetcher *df, StmtParams *params); -static void prepared_statement_fetcher_close(DataFetcher *df); - -static DataFetcherFuncs funcs = { - .close = prepared_statement_fetcher_close, - .fetch_data = prepared_statement_fetcher_fetch_data, - .rescan = prepared_statement_fetcher_rescan, - .rewind = prepared_statement_fetcher_rewind, - .send_fetch_request = prepared_statement_fetcher_send_fetch_request, - .set_fetch_size = prepared_statement_fetcher_set_fetch_size, - .set_tuple_mctx = prepared_statement_fetcher_set_tuple_memcontext, - .store_next_tuple = prepared_statement_fetcher_store_next_tuple, -}; - -static void -prepared_statement_fetcher_set_fetch_size(DataFetcher *df, int fetch_size) -{ - PreparedStatementFetcher *fetcher = cast_fetcher(PreparedStatementFetcher, df); - data_fetcher_set_fetch_size(&fetcher->state, fetch_size); -} - -static void -prepared_statement_fetcher_set_tuple_memcontext(DataFetcher *df, MemoryContext mctx) -{ - PreparedStatementFetcher *fetcher = cast_fetcher(PreparedStatementFetcher, df); - data_fetcher_set_tuple_mctx(&fetcher->state, mctx); -} - -static void -prepared_statement_fetcher_reset(PreparedStatementFetcher *fetcher) -{ - /* Drain the connection, reporting any errors. */ - TSConnection *conn = fetcher->state.conn; - PGresult *res; - while ((res = remote_connection_get_result(conn, TS_NO_TIMEOUT)) != NULL) - { - char *sqlstate = PQresultErrorField(res, PG_DIAG_SQLSTATE); - if (sqlstate != NULL && strcmp(sqlstate, "00000") == 0) - { - remote_result_elog(res, ERROR); - } - PQclear(res); - } - - fetcher->state.open = false; - data_fetcher_reset(&fetcher->state); -} - -static void -prepared_statement_fetcher_send_fetch_request(DataFetcher *df) -{ - PreparedStatementFetcher *fetcher = cast_fetcher(PreparedStatementFetcher, df); - - if (fetcher->state.open) - { - /* data request has already been sent */ - Assert(fetcher->state.data_req != NULL); - return; - } - - /* make sure to have a clean state */ - prepared_statement_fetcher_reset(fetcher); - - TSConnection *conn = fetcher->state.conn; - if (remote_connection_get_status(conn) != CONN_IDLE) - { - elog(ERROR, "unexpected activity on data node connection when sending fetch request"); - } - - PGresult *pgres = remote_connection_get_result(conn, TS_NO_TIMEOUT); - if (pgres != NULL) - { - char *sqlstate = PQresultErrorField(pgres, PG_DIAG_SQLSTATE); - if (sqlstate != NULL && strcmp(sqlstate, "00000") == 0) - { - remote_result_elog(pgres, ERROR); - } - - elog(ERROR, - "unexpected activity on data node connection when sending fetch request " - "(PQresultStatus %d)", - PQresultStatus(pgres)); - } - - PGconn *pg_conn = remote_connection_get_pg_conn(conn); - int ret = PQsendQueryPrepared(pg_conn, - /* stmtName = */ "", - stmt_params_num_params(fetcher->state.stmt_params), - stmt_params_values(fetcher->state.stmt_params), - stmt_params_lengths(fetcher->state.stmt_params), - stmt_params_formats(fetcher->state.stmt_params), - tuplefactory_is_binary(fetcher->state.tf) ? FORMAT_BINARY : - FORMAT_TEXT); - - if (ret != 1) - { - TSConnectionError err; - remote_connection_get_error(conn, &err); - remote_connection_error_elog(&err, ERROR); - } - - if (!remote_connection_set_single_row_mode(conn)) - ereport(ERROR, - (errcode(ERRCODE_CONNECTION_FAILURE), - errmsg("could not set single-row mode on connection to \"%s\"", - remote_connection_node_name(fetcher->state.conn)), - errdetail("The aborted statement is: %s.", fetcher->state.stmt), - errhint("Row-by-row fetching of data is not supported together with sub-queries." - " Use cursor fetcher instead."))); - - fetcher->state.data_req = (void *) 1; - fetcher->state.open = true; -} - -/* - * Process response for ongoing async request - */ -static int -prepared_statement_fetcher_complete(PreparedStatementFetcher *fetcher) -{ - MemoryContext oldcontext; - - Assert(fetcher->state.open); - Assert(fetcher->state.data_req != NULL); - - data_fetcher_validate(&fetcher->state); - - /* - * We'll store the tuples in the batch_mctx. First, flush the previous - * batch. - */ - MemoryContextReset(fetcher->state.batch_mctx); - oldcontext = MemoryContextSwitchTo(fetcher->state.batch_mctx); - const int nattrs = tuplefactory_get_nattrs(fetcher->state.tf); - const int total = nattrs * fetcher->state.fetch_size; - fetcher->batch_nulls = palloc(sizeof(bool) * total); - for (int i = 0; i < total; i++) - { - fetcher->batch_nulls[i] = true; - } - fetcher->batch_values = palloc0(sizeof(Datum) * total); - - TSConnection *conn = fetcher->state.conn; - PGconn *pg_conn = remote_connection_get_pg_conn(conn); - if (PQsetnonblocking(pg_conn, 0) != 0) - { - remote_connection_elog(conn, ERROR); - } - - PG_TRY(); - { - int i; - - for (i = 0; i < fetcher->state.fetch_size; i++) - { - PGresult *res; - - res = remote_connection_get_result(conn, TS_NO_TIMEOUT); - - if (!(PQresultStatus(res) == PGRES_SINGLE_TUPLE || - PQresultStatus(res) == PGRES_TUPLES_OK)) - { - remote_result_elog(res, ERROR); - } - - if (PQresultStatus(res) == PGRES_TUPLES_OK) - { - /* fetched all the data */ - Assert(PQntuples(res) == 0); - PQclear(res); - - fetcher->state.eof = true; - break; - } - - Assert(PQresultStatus(res) == PGRES_SINGLE_TUPLE); - /* Allow creating tuples in alternative memory context if user has set - * it explicitly, otherwise same as batch_mctx */ - MemoryContextSwitchTo(fetcher->state.tuple_mctx); - - PG_USED_FOR_ASSERTS_ONLY ItemPointer ctid = - tuplefactory_make_virtual_tuple(fetcher->state.tf, - res, - 0, - PQbinaryTuples(res), - &fetcher->batch_values[i * nattrs], - &fetcher->batch_nulls[i * nattrs]); - - /* - * This fetcher uses virtual tuples that can't hold ctid, so if we're - * receiving a ctid here, we're doing something wrong. - */ - Assert(ctid == NULL); - - PQclear(res); - } - /* We need to manually reset the context since we've turned off per tuple reset */ - tuplefactory_reset_mctx(fetcher->state.tf); - - fetcher->state.num_tuples = i; - fetcher->state.next_tuple_idx = 0; - fetcher->state.batch_count++; - - if (fetcher->state.eof) - { - fetcher->state.data_req = NULL; - } - } - PG_CATCH(); - { - if (NULL != fetcher->state.data_req) - { - fetcher->state.data_req = NULL; - } - - PG_RE_THROW(); - } - PG_END_TRY(); - - MemoryContextSwitchTo(oldcontext); - - return fetcher->state.num_tuples; -} - -static int -prepared_statement_fetcher_fetch_data(DataFetcher *df) -{ - PreparedStatementFetcher *fetcher = cast_fetcher(PreparedStatementFetcher, df); - - if (fetcher->state.eof) - return 0; - - if (!fetcher->state.open) - prepared_statement_fetcher_send_fetch_request(df); - - return prepared_statement_fetcher_complete(fetcher); -} - -static void -prepared_statement_fetcher_store_tuple(DataFetcher *df, int row, TupleTableSlot *slot) -{ - PreparedStatementFetcher *fetcher = cast_fetcher(PreparedStatementFetcher, df); - - ExecClearTuple(slot); - - if (row >= df->num_tuples) - { - if (df->eof || df->funcs->fetch_data(df) == 0) - { - return; - } - - row = 0; - Assert(row == df->next_tuple_idx); - } - - Assert(fetcher->batch_values != NULL); - Assert(fetcher->batch_nulls != NULL); - Assert(row >= 0 && row < df->num_tuples); - - const int nattrs = tuplefactory_get_nattrs(fetcher->state.tf); - slot->tts_values = &fetcher->batch_values[nattrs * row]; - slot->tts_isnull = &fetcher->batch_nulls[nattrs * row]; - ExecStoreVirtualTuple(slot); -} - -static void -prepared_statement_fetcher_store_next_tuple(DataFetcher *df, TupleTableSlot *slot) -{ - prepared_statement_fetcher_store_tuple(df, df->next_tuple_idx, slot); - - if (!TupIsNull(slot)) - df->next_tuple_idx++; - - Assert(df->next_tuple_idx <= df->num_tuples); -} - -DataFetcher * -prepared_statement_fetcher_create_for_scan(TSConnection *conn, const char *stmt, StmtParams *params, - TupleFactory *tf) -{ - PreparedStatementFetcher *fetcher = palloc0(sizeof(PreparedStatementFetcher)); - - data_fetcher_init(&fetcher->state, conn, stmt, params, tf); - fetcher->state.type = PreparedStatementFetcherType; - fetcher->state.funcs = &funcs; - - PGconn *pg_conn = remote_connection_get_pg_conn(conn); - if (remote_connection_get_status(conn) != CONN_IDLE) - { - elog(ERROR, - "unexpected activity on data node connection when creating the row-by-row fetcher"); - } - - /* - * Force using the generic plan for each execution of the data node query, - * because it would be very expensive and pointless to replan it for each - * subsequent parameter value. - */ - PGresult *res = remote_connection_exec(conn, "SET plan_cache_mode = 'force_generic_plan'"); - if (PQresultStatus(res) != PGRES_COMMAND_OK) - { - TSConnectionError err; - remote_connection_get_result_error(res, &err); - remote_connection_error_elog(&err, ERROR); - } - PQclear(res); - - if (1 != PQsendPrepare(pg_conn, - /* stmtName = */ "", - stmt, - stmt_params_num_params(params), - /* paramTypes = */ NULL)) - { - TSConnectionError err; - remote_connection_get_error(conn, &err); - remote_connection_error_elog(&err, ERROR); - } - - res = remote_connection_get_result(conn, TS_NO_TIMEOUT); - if (PQresultStatus(res) != PGRES_COMMAND_OK) - { - TSConnectionError err; - remote_connection_get_result_error(res, &err); - remote_connection_error_elog(&err, ERROR); - } - - PQclear(res); - - return &fetcher->state; -} - -static void -prepared_statement_fetcher_close(DataFetcher *df) -{ - PreparedStatementFetcher *fetcher = cast_fetcher(PreparedStatementFetcher, df); - - if (fetcher->state.open) - { - if (fetcher->state.data_req != NULL) - { - fetcher->state.data_req = NULL; - } - prepared_statement_fetcher_reset(fetcher); - } - else - { - Assert(fetcher->state.data_req == NULL); - Assert(fetcher->state.num_tuples == 0); - -#ifdef USE_ASSERT_CHECKING - TSConnection *conn = fetcher->state.conn; - PGconn *pg_conn = remote_connection_get_pg_conn(conn); - - Assert(PQtransactionStatus(pg_conn) != PQTRANS_ACTIVE); -#endif - } - - PGresult *res = remote_connection_exec(fetcher->state.conn, "RESET plan_cache_mode"); - if (PQresultStatus(res) != PGRES_COMMAND_OK) - { - TSConnectionError err; - remote_connection_get_result_error(res, &err); - remote_connection_error_elog(&err, ERROR); - } - PQclear(res); -} - -static void -prepared_statement_fetcher_rewind(DataFetcher *df) -{ - PreparedStatementFetcher *fetcher = cast_fetcher(PreparedStatementFetcher, df); - - if (fetcher->state.batch_count > 1) - /* we're over the first batch so we need to reset fetcher and restart from clean state */ - prepared_statement_fetcher_reset(fetcher); - else - /* we can reuse current batch of results */ - fetcher->state.next_tuple_idx = 0; -} - -static void -prepared_statement_fetcher_rescan(DataFetcher *df, StmtParams *params) -{ - PreparedStatementFetcher *fetcher = cast_fetcher(PreparedStatementFetcher, df); - prepared_statement_fetcher_reset(fetcher); - df->stmt_params = params; -} diff --git a/tsl/src/remote/prepared_statement_fetcher.h b/tsl/src/remote/prepared_statement_fetcher.h deleted file mode 100644 index ca0b4acee11..00000000000 --- a/tsl/src/remote/prepared_statement_fetcher.h +++ /dev/null @@ -1,14 +0,0 @@ -/* - * 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. - */ -#pragma once - -#include - -#include "data_fetcher.h" - -extern DataFetcher *prepared_statement_fetcher_create_for_scan(TSConnection *conn, const char *stmt, - StmtParams *params, - TupleFactory *tf); diff --git a/tsl/src/remote/stmt_params.c b/tsl/src/remote/stmt_params.c deleted file mode 100644 index ba06c4cd7a5..00000000000 --- a/tsl/src/remote/stmt_params.c +++ /dev/null @@ -1,313 +0,0 @@ -/* - * 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. - */ -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include "guc.h" -#include "utils.h" -#include "data_format.h" -#include "stmt_params.h" - -#define MAX_PG_STMT_PARAMS \ - USHRT_MAX /* PostgreSQL limitation of max parameters in the statement \ - */ - -typedef struct StmtParams -{ - FmgrInfo *conv_funcs; - const char **values; - int *formats; - int *lengths; - int num_params; - int num_tuples; - int converted_tuples; - bool ctid; - List *target_attr_nums; - MemoryContext mctx; /* where we allocate param values */ - MemoryContext tmp_ctx; /* used for converting values */ - bool preset; /* idicating if we set values explicitly */ -} StmtParams; - -/* - * Check that chosen num_tuples value does not reach the maximum number of - * prepared statement parameters. - * - * Otherwise recalculate and return max num_tuples value that will - * respect the limit. - */ -int -stmt_params_validate_num_tuples(int num_params, int num_tuples) -{ - Assert(num_params <= MAX_PG_STMT_PARAMS); - - /* Sanity check num_params and avoid division by zero */ - if (num_params > 0 && ((num_params * num_tuples) > MAX_PG_STMT_PARAMS)) - return MAX_PG_STMT_PARAMS / num_params; - - return num_tuples; -} - -/* - * ctid should be set to true if we're going to send it - * num_tuples is used for batching - * mctx memory context where we'll allocate StmtParams with all the values - */ -StmtParams * -stmt_params_create(List *target_attr_nums, bool ctid, TupleDesc tuple_desc, int num_tuples) -{ - StmtParams *params; - ListCell *lc; - Oid typefnoid; - bool isbinary; - int idx = 0; - int tup_cnt; - MemoryContext old; - MemoryContext new; - MemoryContext tmp_ctx; - - new = AllocSetContextCreate(CurrentMemoryContext, - "stmt params mem context", - ALLOCSET_DEFAULT_SIZES); - old = MemoryContextSwitchTo(new); - tmp_ctx = AllocSetContextCreate(new, "stmt params conversion", ALLOCSET_DEFAULT_SIZES); - - params = palloc(sizeof(StmtParams)); - params->num_params = ctid ? list_length(target_attr_nums) + 1 : list_length(target_attr_nums); - Assert(num_tuples > 0); - if (params->num_params * num_tuples > MAX_PG_STMT_PARAMS) - elog(ERROR, "too many parameters in prepared statement. Max is %d", MAX_PG_STMT_PARAMS); - params->conv_funcs = palloc(sizeof(FmgrInfo) * params->num_params); - params->formats = palloc(sizeof(int) * params->num_params * num_tuples); - params->lengths = palloc(sizeof(int) * params->num_params * num_tuples); - params->values = palloc(sizeof(char *) * params->num_params * num_tuples); - params->ctid = ctid; - params->target_attr_nums = target_attr_nums; - params->num_tuples = num_tuples; - params->converted_tuples = 0; - params->mctx = new; - params->tmp_ctx = tmp_ctx; - params->preset = false; - - if (params->ctid) - { - typefnoid = data_format_get_type_output_func(TIDOID, - &isbinary, - !ts_guc_enable_connection_binary_data); - fmgr_info(typefnoid, ¶ms->conv_funcs[idx]); - params->formats[idx] = isbinary ? FORMAT_BINARY : FORMAT_TEXT; - idx++; - } - - foreach (lc, target_attr_nums) - { - int attr_num = lfirst_int(lc); - Form_pg_attribute attr = TupleDescAttr(tuple_desc, AttrNumberGetAttrOffset(attr_num)); - Assert(!attr->attisdropped); - - typefnoid = data_format_get_type_output_func(attr->atttypid, - &isbinary, - !ts_guc_enable_connection_binary_data); - params->formats[idx] = isbinary ? FORMAT_BINARY : FORMAT_TEXT; - - fmgr_info(typefnoid, ¶ms->conv_funcs[idx++]); - } - - Assert(params->num_params == idx); - - for (tup_cnt = 1; tup_cnt < params->num_tuples; tup_cnt++) - memcpy(params->formats + tup_cnt * params->num_params, - params->formats, - sizeof(int) * params->num_params); - - MemoryContextSwitchTo(old); - return params; -} - -StmtParams * -stmt_params_create_from_values(const char **param_values, int n_params) -{ - StmtParams *params; - MemoryContext old; - MemoryContext new; - - if (n_params > MAX_PG_STMT_PARAMS) - elog(ERROR, "too many parameters in prepared statement. Max is %d", MAX_PG_STMT_PARAMS); - - new = AllocSetContextCreate(CurrentMemoryContext, - "stmt params mem context", - ALLOCSET_DEFAULT_SIZES); - old = MemoryContextSwitchTo(new); - - params = palloc(sizeof(StmtParams)); - memset(params, 0, sizeof(StmtParams)); - params->mctx = new; - params->num_params = n_params; - - params->values = param_values; - params->preset = true; - MemoryContextSwitchTo(old); - return params; -} - -static bool -all_values_in_binary_format(int *formats, int num_params) -{ - int i; - - for (i = 0; i < num_params; i++) - if (formats[i] != FORMAT_BINARY) - return false; - return true; -} - -/* - * tupleid is ctid. If ctid was set to true tupleid has to be provided - */ -void -stmt_params_convert_values(StmtParams *params, TupleTableSlot *slot, ItemPointer tupleid) -{ - MemoryContext old; - int idx; - ListCell *lc; - int nest_level = 0; - bool all_binary; - int param_idx = 0; - - Assert(params->num_params > 0); - Assert(params->formats != NULL); - idx = params->converted_tuples * params->num_params; - - Assert(params->converted_tuples < params->num_tuples); - - old = MemoryContextSwitchTo(params->tmp_ctx); - - if (tupleid != NULL) - { - bytea *output_bytes; - Assert(params->ctid); - if (params->formats[idx] == FORMAT_BINARY) - { - output_bytes = - SendFunctionCall(¶ms->conv_funcs[param_idx], PointerGetDatum(tupleid)); - params->values[idx] = VARDATA(output_bytes); - params->lengths[idx] = (int) VARSIZE(output_bytes) - VARHDRSZ; - } - else - params->values[idx] = - OutputFunctionCall(¶ms->conv_funcs[param_idx], PointerGetDatum(tupleid)); - - idx++; - param_idx++; - } - else if (params->ctid) - elog(ERROR, "was configured to use ctid, but tupleid is NULL"); - - all_binary = all_values_in_binary_format(params->formats, params->num_params); - if (!all_binary) - nest_level = set_transmission_modes(); - - foreach (lc, params->target_attr_nums) - { - int attr_num = lfirst_int(lc); - Datum value; - bool isnull; - - value = slot_getattr(slot, attr_num, &isnull); - - if (isnull) - params->values[idx] = NULL; - else if (params->formats[idx] == FORMAT_TEXT) - params->values[idx] = OutputFunctionCall(¶ms->conv_funcs[param_idx], value); - else if (params->formats[idx] == FORMAT_BINARY) - { - bytea *output_bytes = SendFunctionCall(¶ms->conv_funcs[param_idx], value); - params->values[idx] = VARDATA(output_bytes); - params->lengths[idx] = VARSIZE(output_bytes) - VARHDRSZ; - } - else - elog(ERROR, "unexpected parameter format: %d", params->formats[idx]); - idx++; - param_idx++; - } - - params->converted_tuples++; - - if (!all_binary) - reset_transmission_modes(nest_level); - - MemoryContextSwitchTo(old); -} - -void -stmt_params_reset(StmtParams *params) -{ - if (params->tmp_ctx) - MemoryContextReset(params->tmp_ctx); - params->converted_tuples = 0; -} - -/* - * Free params memory context and child context we've used for converting values to binary or text - */ -void -stmt_params_free(StmtParams *params) -{ - MemoryContextDelete(params->mctx); -} - -int * -stmt_params_formats(StmtParams *stmt_params) -{ - if (stmt_params) - return stmt_params->formats; - return NULL; -} - -int * -stmt_params_lengths(StmtParams *stmt_params) -{ - if (stmt_params) - return stmt_params->lengths; - return NULL; -} - -const char *const * -stmt_params_values(StmtParams *stmt_params) -{ - if (stmt_params) - return stmt_params->values; - return NULL; -} - -int -stmt_params_num_params(StmtParams *stmt_params) -{ - if (stmt_params) - return stmt_params->num_params; - return 0; -} - -int -stmt_params_total_values(StmtParams *stmt_params) -{ - if (stmt_params) - return stmt_params->preset ? stmt_params->num_params : - stmt_params->converted_tuples * stmt_params->num_params; - return 0; -} - -int -stmt_params_converted_tuples(StmtParams *stmt_params) -{ - return stmt_params->converted_tuples; -} diff --git a/tsl/src/remote/stmt_params.h b/tsl/src/remote/stmt_params.h deleted file mode 100644 index ac41c1bc6c1..00000000000 --- a/tsl/src/remote/stmt_params.h +++ /dev/null @@ -1,30 +0,0 @@ -/* - * 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. - */ -#pragma once - -#include -#include -#include -#include - -#include "data_format.h" - -typedef struct StmtParams StmtParams; - -extern int stmt_params_validate_num_tuples(int num_params, int num_tuples); -extern StmtParams *stmt_params_create(List *target_attr_nums, bool ctid, TupleDesc tuple_desc, - int num_tuples); -extern StmtParams *stmt_params_create_from_values(const char **param_values, int n_params); -extern void stmt_params_convert_values(StmtParams *params, TupleTableSlot *slot, - ItemPointer tupleid); -extern int *stmt_params_formats(StmtParams *stmt_params); -extern int *stmt_params_lengths(StmtParams *stmt_params); -extern const char *const *stmt_params_values(StmtParams *stmt_params); -extern int stmt_params_num_params(StmtParams *stmt_params); -extern void stmt_params_reset(StmtParams *params); -extern void stmt_params_free(StmtParams *params); -extern int stmt_params_total_values(StmtParams *stmt_params); -extern int stmt_params_converted_tuples(StmtParams *stmt_params); diff --git a/tsl/src/remote/tuplefactory.c b/tsl/src/remote/tuplefactory.c deleted file mode 100644 index c4be1781ded..00000000000 --- a/tsl/src/remote/tuplefactory.c +++ /dev/null @@ -1,431 +0,0 @@ -/* - * This file and its contents are licensed under the Timescale License. - * Please see the included NOTICE for copyright information and - * LICENSE-TIMESCALE for a copy of the license. - */ - -/* - * This file contains source code that was copied and/or modified from - * the PostgreSQL database, which is licensed under the open-source - * PostgreSQL License. Please see the NOTICE at the top level - * directory for a copy of the PostgreSQL License. - */ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include "utils.h" -#include "src/utils.h" -#include "compat/compat.h" -#include "remote/data_format.h" -#include "tuplefactory.h" - -/* - * Identify the attribute where data conversion fails. - */ -typedef struct ConversionLocation -{ - Relation rel; /* foreign table's relcache entry. */ - AttrNumber cur_attno; /* attribute number being processed, or 0 */ - - /* - * In case of foreign join push down, fdw_scan_tlist is used to identify - * the Var node corresponding to the error location and - * ss->ps.state gives access to the RTEs of corresponding relation - * to get the relation name and attribute name. - */ - ScanState *ss; -} ConversionLocation; - -typedef struct TupleFactory -{ - MemoryContext temp_mctx; - TupleDesc tupdesc; - Datum *values; - bool *nulls; - List *retrieved_attrs; - AttConvInMetadata *attconv; - ConversionLocation errpos; - ErrorContextCallback errcallback; - bool per_tuple_mctx_reset; -} TupleFactory; - -/* - * Callback function which is called when error occurs during column value - * conversion. Print names of column and relation. - */ -static void -conversion_error_callback(void *arg) -{ - const char *attname = NULL; - const char *relname = NULL; - bool is_wholerow = false; - ConversionLocation *errpos = (ConversionLocation *) arg; - - if (errpos->rel) - { - /* error occurred in a scan against a foreign table */ - TupleDesc tupdesc = RelationGetDescr(errpos->rel); - Form_pg_attribute attr = TupleDescAttr(tupdesc, errpos->cur_attno - 1); - - if (errpos->cur_attno > 0 && errpos->cur_attno <= tupdesc->natts) - attname = NameStr(attr->attname); - else if (errpos->cur_attno == SelfItemPointerAttributeNumber) - attname = "ctid"; - - relname = RelationGetRelationName(errpos->rel); - } - else - { - /* error occurred in a scan against a foreign join */ - ScanState *ss = errpos->ss; - List *tlist = NIL; - EState *estate = ss->ps.state; - - if (IsA(ss->ps.plan, ForeignScan)) - { - ForeignScan *fsplan = (ForeignScan *) ss->ps.plan; - tlist = fsplan->fdw_scan_tlist; - } - else if (IsA(ss->ps.plan, CustomScan)) - { - CustomScan *csplan = (CustomScan *) ss->ps.plan; - - tlist = csplan->scan.plan.targetlist; - } - - if (tlist == NIL) - { - elog(ERROR, - "unknown scan node type %s in error callback", - ts_get_node_name((Node *) ss->ps.plan)); - } - - TargetEntry *tle = list_nth_node(TargetEntry, tlist, errpos->cur_attno - 1); - - /* - * Target list can have Vars and expressions. For Vars, we can get - * its relation, however for expressions we can't. Thus for - * expressions, just show generic context message. - */ - if (IsA(tle->expr, Var)) - { - RangeTblEntry *rte; - Var *var = (Var *) tle->expr; - - rte = rt_fetch(var->varno, estate->es_range_table); - - if (var->varattno == 0) - is_wholerow = true; - else - attname = get_attname(rte->relid, var->varattno, false); - - relname = get_rel_name(rte->relid); - } - else - errcontext("processing expression at position %d in select list", errpos->cur_attno); - } - - if (relname) - { - if (is_wholerow) - errcontext("whole-row reference to foreign table \"%s\"", relname); - else if (attname) - errcontext("column \"%s\" of foreign table \"%s\"", attname, relname); - } -} - -static TupleFactory * -tuplefactory_create_common(TupleDesc tupdesc, List *retrieved_attrs, bool force_text) -{ - TupleFactory *tf = palloc0(sizeof(TupleFactory)); - - tf->temp_mctx = AllocSetContextCreate(CurrentMemoryContext, - "tuple factory temporary data", - ALLOCSET_DEFAULT_SIZES); - - tf->tupdesc = tupdesc; - tf->retrieved_attrs = retrieved_attrs; - tf->attconv = data_format_create_att_conv_in_metadata(tf->tupdesc, force_text); - tf->values = (Datum *) palloc0(tf->tupdesc->natts * sizeof(Datum)); - tf->nulls = (bool *) palloc(tf->tupdesc->natts * sizeof(bool)); - - /* Initialize to nulls for any columns not present in result */ - memset(tf->nulls, true, tf->tupdesc->natts * sizeof(bool)); - - return tf; -} - -TupleFactory * -tuplefactory_create_for_tupdesc(TupleDesc tupdesc, bool force_text) -{ - List *retrieved_attrs = NIL; - int i; - - for (i = 0; i < tupdesc->natts; i++) - { - if (!TupleDescAttr(tupdesc, i)->attisdropped) - retrieved_attrs = lappend_int(retrieved_attrs, i + 1); - } - - return tuplefactory_create_common(tupdesc, retrieved_attrs, force_text); -} - -static TupleFactory * -tuplefactory_create(Relation rel, ScanState *ss, List *retrieved_attrs) -{ - TupleFactory *tf; - TupleDesc tupdesc; - - Assert(!(rel && ss) && (rel || ss)); - - if (NULL != rel) - tupdesc = RelationGetDescr(rel); - else - tupdesc = ss->ss_ScanTupleSlot->tts_tupleDescriptor; - - tf = - tuplefactory_create_common(tupdesc, retrieved_attrs, !ts_guc_enable_connection_binary_data); - tf->errpos.rel = rel; - tf->errpos.cur_attno = 0; - tf->errpos.ss = ss; - tf->errcallback.callback = conversion_error_callback; - tf->errcallback.arg = (void *) &tf->errpos; - tf->errcallback.previous = error_context_stack; - tf->per_tuple_mctx_reset = true; - - return tf; -} - -TupleFactory * -tuplefactory_create_for_rel(Relation rel, List *retrieved_attrs) -{ - return tuplefactory_create(rel, NULL, retrieved_attrs); -} - -TupleFactory * -tuplefactory_create_for_scan(ScanState *ss, List *retrieved_attrs) -{ - return tuplefactory_create(NULL, ss, retrieved_attrs); -} - -void -tuplefactory_destroy(TupleFactory *tf) -{ - if (tf->temp_mctx) - MemoryContextDelete(tf->temp_mctx); - - pfree(tf); -} - -bool -tuplefactory_is_binary(TupleFactory *tf) -{ - return tf->attconv->binary; -} - -void -tuplefactory_set_per_tuple_mctx_reset(TupleFactory *tf, bool reset) -{ - tf->per_tuple_mctx_reset = reset; -} - -void -tuplefactory_reset_mctx(TupleFactory *tf) -{ - MemoryContextReset(tf->temp_mctx); -} - -int -tuplefactory_get_nattrs(TupleFactory *tf) -{ - return tf->tupdesc->natts; -} - -ItemPointer -tuplefactory_make_virtual_tuple(TupleFactory *tf, PGresult *res, int row, int format, Datum *values, - bool *nulls) -{ - ItemPointer ctid = NULL; - ListCell *lc; - int j; - int PG_USED_FOR_ASSERTS_ONLY ntuples = PQntuples(res); - - Assert(row < ntuples); - - /* Install error callback */ - if (tf->errcallback.callback != NULL) - { - tf->errcallback.previous = error_context_stack; - error_context_stack = &tf->errcallback; - } - - /* - * i indexes columns in the relation, j indexes columns in the PGresult. - */ - j = 0; - foreach (lc, tf->retrieved_attrs) - { - int i = lfirst_int(lc); - char *valstr = NULL; - - const int len = PQgetlength(res, row, j); - /* check via PGgetisnull to see if the attr is null */ - if (PQgetisnull(res, row, j)) - valstr = NULL; - else - { - valstr = PQgetvalue(res, row, j); - } - - /* - * Note that this attno is an index inside fdw_scan_tlist, not inside - * tupdesc. - */ - tf->errpos.cur_attno = j + 1; - - /* - * convert value to internal representation - * - * Note: we ignore system columns other than ctid and oid in result - */ - if (i > 0) - { - /* ordinary column */ - Assert(i <= tf->tupdesc->natts); - nulls[i - 1] = (valstr == NULL); - - if (format == FORMAT_TEXT) - { - Assert(!tf->attconv->binary); - /* Apply the input function even to nulls, to support domains */ - values[i - 1] = InputFunctionCall(&tf->attconv->conv_funcs[i - 1], - valstr, - tf->attconv->ioparams[i - 1], - tf->attconv->typmods[i - 1]); - } - else - { - Assert(tf->attconv->binary); - if (valstr != NULL) - { - StringInfoData si = { .data = valstr, .len = len }; - values[i - 1] = ReceiveFunctionCall(&tf->attconv->conv_funcs[i - 1], - &si, - tf->attconv->ioparams[i - 1], - tf->attconv->typmods[i - 1]); - } - else - values[i - 1] = PointerGetDatum(NULL); - } - } - else if (i == SelfItemPointerAttributeNumber) - { - /* ctid */ - if (valstr != NULL) - { - Datum datum; - if (format == FORMAT_TEXT) - datum = DirectFunctionCall1(tidin, CStringGetDatum(valstr)); - else - { - StringInfoData si = { .data = valstr, .len = len }; - datum = DirectFunctionCall1(tidrecv, PointerGetDatum(&si)); - } - ctid = (ItemPointer) DatumGetPointer(datum); - } - } - tf->errpos.cur_attno = 0; - j++; - } - - /* Uninstall error context callback. */ - if (tf->errcallback.callback != NULL) - error_context_stack = tf->errcallback.previous; - - /* - * Check we got the expected number of columns. Note: j == 0 and - * PQnfields == 1 is expected, since deparse emits a NULL if no columns. - */ - if (j > 0 && j != PQnfields(res)) - elog(ERROR, "remote query result does not match the foreign table"); - - return ctid; -} - -HeapTuple -tuplefactory_make_tuple(TupleFactory *tf, PGresult *res, int row, int format) -{ - /* - * Do the following work in a temp context that we reset after each tuple. - * This cleans up not only the data we have direct access to, but any - * cruft the I/O functions might leak. - */ - MemoryContext oldcontext = MemoryContextSwitchTo(tf->temp_mctx); - - ItemPointer ctid = tuplefactory_make_virtual_tuple(tf, res, row, format, tf->values, tf->nulls); - - /* - * Build the result tuple in caller's memory context. - */ - MemoryContextSwitchTo(oldcontext); - - HeapTuple tuple = heap_form_tuple(tf->tupdesc, tf->values, tf->nulls); - - /* - * If we have a CTID to return, install it in both t_self and t_ctid. - * t_self is the normal place, but if the tuple is converted to a - * composite Datum, t_self will be lost; setting t_ctid allows CTID to be - * preserved during EvalPlanQual re-evaluations (see ROW_MARK_COPY code). - */ - if (ctid) - tuple->t_self = tuple->t_data->t_ctid = *ctid; - - /* - * Stomp on the xmin, xmax, and cmin fields from the tuple created by - * heap_form_tuple. heap_form_tuple actually creates the tuple with - * DatumTupleFields, not HeapTupleFields, but the executor expects - * HeapTupleFields and will happily extract system columns on that - * assumption. If we don't do this then, for example, the tuple length - * ends up in the xmin field, which isn't what we want. - */ - HeapTupleHeaderSetXmax(tuple->t_data, InvalidTransactionId); - HeapTupleHeaderSetXmin(tuple->t_data, InvalidTransactionId); - HeapTupleHeaderSetCmin(tuple->t_data, InvalidTransactionId); - - /* Clean up */ - if (tf->per_tuple_mctx_reset) - tuplefactory_reset_mctx(tf); - - return tuple; -} - -struct AttConvInMetadata * -tuplefactory_get_attconv(TupleFactory *tf) -{ - return tf->attconv; -} - -TupleDesc -tuplefactory_get_tupdesc(TupleFactory *tf) -{ - return tf->tupdesc; -} - -List * -tuplefactory_get_retrieved_attrs(TupleFactory *tf) -{ - return tf->retrieved_attrs; -} diff --git a/tsl/src/remote/tuplefactory.h b/tsl/src/remote/tuplefactory.h deleted file mode 100644 index cb6cb6e5f9f..00000000000 --- a/tsl/src/remote/tuplefactory.h +++ /dev/null @@ -1,33 +0,0 @@ -/* - * 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. - */ -#pragma once - -#include -#include -#include -#include -#include -#include - -#include "data_format.h" - -typedef struct TupleFactory TupleFactory; -typedef struct TupleDescData *TupleDesc; - -extern TupleFactory *tuplefactory_create_for_tupdesc(TupleDesc tupdesc, bool force_text); -extern TupleFactory *tuplefactory_create_for_rel(Relation rel, List *retrieved_attrs); -extern TupleFactory *tuplefactory_create_for_scan(ScanState *ss, List *retrieved_attrs); -extern void tuplefactory_destroy(TupleFactory *tf); -extern HeapTuple tuplefactory_make_tuple(TupleFactory *tf, PGresult *res, int row, int format); -extern ItemPointer tuplefactory_make_virtual_tuple(TupleFactory *tf, PGresult *res, int row, - int format, Datum *values, bool *nulls); -extern bool tuplefactory_is_binary(TupleFactory *tf); -extern void tuplefactory_set_per_tuple_mctx_reset(TupleFactory *tf, bool reset); -extern void tuplefactory_reset_mctx(TupleFactory *tf); -extern struct AttConvInMetadata *tuplefactory_get_attconv(TupleFactory *tf); -extern TupleDesc tuplefactory_get_tupdesc(TupleFactory *tf); -extern List *tuplefactory_get_retrieved_attrs(TupleFactory *tf); -extern int tuplefactory_get_nattrs(TupleFactory *tf); diff --git a/tsl/src/remote/txn.c b/tsl/src/remote/txn.c deleted file mode 100644 index 61f6ed996fc..00000000000 --- a/tsl/src/remote/txn.c +++ /dev/null @@ -1,819 +0,0 @@ -/* - * 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. - */ -#include "libpq-fe.h" -#include -#include -#include -#include -#include -#include -#include -#include - -#include "remote/async.h" -#include "remote/txn_store.h" -#include "txn.h" -#include "connection.h" -#include "scanner.h" -#include "ts_catalog/catalog.h" -#include "txn_id.h" - -/* This seemingly long timeout matches what postgres_fdw uses. */ -#define DEFAULT_EXEC_CLEANUP_TIMEOUT_MS 30000 - -/* - * This RemoteTxn represents one remote end in a distributed txn. - * Thus, a distributed txn is made up of a collection remote txn. - * Each remote txn corresponds to one remote connection and there - * is a unique remote connection per TSConnectionId used in the - * distributed txn. Because of this uniqueness property, - * the connection id appears first in the object, to allow - * it to be a hash key. - * - * The "conn" pointer can be NULL if we don't currently have a live connection. - * When we do have a connection, xact_depth tracks the current depth of - * transactions and subtransactions open on the remote side. We need to issue - * commands at the same nesting depth on the remote as we're executing at - * ourselves, so that rolling back a subtransaction will kill the right - * queries and not the wrong ones. - */ - -typedef struct RemoteTxn -{ - TSConnectionId id; /* hash key (must be first) */ - TSConnection *conn; /* connection to data node, or NULL */ - /* Remaining fields are invalid when conn is NULL: */ - bool have_prep_stmt; /* have we prepared any stmts in this xact? */ - bool have_subtxn_error; /* have any subxacts aborted in this xact? */ - RemoteTxnId *remote_txn_id; -} RemoteTxn; - -/* - * Start remote transaction or subtransaction, if it hasn't been - * already started (e.g. by a previous command in the same txn). - * - * We always use at least REPEATABLE READ in the remote session. - * This is important even for cases where we use the a single connection to - * a data node. This is because a single command from the access node may cause - * multiple remote commands to be executed (e.g. a join of two tables on one remote - * node might not be pushed down and instead two different queries are sent - * to the remote node, one for each table in the join). Since in READ - * COMMITTED the snapshot is refreshed on each command, the semantics are off - * when multiple commands are meant to be part of the same one. - * - * This isn't great but we have no alternative unless we ensure that each access - * node command always translates to one data node query or if we had some other way to - * control which remote queries share a snapshot or when a snapshot is refreshed. - * - * NOTE: this does not guarantee any kind of snapshot isolation to different connections - * to the same data node. That only happens if we use multiple connection ids to the same data node - * in one access node transaction. Thus, such connections that use different users will potentially - * see inconsistent results. To solve this problem of inconsistent results, we could export the - * snapshot of the first connection to a remote node using pg_export_snapshot() and then use that - * using SET TRANSACTION SNAPSHOT xxxx across all other connections to that node during the - * transaction. However, given that we currently don't have snapshot isolation across different - * nodes, we don't want to commit to the overhead of exporting snapshots at this time. - */ -void -remote_txn_begin(RemoteTxn *entry, int curlevel) -{ - int xact_depth = remote_connection_xact_depth_get(entry->conn); - - /* Start main transaction if we haven't yet */ - if (xact_depth == 0) - { - StringInfoData sql; - char *xactReadOnly; - - Assert(remote_connection_get_status(entry->conn) == CONN_IDLE); - elog(DEBUG3, "starting remote transaction on connection %p", entry->conn); - - initStringInfo(&sql); - appendStringInfo(&sql, "%s", "START TRANSACTION ISOLATION LEVEL"); - if (IsolationIsSerializable()) - appendStringInfo(&sql, "%s", " SERIALIZABLE"); - else - appendStringInfo(&sql, "%s", " REPEATABLE READ"); - - /* - * Windows MSVC builds have linking issues for GUC variables from postgres for - * use inside this extension. So we use GetConfigOptionByName - */ - xactReadOnly = GetConfigOptionByName("transaction_read_only", NULL, false); - - /* - * If we are initiating connection from a standby (of an AN for example), - * then the remote connection transaction needs to be also set up as a - * READ ONLY one. This will catch any commands that are sent from the - * read only AN to datanodes but which could have potential read-write - * side effects on data nodes. - * - * Note that when the STANDBY gets promoted then the ongoing transaction - * will remain READ ONLY till its completion. New transactions will be - * suitably READ WRITE. This is a slight change in behavior as compared to - * regular Postgres, but promotion is not a routine activity, so it should - * be acceptable and typically users would be reconnecting to the new - * promoted AN anyways. - * - * Note that the below will also handle the case when primary AN has a - * transaction which does an explicit "BEGIN TRANSACTION READ ONLY;". The - * treatment is the same, mark the remote DN transaction as READ ONLY - */ - if (strncmp(xactReadOnly, "on", sizeof("on")) == 0) - appendStringInfo(&sql, "%s", " READ ONLY"); - - remote_connection_xact_transition_begin(entry->conn); - remote_connection_cmd_ok(entry->conn, sql.data); - remote_connection_xact_transition_end(entry->conn); - xact_depth = remote_connection_xact_depth_inc(entry->conn); - pfree(sql.data); - } - /* If the connection is in COPY mode, then exit out of it */ - else if (remote_connection_get_status(entry->conn) == CONN_COPY_IN) - { - TSConnectionError err; - - if (!remote_connection_end_copy(entry->conn, &err)) - remote_connection_error_elog(&err, ERROR); - } - - /* - * If we're in a subtransaction, stack up savepoints to match our level. - * This ensures we can rollback just the desired effects when a - * subtransaction aborts. - */ - while (xact_depth < curlevel) - { - remote_connection_xact_transition_begin(entry->conn); - remote_connection_cmdf_ok(entry->conn, "SAVEPOINT s%d", xact_depth + 1); - remote_connection_xact_transition_end(entry->conn); - xact_depth = remote_connection_xact_depth_inc(entry->conn); - } -} - -/* - * Check if the access node transaction which is driving the 2PC on the datanodes is - * still in progress. - */ -bool -remote_txn_is_still_in_progress_on_access_node(TransactionId access_node_xid) -{ - if (TransactionIdIsCurrentTransactionId(access_node_xid)) - elog(ERROR, "checking if a commit is still in progress on same txn"); - - return TransactionIdIsInProgress(access_node_xid); -} - -size_t -remote_txn_size() -{ - return sizeof(RemoteTxn); -} - -void -remote_txn_init(RemoteTxn *entry, TSConnection *conn) -{ - Assert(NULL != conn); - Assert(remote_connection_xact_depth_get(conn) == 0); - - /* Reset all transient state fields, to be sure all are clean */ - entry->have_prep_stmt = false; - entry->have_subtxn_error = false; - entry->remote_txn_id = NULL; - - /* Now try to make the connection */ - /* in connection */ - entry->conn = conn; - - elog(DEBUG3, - "new connection %p for data node \"%s\" (server " - "oid %u, userid %u)", - entry->conn, - remote_connection_node_name(conn), - entry->id.server_id, - entry->id.user_id); -} - -RemoteTxn * -remote_txn_begin_on_connection(TSConnection *conn) -{ - RemoteTxn *txn = palloc0(sizeof(RemoteTxn)); - - remote_txn_init(txn, conn); - remote_txn_begin(txn, GetCurrentTransactionNestLevel()); - - return txn; -} - -void -remote_txn_set_will_prep_statement(RemoteTxn *entry, RemoteTxnPrepStmtOption prep_stmt_option) -{ - bool will_prep_stmt = (prep_stmt_option == REMOTE_TXN_USE_PREP_STMT); - - entry->have_prep_stmt |= will_prep_stmt; -} - -TSConnection * -remote_txn_get_connection(RemoteTxn *txn) -{ - return txn->conn; -} - -TSConnectionId -remote_txn_get_connection_id(RemoteTxn *txn) -{ - return txn->id; -} - -void -remote_txn_report_prepare_transaction_result(RemoteTxn *txn, bool success) -{ - if (!success) - txn->remote_txn_id = NULL; -} - -/* - * This function submits commands to remote nodes during (sub)abort processing. - * Because remote nodes can be in a weird state and at the same time errors should - * not be thrown here, the processing here is a bit different. - * - * We submit a query during and wait up to 30 seconds for the result. All errors - * are reported as WARNINGS into the log. - * - * If the query is executed without error, the return value is true. - * If the query can't be sent, errors out, or times out, the return value is false. - */ -static bool -exec_cleanup_command(TSConnection *conn, const char *query) -{ - TimestampTz end_time; - AsyncRequest *req; - AsyncResponseResult *result; - AsyncResponse *response = NULL; - PGresult *pg_result; - bool success = false; - - /* - * If it takes too long to execute a cleanup query, assume the connection - * is dead. It's fairly likely that this is why we aborted in the first - * place (e.g. statement timeout, user cancel), so the timeout shouldn't - * be too long. - */ - end_time = TimestampTzPlusMilliseconds(GetCurrentTimestamp(), DEFAULT_EXEC_CLEANUP_TIMEOUT_MS); - - /* - * Send the query. Since we don't use non-blocking mode, this also can - * block. But its risk is relatively small, so we ignore that for now. - */ - req = async_request_send_with_error(conn, query, WARNING); - - if (req == NULL) - return false; - - /* Wait until the command completes or there is a timeout or error */ - response = async_request_cleanup_result(req, end_time); - Assert(response != NULL); - - switch (async_response_get_type(response)) - { - case RESPONSE_TIMEOUT: - elog(DEBUG3, "abort processing: timeout executing %s", query); - success = false; - break; - case RESPONSE_COMMUNICATION_ERROR: - elog(DEBUG3, "abort processing: communication error executing %s", query); - success = false; - break; - case RESPONSE_ERROR: - elog(DEBUG3, "abort processing: error while executing %s", query); - success = false; - break; - case RESPONSE_RESULT: - result = (AsyncResponseResult *) response; - pg_result = async_response_result_get_pg_result(result); - if (PQresultStatus(pg_result) != PGRES_COMMAND_OK) - { - elog(DEBUG3, "abort processing: error in result executing %s", query); - success = false; - } - else - success = true; - break; - case RESPONSE_ROW: - elog(DEBUG3, - "abort processing: unexpected response type %d while executing %s", - async_response_get_type(response), - query); - success = false; - break; - } - - if (!success) - async_response_report_error(response, WARNING); - - async_response_close(response); - - return success; -} - -#ifdef DEBUG -/* Prepared statements can leak if the were created during a subtxn - * and the subtxn rolled back before the prepared stmt was deallocated. - * This function checks for such leaks inside of tests (thus only compiled - * in DEBUG mode). It can be quite expensive so not run under normal operations. - */ -void -remote_txn_check_for_leaked_prepared_statements(RemoteTxn *entry) -{ - PGresult *res; - char *count_string; - ExecStatusType status; - - if (PQTRANS_IDLE != PQtransactionStatus(remote_connection_get_pg_conn(entry->conn))) - return; - - res = remote_connection_exec(entry->conn, "SELECT count(*) FROM pg_prepared_statements"); - - status = PQresultStatus(res); - - switch (status) - { - case PGRES_TUPLES_OK: - if (PQntuples(res) == 1 && PQnfields(res) == 1) - { - count_string = PQgetvalue(res, 0, 0); - if (strcmp("0", count_string) != 0) - elog(WARNING, "leak check: connection leaked prepared statement"); - } - else - elog(ERROR, "leak check: unexpected number of rows or columns returned"); - break; - case PGRES_FATAL_ERROR: - case PGRES_NONFATAL_ERROR: - elog(WARNING, "leak check: ERROR [\"%s\"]", PQresultErrorMessage(res)); - break; - default: - elog(WARNING, "leak check: unexpected result state %u", status); - break; - } - - remote_result_close(res); -} -#endif - -bool -remote_txn_abort(RemoteTxn *entry) -{ - const char *abort_sql; - bool success = true; - - if (entry->remote_txn_id == NULL) - { - /* Rollback a regular (non two-phase commit) transaction */ - abort_sql = "ROLLBACK TRANSACTION"; - } - else - { - /* Rollback a transaction prepared for two-phase commit (PREPARE - * TRANSACTION) */ - abort_sql = remote_txn_id_rollback_prepared_sql(entry->remote_txn_id); - } - - entry->remote_txn_id = NULL; - - Assert(entry->conn != NULL); - Assert(remote_connection_xact_depth_get(entry->conn) > 0); - - elog(DEBUG3, "aborting remote transaction on connection %p", entry->conn); - - /* Already in bad state */ - if (remote_connection_xact_is_transitioning(entry->conn)) - return false; - else if (in_error_recursion_trouble() || - PQstatus(remote_connection_get_pg_conn(entry->conn)) == CONNECTION_BAD) - { - /* - * Don't try to recover the connection if we're already in error - * recursion trouble or the connection is bad. Instead, mark it as a - * failed transition. This is a really bad case and so controlled - * cleanup cannot happen here. The calling function will instead break - * this ongoing connection and so no cleanup is necessary. - */ - remote_connection_xact_transition_begin(entry->conn); - return false; - } - - /* Mark the connection as transitioning to new transaction state */ - remote_connection_xact_transition_begin(entry->conn); - - /* - * Check if a command has been submitted to the data node by using an - * asynchronous execution function and the command had not yet completed. - * If so, request cancellation of the command. - */ - if (PQtransactionStatus(remote_connection_get_pg_conn(entry->conn)) == PQTRANS_ACTIVE) - success = remote_connection_cancel_query(entry->conn); - - if (success) - { - /* At this point any on going queries should have completed */ - remote_connection_set_status(entry->conn, CONN_IDLE); - success = exec_cleanup_command(entry->conn, abort_sql); - } - - /* - * Assume we might may have not deallocated all the prepared statements we - * created because the deallocation would have happened after the abort. - * - * prepared stmts are per session not per transaction. But we don't want - * prepared_stmts to survive transactions in our use case. - */ - if (success && entry->have_prep_stmt) - success = exec_cleanup_command(entry->conn, "DEALLOCATE ALL"); - - if (success) - { - entry->have_prep_stmt = false; - entry->have_subtxn_error = false; - - /* Everything succeeded, so we have finished transitioning */ - remote_connection_xact_transition_end(entry->conn); - } - - return success; -} - -/* Check if there is ongoing transaction on the remote node */ -bool -remote_txn_is_ongoing(RemoteTxn *entry) -{ - Assert(remote_connection_xact_depth_get(entry->conn) >= 0); - return remote_connection_xact_depth_get(entry->conn) > 0; -} - -/* - * If there were any errors in subtransactions, and we made prepared - * statements, those prepared statements may not have been cleared - * because of the subtxn error. Thus, do a DEALLOCATE ALL to make sure - * we get rid of all prepared statements. - * - * This is annoying and not terribly bulletproof, but it's - * probably not worth trying harder. - */ -void -remote_txn_deallocate_prepared_stmts_if_needed(RemoteTxn *entry) -{ - Assert(entry->conn != NULL && remote_connection_xact_depth_get(entry->conn) > 0); - - if (entry->have_prep_stmt && entry->have_subtxn_error) - { - AsyncRequestSet *set = async_request_set_create(); - AsyncResponse *response; - - async_request_set_add(set, async_request_send(entry->conn, "DEALLOCATE ALL")); - response = async_request_set_wait_any_response(set); - async_response_report_error_or_close(response, WARNING); - response = async_request_set_wait_any_response(set); - Assert(response == NULL); - } - entry->have_prep_stmt = false; - entry->have_subtxn_error = false; -} - -/* - * Ensure state changes are marked successful when a remote transaction - * completes asynchronously and successfully. - * - * We do this in a callback which is guaranteed to be called when a reponse is - * received or a timeout occurs. - * - * There is no decision on whether to fail or not in this callback; this is - * only to guarantee that we're always updating the internal connection - * state. Someone still has to handle the responses elsewehere. - */ -static bool -on_remote_txn_response(AsyncRequest *req, AsyncResponse *rsp) -{ - TSConnection *conn = async_request_get_connection(req); - bool success = false; - - if (async_response_get_type(rsp) == RESPONSE_RESULT) - { - AsyncResponseResult *res = (AsyncResponseResult *) rsp; - PGresult *pgres = async_response_result_get_pg_result(res); - - if (PQresultStatus(pgres) == PGRES_COMMAND_OK) - { - remote_connection_xact_transition_end(conn); - success = true; - } - } - - return success; -} - -static void -on_commit_or_commit_prepared_response(AsyncRequest *req, AsyncResponse *rsp, void *data) -{ - on_remote_txn_response(req, rsp); -} - -AsyncRequest * -remote_txn_async_send_commit(RemoteTxn *entry) -{ - AsyncRequest *req; - - Assert(entry->conn != NULL); - Assert(remote_connection_xact_depth_get(entry->conn) > 0); - - elog(DEBUG3, "committing remote transaction on connection %p", entry->conn); - - remote_connection_xact_transition_begin(entry->conn); - req = async_request_send(entry->conn, "COMMIT TRANSACTION"); - async_request_set_response_callback(req, on_commit_or_commit_prepared_response, entry); - - return req; -} - -void -remote_txn_write_persistent_record(RemoteTxn *entry) -{ - entry->remote_txn_id = remote_txn_persistent_record_write(entry->id); -} - -static void -on_prepare_transaction_response(AsyncRequest *req, AsyncResponse *rsp, void *data) -{ - bool success = on_remote_txn_response(req, rsp); - - if (!success) - { - RemoteTxn *txn = data; - - /* If the prepare is not successful, reset the remote transaction ID - * to indicate we need to do a rollback */ - txn->remote_txn_id = NULL; - } -} - -AsyncRequest * -remote_txn_async_send_prepare_transaction(RemoteTxn *entry) -{ - AsyncRequest *req; - - Assert(entry->conn != NULL); - Assert(remote_connection_xact_depth_get(entry->conn) > 0); - Assert(entry->remote_txn_id != NULL); - - elog(DEBUG3, - "2pc: preparing remote transaction on connection %p: %s", - entry->conn, - remote_txn_id_out(entry->remote_txn_id)); - - remote_connection_xact_transition_begin(entry->conn); - req = async_request_send(entry->conn, - remote_txn_id_prepare_transaction_sql(entry->remote_txn_id)); - async_request_set_response_callback(req, on_prepare_transaction_response, entry); - - return req; -} - -AsyncRequest * -remote_txn_async_send_commit_prepared(RemoteTxn *entry) -{ - AsyncRequest *req; - - Assert(entry->conn != NULL); - Assert(entry->remote_txn_id != NULL); - - elog(DEBUG3, - "2pc: commiting remote transaction on connection %p: '%s'", - entry->conn, - remote_txn_id_out(entry->remote_txn_id)); - - remote_connection_xact_transition_begin(entry->conn); - - req = async_request_send_with_error(entry->conn, - remote_txn_id_commit_prepared_sql(entry->remote_txn_id), - WARNING); - async_request_set_response_callback(req, on_commit_or_commit_prepared_response, entry); - - return req; -} - -/* - * Rollback a subtransaction to a given savepoint. - */ -bool -remote_txn_sub_txn_abort(RemoteTxn *entry, int curlevel) -{ - PGconn *pg_conn = remote_connection_get_pg_conn(entry->conn); - bool success = false; - - Assert(remote_connection_xact_depth_get(entry->conn) == curlevel); - Assert(remote_connection_xact_depth_get(entry->conn) > 1); - - if (in_error_recursion_trouble() && remote_connection_xact_is_transitioning(entry->conn)) - remote_connection_xact_transition_begin(entry->conn); - - if (!remote_connection_xact_is_transitioning(entry->conn)) - { - StringInfoData sql; - - initStringInfo(&sql); - entry->have_subtxn_error = true; - remote_connection_xact_transition_begin(entry->conn); - - /* - * If a command has been submitted to the data node by using an - * asynchronous execution function, the command might not have yet - * completed. Check to see if a command is still being processed by the - * data node, and if so, request cancellation of the command. - */ - if (PQtransactionStatus(pg_conn) == PQTRANS_ACTIVE && - !remote_connection_cancel_query(entry->conn)) - success = false; - else - { - /* Rollback all remote subtransactions during abort */ - appendStringInfo(&sql, "ROLLBACK TO SAVEPOINT s%d", curlevel); - success = exec_cleanup_command(entry->conn, sql.data); - - if (success) - { - resetStringInfo(&sql); - appendStringInfo(&sql, "RELEASE SAVEPOINT s%d", curlevel); - success = exec_cleanup_command(entry->conn, sql.data); - } - } - - if (success) - remote_connection_xact_transition_end(entry->conn); - } - - Assert(remote_connection_xact_depth_get(entry->conn) > 0); - - return success; -} - -bool -remote_txn_is_at_sub_txn_level(RemoteTxn *entry, int curlevel) -{ - int xact_depth; - - /* - * We only care about connections with open remote subtransactions of the - * current level. - */ - Assert(entry->conn != NULL); - - xact_depth = remote_connection_xact_depth_get(entry->conn); - - if (xact_depth < curlevel) - return false; - - if (xact_depth > curlevel) - elog(ERROR, "missed cleaning up remote subtransaction at level %d", xact_depth); - - Assert(xact_depth == curlevel); - - return true; -} - -void -remote_txn_sub_txn_pre_commit(RemoteTxn *entry, int curlevel) -{ - Assert(remote_connection_xact_depth_get(entry->conn) == curlevel); - Assert(remote_connection_xact_depth_get(entry->conn) > 0); - Assert(!remote_connection_xact_is_transitioning(entry->conn)); - - remote_connection_xact_transition_begin(entry->conn); - remote_connection_cmdf_ok(entry->conn, "RELEASE SAVEPOINT s%d", curlevel); - remote_connection_xact_transition_end(entry->conn); -} - -/* - * Functions for storing a persistent transaction records for two-phase - * commit. - */ -static int -persistent_record_pkey_scan(const RemoteTxnId *id, tuple_found_func tuple_found, LOCKMODE lock_mode) -{ - Catalog *catalog = ts_catalog_get(); - ScanKeyData scankey[1]; - ScannerCtx scanctx = { - .table = catalog->tables[REMOTE_TXN].id, - .index = catalog_get_index(catalog, REMOTE_TXN, REMOTE_TXN_PKEY_IDX), - .nkeys = 1, - .scankey = scankey, - .tuple_found = tuple_found, - .lockmode = lock_mode, - .limit = 1, - .scandirection = ForwardScanDirection, - }; - - ScanKeyInit(&scankey[0], - Anum_remote_txn_pkey_idx_remote_transaction_id, - BTEqualStrategyNumber, - F_TEXTEQ, - CStringGetTextDatum(remote_txn_id_out(id))); - - return ts_scanner_scan(&scanctx); -} - -bool -remote_txn_persistent_record_exists(const RemoteTxnId *parsed) -{ - return persistent_record_pkey_scan(parsed, NULL, AccessShareLock) > 0; -} - -static ScanTupleResult -persistent_record_tuple_delete(TupleInfo *ti, void *data) -{ - ts_catalog_delete_tid(ti->scanrel, ts_scanner_get_tuple_tid(ti)); - return SCAN_CONTINUE; -} - -/* If gid is NULL, then delete all entries belonging to the provided datanode. */ -int -remote_txn_persistent_record_delete_for_data_node(Oid foreign_server_oid, const char *gid) -{ - Catalog *catalog = ts_catalog_get(); - ScanKeyData scankey[1]; - ScannerCtx scanctx; - int scanidx; - ForeignServer *server = GetForeignServer(foreign_server_oid); - - if (gid == NULL) - { - ScanKeyInit(&scankey[0], - Anum_remote_txn_data_node_name_idx_data_node_name, - BTEqualStrategyNumber, - F_NAMEEQ, - CStringGetDatum(server->servername)); - scanidx = REMOTE_TXN_DATA_NODE_NAME_IDX; - } - else - { - ScanKeyInit(&scankey[0], - Anum_remote_txn_pkey_idx_remote_transaction_id, - BTEqualStrategyNumber, - F_TEXTEQ, - CStringGetTextDatum(gid)); - scanidx = REMOTE_TXN_PKEY_IDX; - } - - scanctx = (ScannerCtx){ - .table = catalog->tables[REMOTE_TXN].id, - .index = catalog_get_index(catalog, REMOTE_TXN, scanidx), - .nkeys = 1, - .scankey = scankey, - .tuple_found = persistent_record_tuple_delete, - .lockmode = RowExclusiveLock, - .snapshot = GetTransactionSnapshot(), - .scandirection = ForwardScanDirection, - }; - - return ts_scanner_scan(&scanctx); -} - -static void -persistent_record_insert_relation(Relation rel, RemoteTxnId *id) -{ - TupleDesc desc = RelationGetDescr(rel); - Datum values[Natts_remote_txn]; - bool nulls[Natts_remote_txn] = { false }; - CatalogSecurityContext sec_ctx; - ForeignServer *server = GetForeignServer(id->id.server_id); - - values[AttrNumberGetAttrOffset(Anum_remote_txn_data_node_name)] = - DirectFunctionCall1(namein, CStringGetDatum(server->servername)); - values[AttrNumberGetAttrOffset(Anum_remote_txn_remote_transaction_id)] = - CStringGetTextDatum(remote_txn_id_out(id)); - - ts_catalog_database_info_become_owner(ts_catalog_database_info_get(), &sec_ctx); - ts_catalog_insert_values(rel, desc, values, nulls); - ts_catalog_restore_user(&sec_ctx); -} - -/* - * Add a commit record to catalog. - */ -RemoteTxnId * -remote_txn_persistent_record_write(TSConnectionId cid) -{ - RemoteTxnId *id = remote_txn_id_create(GetTopTransactionId(), cid); - Catalog *catalog = ts_catalog_get(); - Relation rel; - - rel = table_open(catalog->tables[REMOTE_TXN].id, RowExclusiveLock); - persistent_record_insert_relation(rel, id); - - /* Keep the table lock until transaction completes in order to - * synchronize with distributed restore point creation */ - table_close(rel, NoLock); - return id; -} diff --git a/tsl/src/remote/txn.h b/tsl/src/remote/txn.h deleted file mode 100644 index 3221a76b017..00000000000 --- a/tsl/src/remote/txn.h +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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. - */ -#pragma once - -#include -#include - -#include "cache.h" -#include "connection.h" -#include "txn_id.h" - -typedef struct RemoteTxn RemoteTxn; - -typedef enum -{ - REMOTE_TXN_NO_PREP_STMT = 0, - REMOTE_TXN_USE_PREP_STMT, -} RemoteTxnPrepStmtOption; - -/* actions */ -extern void remote_txn_init(RemoteTxn *entry, TSConnection *conn); -extern RemoteTxn *remote_txn_begin_on_connection(TSConnection *conn); -extern void remote_txn_begin(RemoteTxn *entry, int curlevel); -extern bool remote_txn_abort(RemoteTxn *entry); -extern void remote_txn_write_persistent_record(RemoteTxn *entry); -extern void remote_txn_deallocate_prepared_stmts_if_needed(RemoteTxn *entry); -extern bool remote_txn_sub_txn_abort(RemoteTxn *entry, int curlevel); -extern void remote_txn_sub_txn_pre_commit(RemoteTxn *entry, int curlevel); - -/* accessors/info */ -extern void remote_txn_set_will_prep_statement(RemoteTxn *entry, - RemoteTxnPrepStmtOption prep_stmt_option); -extern TSConnection *remote_txn_get_connection(RemoteTxn *txn); -extern TSConnectionId remote_txn_get_connection_id(RemoteTxn *txn); -extern bool remote_txn_is_still_in_progress_on_access_node(TransactionId access_node_xid); -extern size_t remote_txn_size(void); -extern bool remote_txn_is_at_sub_txn_level(RemoteTxn *entry, int curlevel); -extern bool remote_txn_is_ongoing(RemoteTxn *entry); - -/* Messages/communication */ -extern AsyncRequest *remote_txn_async_send_commit(RemoteTxn *entry); -extern AsyncRequest *remote_txn_async_send_prepare_transaction(RemoteTxn *entry); -extern AsyncRequest *remote_txn_async_send_commit_prepared(RemoteTxn *entry); -extern void remote_txn_report_prepare_transaction_result(RemoteTxn *txn, bool success); - -/* Persitent record */ -extern RemoteTxnId *remote_txn_persistent_record_write(TSConnectionId id); -extern bool remote_txn_persistent_record_exists(const RemoteTxnId *parsed); -extern int remote_txn_persistent_record_delete_for_data_node(Oid foreign_server_oid, - const char *gid); - -#ifdef DEBUG -/* Debugging functions used in testing */ -extern void remote_txn_check_for_leaked_prepared_statements(RemoteTxn *entry); -#endif diff --git a/tsl/src/remote/txn_id.c b/tsl/src/remote/txn_id.c deleted file mode 100644 index 37499376bef..00000000000 --- a/tsl/src/remote/txn_id.c +++ /dev/null @@ -1,136 +0,0 @@ -/* - * 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. - */ -#include -#include -#include -#include -#include -#include -#include -#include - -#include "connection.h" -#include "txn_id.h" - -#define GID_SEP "-" - -/* The separator is part of the GID prefix */ -#define GID_PREFIX "ts-" -/* This is the maximum size of the literal accepted by PREPARE TRANSACTION, etc. */ -#define GID_MAX_SIZE 200 - -#define REMOTE_TXN_ID_VERSION ((uint8) 1) - -/* current_pattern: ts-version-xid-server_id-user_id */ -#define FMT_PATTERN GID_PREFIX "%hhu" GID_SEP "%u" GID_SEP "%u" GID_SEP "%u" - -static char * -remote_txn_id_get_sql(const char *command, RemoteTxnId *id) -{ - StringInfoData sql; - - initStringInfo(&sql); - appendStringInfoString(&sql, command); - appendStringInfoSpaces(&sql, 1); - appendStringInfoString(&sql, quote_literal_cstr(remote_txn_id_out(id))); - return sql.data; -} - -const char * -remote_txn_id_prepare_transaction_sql(RemoteTxnId *id) -{ - return remote_txn_id_get_sql("PREPARE TRANSACTION", id); -} - -const char * -remote_txn_id_commit_prepared_sql(RemoteTxnId *id) -{ - return remote_txn_id_get_sql("COMMIT PREPARED", id); -} - -const char * -remote_txn_id_rollback_prepared_sql(RemoteTxnId *id) -{ - return remote_txn_id_get_sql("ROLLBACK PREPARED", id); -} - -bool -remote_txn_id_matches_prepared_txn(const char *id_string) -{ - if (strncmp(GID_PREFIX, id_string, strlen(GID_PREFIX)) == 0) - return true; - return false; -} - -RemoteTxnId * -remote_txn_id_in(const char *id_string) -{ - RemoteTxnId *id = palloc0(sizeof(RemoteTxnId)); - char dummy; - - if (sscanf(id_string, - FMT_PATTERN "%c", - &id->version, - &id->xid, - &id->id.server_id, - &id->id.user_id, - &dummy) != 4) - ereport(ERROR, - (errcode(ERRCODE_INVALID_TEXT_REPRESENTATION), - errmsg("invalid input syntax for remote transaction ID: '%s'", id_string))); - - if (id->version != REMOTE_TXN_ID_VERSION) - elog(ERROR, "invalid version for remote transaction ID: %hhu", id->version); - - return id; -} - -Datum -remote_txn_id_in_pg(PG_FUNCTION_ARGS) -{ - const char *id_string = PG_GETARG_CSTRING(0); - - PG_RETURN_POINTER(remote_txn_id_in(id_string)); -} - -const char * -remote_txn_id_out(const RemoteTxnId *id) -{ - char *out = palloc0(sizeof(char) * GID_MAX_SIZE); - int written; - - written = snprintf(out, - GID_MAX_SIZE, - FMT_PATTERN, - REMOTE_TXN_ID_VERSION, - id->xid, - id->id.server_id, - id->id.user_id); - - if (written < 0 || written >= GID_MAX_SIZE) - elog(ERROR, "unexpected length when generating a 2pc transaction name: %d", written); - - return out; -} - -Datum -remote_txn_id_out_pg(PG_FUNCTION_ARGS) -{ - RemoteTxnId *id = (RemoteTxnId *) PG_GETARG_POINTER(0); - - PG_RETURN_POINTER(remote_txn_id_out(id)); -} - -RemoteTxnId * -remote_txn_id_create(TransactionId xid, TSConnectionId cid) -{ - RemoteTxnId *id = palloc0(sizeof(RemoteTxnId)); - - id->xid = xid; - id->id = cid; - - return id; -} diff --git a/tsl/src/remote/txn_id.h b/tsl/src/remote/txn_id.h deleted file mode 100644 index 22112c2437e..00000000000 --- a/tsl/src/remote/txn_id.h +++ /dev/null @@ -1,59 +0,0 @@ -/* - * 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. - */ -#pragma once - -#include -#include - -#include "connection.h" - -/* - * This is the data node dist txn id to be used in PREPARE TRANSACTION and friends. - * From the data node perspective it has to be unique with regard to any concurrent - * prepared transactions. - * - * From the point of view of the access node, given such an id, an access node - * must be able to decide whether or not the corresponding distributed txn is still - * in progress or has committed or aborted. Therefore, an id issued by an access node - * must be unique for each of its connections. - * - * Note: a subtle point is that given this identifier we need to tell if the access node's - * transaction is still ongoing in the resolution logic without consulting the - * remote_txn table. This is because the remote_txn table is only populated once the txn - * is committed. Therefore this id contains the acess node's transaction_id directly. - * - * The current format is: version;xid;server_id;user_id. Both parts are necessary to - * guarantee uniqueness from the point of view of the data node. It is also critical to - * make sure the transaction has completed on the access node. - * - * - xid is a unique identifier for the dist txn on the access node. - * - * - pair of server_id and user_id dedups the connections made under different - * TSConnectionId mappings as part of the same access node's distributed txn. - * - * Note: When moving to multiple access nodes, we'll need to add a unique prefix for - * each access node. - */ - -typedef struct RemoteTxnId -{ - uint8 version; - char reserved[3]; /* not currently serialized */ - TransactionId xid; - TSConnectionId id; -} RemoteTxnId; - -extern RemoteTxnId *remote_txn_id_create(TransactionId xid, TSConnectionId id); -extern RemoteTxnId *remote_txn_id_in(const char *gid_string); - -extern bool remote_txn_id_matches_prepared_txn(const char *id_string); -extern Datum remote_txn_id_in_pg(PG_FUNCTION_ARGS); -extern const char *remote_txn_id_out(const RemoteTxnId *remote_txn_id); -extern Datum remote_txn_id_out_pg(PG_FUNCTION_ARGS); - -extern const char *remote_txn_id_prepare_transaction_sql(RemoteTxnId *); -extern const char *remote_txn_id_commit_prepared_sql(RemoteTxnId *); -extern const char *remote_txn_id_rollback_prepared_sql(RemoteTxnId *); diff --git a/tsl/src/remote/txn_resolve.c b/tsl/src/remote/txn_resolve.c deleted file mode 100644 index 4e1ee4bf342..00000000000 --- a/tsl/src/remote/txn_resolve.c +++ /dev/null @@ -1,230 +0,0 @@ -/* - * 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. - */ -#include -#include -#include -#include -#include -#include -#include -#include - -#include "txn_resolve.h" -#include "connection.h" -#include "txn.h" - -RemoteTxnResolution -remote_txn_resolution(Oid foreign_server, const RemoteTxnId *transaction_id) -{ - if (remote_txn_is_still_in_progress_on_access_node(transaction_id->xid)) - /* transaction still ongoing; don't know its state */ - return REMOTE_TXN_RESOLUTION_IN_PROGRESS; - - /* - * If an entry exists in the "remote_txn" table and is visible then it means - * that the transaction committed on the AN - */ - if (remote_txn_persistent_record_exists(transaction_id)) - return REMOTE_TXN_RESOLUTION_COMMIT; - - /* - * If the txn is not in progress and is not committed as per the "remote_txn" - * table then it's presumed to be aborted. - * - * We could ask PG machinery to confirm the abort but as long as we are sticking - * to one uniform behavior consistently it should be ok for now. - */ - return REMOTE_TXN_RESOLUTION_ABORT; -} - -/* - * Resolve any unresolved 2-pc transaction on a data node. - * Since the remote_txn log can be long, and most txn there - * will have been resolved, do not iterate that list. - * - * Instead query the data node for the list of unresolved txns - * via the pg_prepared_xacts view. Using that list, then check - * remote_txn. Use this as an opportunity to clean up remote_txn - * as well. - * - * Note that pg_prepared_xacts shared across other databases which - * also could be distributed. Right now we interested only in - * the current one. - */ -#define GET_PREPARED_XACT_SQL \ - "SELECT gid FROM pg_prepared_xacts WHERE database = current_database()" - -Datum -remote_txn_heal_data_node(PG_FUNCTION_ARGS) -{ - Oid foreign_server_oid = PG_GETARG_OID(0); - TSConnectionId id = remote_connection_id(foreign_server_oid, GetUserId()); - TSConnection *conn = remote_connection_open_session_by_id(id); - int resolved = 0; - - /* - * Use a raw connection since you need to be out of transaction to do - * COMMIT/ROLLBACK PREPARED - */ - PGresult *res; - int row; - List *in_progress_txn_gids = NIL, *healed_txn_gids = NIL; - int non_ts_txns = 0, ntuples; -#ifdef TS_DEBUG - int n_gid_errors = 0; /* how many errors to induce? */ -#endif - - /* - * This function cannot be called inside a transaction block since effects - * cannot be rolled back - */ - PreventInTransactionBlock(true, "remote_txn_heal_data_node"); - - res = remote_connection_query_ok(conn, GET_PREPARED_XACT_SQL); - - Assert(1 == PQnfields(res)); - ntuples = PQntuples(res); - for (row = 0; row < ntuples; row++) - { - char *id_string = PQgetvalue(res, row, 0); - RemoteTxnId *tpc_gid; - RemoteTxnResolution resolution; - - if (!remote_txn_id_matches_prepared_txn(id_string)) - { - non_ts_txns++; - continue; - } - - tpc_gid = remote_txn_id_in(id_string); - resolution = remote_txn_resolution(foreign_server_oid, tpc_gid); - -#ifdef TS_DEBUG - /* - * Induce an error in the GID so that the remote side errors out when it tries - * to heal it. - * - * We inject the error by checking the value of the below session variable. Not - * a full GUC, just a tool to allow us to randomly inject error for testing - * purposes. Depending on the value we will inject an error in the GID and also - * additionally change the resolution as per the accepted value: - * - * "commit" : change GID + set resolution as COMMITTED - * "abort" : change GID + set resolution as ABORTED - * "inprogress" : set resolution as IN_PROGRESS - * - * Any other setting will not have any effect - * - * We currently induce error in one GID processing. If needed this can be - * changed in the future via another session variable to set to a specific - * number of errors to induce. Note that this variable is incremented only - * for valid values of "timescaledb.debug_inject_gid_error. - * - * Current logic also means that the first GID being processed will always - * induce a change in resolution behavior. But that's ok, we could randomize - * it later to any arbitrary integer value less than ntuples in the future. - */ - if (n_gid_errors < 1) - { - const char *inject_gid_error = - GetConfigOption("timescaledb.debug_inject_gid_error", true, false); - - /* increment the user_id field to cause mismatch in GID */ - if (inject_gid_error) - { - if (strcmp(inject_gid_error, "abort") == 0) - { - tpc_gid->id.user_id++; - resolution = REMOTE_TXN_RESOLUTION_ABORT; - n_gid_errors++; - } - else if (strcmp(inject_gid_error, "commit") == 0) - { - tpc_gid->id.user_id++; - resolution = REMOTE_TXN_RESOLUTION_COMMIT; - n_gid_errors++; - } - else if (strcmp(inject_gid_error, "inprogress") == 0) - { - resolution = REMOTE_TXN_RESOLUTION_IN_PROGRESS; - n_gid_errors++; - } - /* any other value is simply ignored, n_gid_errors is also not incremented */ - } - } -#endif - /* - * We don't expect these commands to fail, but if they do, continue and move on to - * healing up the next GID in the list. The ones that failed will get retried if - * they are still around on the datanodes the next time over. - */ - switch (resolution) - { - case REMOTE_TXN_RESOLUTION_COMMIT: - if (PQresultStatus( - remote_connection_exec(conn, remote_txn_id_commit_prepared_sql(tpc_gid))) == - PGRES_COMMAND_OK) - { - healed_txn_gids = lappend(healed_txn_gids, id_string); - resolved++; - } - else - ereport(WARNING, - (errmsg("could not commit prepared transaction on data node \"%s\"", - remote_connection_node_name(conn)), - errhint("To retry, manually run \"COMMIT PREPARED %s\" on the data " - "node or run the healing function again.", - id_string))); - break; - case REMOTE_TXN_RESOLUTION_ABORT: - if (PQresultStatus(remote_connection_exec(conn, - remote_txn_id_rollback_prepared_sql( - tpc_gid))) == PGRES_COMMAND_OK) - { - healed_txn_gids = lappend(healed_txn_gids, id_string); - resolved++; - } - else - ereport(WARNING, - (errmsg("could not roll back prepared transaction on data node \"%s\"", - remote_connection_node_name(conn)), - errhint("To retry, manually run \"ROLLBACK PREPARED %s\" on the data " - "node or run the healing function again.", - id_string))); - break; - case REMOTE_TXN_RESOLUTION_IN_PROGRESS: - in_progress_txn_gids = lappend(in_progress_txn_gids, id_string); - break; - } - } - - if (non_ts_txns > 0) - elog(NOTICE, "skipping %d non-TimescaleDB prepared transaction", non_ts_txns); - - /* - * Perform cleanup of all records if there are no in progress txns and if the number of - * resolved entities is same as the number of rows obtained from the datanode. - * - * In a heavily loaded system there's a possibility of ongoing transactions always being - * present in which case we will never get a chance to clean up entries in "remote_txn" - * table. So, we track healed gids in a list and delete those specific rows to keep the - * "remote_txn" table from growing up indefinitely. - */ - if (list_length(in_progress_txn_gids) == 0 && resolved == ntuples) - remote_txn_persistent_record_delete_for_data_node(foreign_server_oid, NULL); - else if (resolved) - { - ListCell *lc; - Assert(healed_txn_gids != NIL); - - foreach (lc, healed_txn_gids) - remote_txn_persistent_record_delete_for_data_node(foreign_server_oid, lfirst(lc)); - } - - remote_result_close(res); - remote_connection_close(conn); - PG_RETURN_INT32(resolved); -} diff --git a/tsl/src/remote/txn_resolve.h b/tsl/src/remote/txn_resolve.h deleted file mode 100644 index 5b844ec249a..00000000000 --- a/tsl/src/remote/txn_resolve.h +++ /dev/null @@ -1,85 +0,0 @@ -/* - * 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. - */ -#pragma once - -#include - -#include "txn_id.h" -#include "fmgr.h" - -/* - * 2PC implementation details. - * - * This implementation uses the presumed-abort variant of 2PC. - * - * The access node is the coordinator and data nodes are the participants. - * Participant actions are implemented by native postgres `PREPARE TRANSACTION`/ - * `COMMIT PREPARED`/`ROLLBACK PREPARED`. - * - * This code relates to the coordinator actions. In presumed-abort, the coordinator - * needs to write a commit message to stable storage between the 1st and 2nd phase of 2PC. - * This is accomplished in this implementation by doing the 1st phase and writing records to - * `remote_txn` in the pre-commit hook. Then doing a commit, thus will write the `remote_txn` - * to stable storage. Finally we do the 2nd phase of 2PC in a post-commit hook - * (possible, later, in a background task). - * - * For the two-pc implementation we define the following correctness criteria: - * For all transactions, every associated remote transaction either commits or aborts. - * - * This is implements by using PREPARE TRANSACTION to make sure all remote nodes can commit. - * If the access node receives OKs for all PREPARE TRANSACTIONS, it writes an entry into the - * `remote_txn` table and then COMMITS locally. That commit serves as the sync point, - * if it happened then all nodes in the transaction should commit, otherwise nodes are free to - * abort. - * - * For each remote transactions there are three possible states of the leading transaction - * on the access node: - * - * Case 1 - The transaction is ongoing: - * In this case the state of the remote transaction is in progress - *(REMOTE_TXN_RESOLVE_IN_PROGRESS) - * - * Case 2 - The transaction is committed: - * The remote transaction MUST BE be committed (REMOTE_TXN_RESOLVE_COMMT) - * - Invariant: All remote transaction have been `PREPARE TRANSACTION` successfully - * -> Otherwise the acces node's transaction would have aborted - * -> Note: This guarantees that the remote transaction can be committed (commit cannot fail) - * - Invariant: An entry for each remote commit will exist in `remote_txn` - * - * Case 3 - The transaction is aborted: - * The remote transactions MUST BE aborted * (REMOTE_TXN_RESOLVE_COMMT) - * - Invariant: No entry will exist in `remote_txn`. - * - * Resolution procedure: - * - * A remote transaction commits if the access node's transaction is finished and - * there is an entry in `remote_txn`. If the transaction is ongoing, wait. - * Otherwise, abort. - * - * Note from the above we can do a case analysis: - * - * Case 1 - the transaction is ongoing: - * The transaction will eventually end up in either case 2 or 3. - * No remote transactions have been committed. - * - * Case 2 - access node commit: - * All remote transactions will eventually commit since they have been PREPARED - * and there is an entry in `remote_txn` - * - * Case 3 - access node abort: - * All remote transactions will eventually abort since there is no entry in `remote_txn` - */ - -typedef enum RemoteTxnResolution -{ - REMOTE_TXN_RESOLUTION_IN_PROGRESS = 0, - REMOTE_TXN_RESOLUTION_ABORT, - REMOTE_TXN_RESOLUTION_COMMIT -} RemoteTxnResolution; - -extern RemoteTxnResolution remote_txn_resolution(Oid foreign_server, - const RemoteTxnId *transaction_id); -extern Datum remote_txn_heal_data_node(PG_FUNCTION_ARGS); diff --git a/tsl/src/remote/txn_store.c b/tsl/src/remote/txn_store.c deleted file mode 100644 index b322c670dd6..00000000000 --- a/tsl/src/remote/txn_store.c +++ /dev/null @@ -1,101 +0,0 @@ -/* - * 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. - */ -#include -#include - -#include "txn_store.h" -#include "txn.h" -#include "connection.h" - -#include "connection_cache.h" - -#define DEFAULT_NUM_ITEMS 100 - -RemoteTxnStore * -remote_txn_store_create(MemoryContext mctx) -{ - HASHCTL ctl; - RemoteTxnStore *store = MemoryContextAlloc(mctx, sizeof(RemoteTxnStore)); - - MemSet(&ctl, 0, sizeof(ctl)); - ctl.keysize = sizeof(TSConnectionId); - ctl.entrysize = remote_txn_size(); - ctl.hcxt = mctx; - *store = (RemoteTxnStore){ - .hashtable = hash_create("RemoteTxnStore", - DEFAULT_NUM_ITEMS, - &ctl, - HASH_ELEM | HASH_BLOBS | HASH_CONTEXT), - .mctx = mctx, - }; - return store; -} - -RemoteTxn * -remote_txn_store_get(RemoteTxnStore *store, TSConnectionId id, bool *found_out) -{ - bool found; - RemoteTxn *entry; - - entry = hash_search(store->hashtable, &id, HASH_ENTER, &found); - - PG_TRY(); - { - TSConnection *conn; - - /* Get a connection from the connection cache. We do this even for - * existing remote transactions because it will run checks on connections - * to ensure they're in a good state. We validate below that connections - * aren't remade for existing transactions. Always getting a connection - * from the cache avoids having to redo the same checks here and we can - * keep connection validation in one place. */ - conn = remote_connection_cache_get_connection(id); - - if (found) - { - /* For existing transactions, we'd expect to continue on the same - * connection */ - if (remote_txn_get_connection(entry) != conn) - elog(ERROR, - "unexpected connection state for remote transaction on node \"%s\"", - remote_connection_node_name(conn)); - } - else - remote_txn_init(entry, conn); - } - PG_CATCH(); - { - remote_txn_store_remove(store, id); - PG_RE_THROW(); - } - PG_END_TRY(); - - if (found_out != NULL) - *found_out = found; - - return entry; -} - -void -remote_txn_store_remove(RemoteTxnStore *store, TSConnectionId id) -{ - bool found; - - hash_search(store->hashtable, &id, HASH_REMOVE, &found); - Assert(found); - remote_connection_cache_remove(id); -} - -void -remote_txn_store_destroy(RemoteTxnStore *store) -{ -#ifdef DEBUG - RemoteTxn *txn; - remote_txn_store_foreach(store, txn) { remote_txn_check_for_leaked_prepared_statements(txn); } -#endif - hash_destroy(store->hashtable); - store->hashtable = NULL; -} diff --git a/tsl/src/remote/txn_store.h b/tsl/src/remote/txn_store.h deleted file mode 100644 index 36b25810c41..00000000000 --- a/tsl/src/remote/txn_store.h +++ /dev/null @@ -1,34 +0,0 @@ -/* - * 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. - */ -#pragma once - -#include -#include - -#include "connection.h" -#include "txn.h" -#include "cache.h" - -/* Maps a TSConnectionId to a RemoteTxn. Used by the distributed txn to store the remote txns - * associated with a distributed txn. Note that this forces a distributed txn to contain a single - * RemoteTxn per TSConnectionId. This is actually required to maintain a consistent snapshot for - * each local user on a per-data-node basis. */ -typedef struct RemoteTxnStore -{ - HTAB *hashtable; - MemoryContext mctx; - HASH_SEQ_STATUS scan; -} RemoteTxnStore; - -extern RemoteTxnStore *remote_txn_store_create(MemoryContext mctx); -extern RemoteTxn *remote_txn_store_get(RemoteTxnStore *store, TSConnectionId id, bool *found); -extern void remote_txn_store_remove(RemoteTxnStore *store, TSConnectionId id); -extern void remote_txn_store_destroy(RemoteTxnStore *store); - -/* iterators */ -#define remote_txn_store_foreach(store, remote_txn) \ - for (hash_seq_init(&store->scan, store->hashtable); \ - NULL != (remote_txn = (RemoteTxn *) hash_seq_search(&store->scan));) diff --git a/tsl/src/remote/utils.c b/tsl/src/remote/utils.c deleted file mode 100644 index 64021017af4..00000000000 --- a/tsl/src/remote/utils.c +++ /dev/null @@ -1,90 +0,0 @@ -/* - * This file and its contents are licensed under the Timescale License. - * Please see the included NOTICE for copyright information and - * LICENSE-TIMESCALE for a copy of the license. - */ - -/* - * This file contains source code that was copied and/or modified from - * the PostgreSQL database, which is licensed under the open-source - * PostgreSQL License. Please see the NOTICE at the top level - * directory for a copy of the PostgreSQL License. - */ -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include "utils.h" -#include "compat/compat.h" -#include "guc.h" - -/* - * Force assorted GUC parameters to settings that ensure that we'll output - * data values in a form that is unambiguous to the data node. - * - * This is rather expensive and annoying to do once per row, but there's - * little choice if we want to be sure values are transmitted accurately; - * we can't leave the settings in place between rows for fear of affecting - * user-visible computations. - * - * We use the equivalent of a function SET option to allow the settings to - * persist only until the caller calls reset_transmission_modes(). If an - * error is thrown in between, guc.c will take care of undoing the settings. - * - * The return value is the nestlevel that must be passed to - * reset_transmission_modes() to undo things. - */ -int -set_transmission_modes(void) -{ - int nestlevel = NewGUCNestLevel(); - - /* - * The values set here should match what pg_dump does. See also - * configure_remote_session in connection.c. - */ - if (DateStyle != USE_ISO_DATES) - (void) set_config_option("datestyle", - "ISO", - PGC_USERSET, - PGC_S_SESSION, - GUC_ACTION_SAVE, - true, - 0, - false); - if (IntervalStyle != INTSTYLE_POSTGRES) - (void) set_config_option("intervalstyle", - "postgres", - PGC_USERSET, - PGC_S_SESSION, - GUC_ACTION_SAVE, - true, - 0, - false); - if (extra_float_digits < 3) - (void) set_config_option("extra_float_digits", - "3", - PGC_USERSET, - PGC_S_SESSION, - GUC_ACTION_SAVE, - true, - 0, - false); - - return nestlevel; -} - -/* - * Undo the effects of set_transmission_modes(). - */ -void -reset_transmission_modes(int nestlevel) -{ - AtEOXact_GUC(true, nestlevel); -} diff --git a/tsl/src/remote/utils.h b/tsl/src/remote/utils.h deleted file mode 100644 index 376eb02561f..00000000000 --- a/tsl/src/remote/utils.h +++ /dev/null @@ -1,12 +0,0 @@ -/* - * 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. - */ -#pragma once - -#include -#include - -extern int set_transmission_modes(void); -extern void reset_transmission_modes(int nestlevel); diff --git a/tsl/src/reorder.c b/tsl/src/reorder.c index 3b344b56f3a..f824da919d4 100644 --- a/tsl/src/reorder.c +++ b/tsl/src/reorder.c @@ -239,12 +239,6 @@ reorder_chunk(Oid chunk_id, Oid index_id, bool verbose, Oid wait_id, Oid destina aclcheck_error(ACLCHECK_NOT_OWNER, OBJECT_TABLE, get_rel_name(main_table_relid)); } - if (hypertable_is_distributed(ht)) - ereport(ERROR, - (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("move_chunk() and reorder_chunk() cannot be used " - "with distributed hypertables"))); - if (!chunk_get_reorder_index(ht, chunk, index_id, &cim)) { ts_cache_release(hcache); diff --git a/tsl/test/shared/expected/compat.out b/tsl/test/shared/expected/compat.out index 19f554702c1..44976e09648 100644 --- a/tsl/test/shared/expected/compat.out +++ b/tsl/test/shared/expected/compat.out @@ -82,9 +82,6 @@ ERROR: invalid Oid SELECT _timescaledb_internal.drop_chunk(0); WARNING: function _timescaledb_internal.drop_chunk(regclass) is deprecated and has been moved to _timescaledb_functions schema. this compatibility function will be removed in a future version. ERROR: chunk not found -SELECT _timescaledb_internal.drop_dist_ht_invalidation_trigger(0); -WARNING: function _timescaledb_internal.drop_dist_ht_invalidation_trigger(integer) is deprecated and has been moved to _timescaledb_functions schema. this compatibility function will be removed in a future version. -ERROR: function was not provided with a valid distributed hypertable id SELECT _timescaledb_internal.freeze_chunk(0); WARNING: function _timescaledb_internal.freeze_chunk(regclass) is deprecated and has been moved to _timescaledb_functions schema. this compatibility function will be removed in a future version. ERROR: invalid Oid diff --git a/tsl/test/shared/expected/extension.out b/tsl/test/shared/expected/extension.out index b2421ff85e7..de52fae4cc0 100644 --- a/tsl/test/shared/expected/extension.out +++ b/tsl/test/shared/expected/extension.out @@ -61,7 +61,6 @@ ORDER BY pronamespace::regnamespace::text COLLATE "C", p.oid::regprocedure::text _timescaledb_functions.dimension_info_in(cstring) _timescaledb_functions.dimension_info_out(_timescaledb_internal.dimension_info) _timescaledb_functions.drop_chunk(regclass) - _timescaledb_functions.drop_dist_ht_invalidation_trigger(integer) _timescaledb_functions.finalize_agg(text,name,name,name[],bytea,anyelement) _timescaledb_functions.finalize_agg_ffunc(internal,text,name,name,name[],bytea,anyelement) _timescaledb_functions.finalize_agg_sfunc(internal,text,name,name,name[],bytea,anyelement) @@ -165,7 +164,6 @@ ORDER BY pronamespace::regnamespace::text COLLATE "C", p.oid::regprocedure::text _timescaledb_internal.create_chunk_table(regclass,jsonb,name,name) _timescaledb_internal.create_compressed_chunk(regclass,regclass,bigint,bigint,bigint,bigint,bigint,bigint,bigint,bigint) _timescaledb_internal.drop_chunk(regclass) - _timescaledb_internal.drop_dist_ht_invalidation_trigger(integer) _timescaledb_internal.finalize_agg(text,name,name,name[],bytea,anyelement) _timescaledb_internal.finalize_agg_ffunc(internal,text,name,name,name[],bytea,anyelement) _timescaledb_internal.finalize_agg_sfunc(internal,text,name,name,name[],bytea,anyelement) diff --git a/tsl/test/shared/sql/compat.sql b/tsl/test/shared/sql/compat.sql index 337615729b1..702226ca597 100644 --- a/tsl/test/shared/sql/compat.sql +++ b/tsl/test/shared/sql/compat.sql @@ -25,7 +25,6 @@ SELECT _timescaledb_internal.create_chunk(0,NULL,NULL,NULL,0); SELECT _timescaledb_internal.create_chunk_table(0,NULL,NULL,NULL); SELECT _timescaledb_internal.create_compressed_chunk(0,0,0,0,0,0,0,0,0,0); SELECT _timescaledb_internal.drop_chunk(0); -SELECT _timescaledb_internal.drop_dist_ht_invalidation_trigger(0); SELECT _timescaledb_internal.freeze_chunk(0); SELECT FROM _timescaledb_internal.generate_uuid(); SELECT _timescaledb_internal.get_approx_row_count(0); diff --git a/tsl/test/sql/include/deparse_create.sql b/tsl/test/sql/include/deparse_create.sql deleted file mode 100644 index 12310c4916e..00000000000 --- a/tsl/test/sql/include/deparse_create.sql +++ /dev/null @@ -1,56 +0,0 @@ --- 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. - --- Lets create some tabels that we will try to deparse and recreate - -\c :TEST_DBNAME :ROLE_SUPERUSER - -SET ROLE :ROLE_DEFAULT_PERM_USER; - -CREATE TABLE table1(time TIMESTAMP, v FLOAT8, c CHAR(10), x NUMERIC(10,4), i interval hour to minute); - -CREATE TABLE table2(time TIMESTAMP NOT NULL, v FLOAT8[], d TEXT COLLATE "POSIX", num INT DEFAULT 100); - -CREATE TABLE table3(time TIMESTAMP PRIMARY KEY, v FLOAT8[][], num INT CHECK (num > 0), d INT UNIQUE, CONSTRAINT validate_num_and_d CHECK ( num > d)); - -CREATE TABLE table4(t TIMESTAMP , d INT, PRIMARY KEY (t, d)); - -CREATE TABLE ref_table(id INT PRIMARY KEY, d TEXT); - -CREATE TABLE table5(t TIMESTAMP PRIMARY KEY, v FLOAT8, d INT REFERENCES ref_table ON DELETE CASCADE); - -CREATE SEQUENCE my_seq; - -CREATE UNLOGGED TABLE table6(id INT NOT NULL DEFAULT nextval('my_seq'), t TEXT); - -CREATE INDEX ON table6 USING BTREE (t); - -RESET ROLE; - -CREATE TYPE device_status AS ENUM ('OFF', 'ON', 'BROKEN'); - -CREATE SCHEMA myschema AUTHORIZATION :ROLE_DEFAULT_PERM_USER;; - -SET ROLE :ROLE_DEFAULT_PERM_USER; - -CREATE TABLE table7(t TIMESTAMP, v INT); - -CREATE TABLE table8(id INT, status device_status); - -CREATE OR REPLACE FUNCTION empty_trigger_func() - RETURNS TRIGGER LANGUAGE PLPGSQL AS -$BODY$ -BEGIN -END -$BODY$; - -CREATE TRIGGER test_trigger BEFORE UPDATE OR DELETE ON table8 -FOR EACH STATEMENT EXECUTE FUNCTION empty_trigger_func(); - -CREATE RULE notify_me AS ON UPDATE TO table8 DO ALSO NOTIFY table8; - -CREATE TABLE table9(c CIRCLE, EXCLUDE USING gist (c WITH &&)); - -CREATE TABLE myschema.table10(t TIMESTAMP); - diff --git a/tsl/test/sql/include/deparse_func.sql b/tsl/test/sql/include/deparse_func.sql deleted file mode 100644 index e8376578d0b..00000000000 --- a/tsl/test/sql/include/deparse_func.sql +++ /dev/null @@ -1,47 +0,0 @@ --- 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. - -\c :TEST_DBNAME :ROLE_SUPERUSER - -CREATE OR REPLACE FUNCTION _timescaledb_internal.get_tabledef(tbl REGCLASS) RETURNS TEXT -AS :TSL_MODULE_PATHNAME, 'ts_test_get_tabledef' LANGUAGE C VOLATILE STRICT; - -CREATE OR REPLACE FUNCTION tsl_test_deparse_drop_chunks( - table_name REGCLASS, - older_than "any" = NULL, - newer_than "any" = NULL, - verbose BOOLEAN = FALSE) RETURNS TEXT -AS :TSL_MODULE_PATHNAME, 'ts_test_deparse_drop_chunks' LANGUAGE C VOLATILE; - -CREATE OR REPLACE FUNCTION tsl_test_deparse_scalar_func( - schema_name NAME = NULL, - table_name NAME = NULL, - "time" TIMESTAMPTZ = NOW(), - message TEXT = NULL, - not_set BOOLEAN = TRUE, - option BOOLEAN = FALSE - ) RETURNS BOOLEAN -AS :TSL_MODULE_PATHNAME, 'ts_test_deparse_func' LANGUAGE C VOLATILE; - -CREATE OR REPLACE FUNCTION tsl_test_deparse_named_scalar_func( - schema_name NAME = NULL, - table_name NAME = NULL, - "time" TIMESTAMPTZ = NOW(), - message TEXT = NULL, - not_set BOOLEAN = TRUE, - INOUT option BOOLEAN = FALSE - ) -AS :TSL_MODULE_PATHNAME, 'ts_test_deparse_func' LANGUAGE C VOLATILE; - -CREATE OR REPLACE FUNCTION tsl_test_deparse_composite_func( - schema_name NAME = NULL, - table_name NAME = NULL, - "time" TIMESTAMPTZ = NOW(), - message TEXT = NULL, - not_set BOOLEAN = NULL, - option BOOLEAN = FALSE - ) RETURNS TABLE (success BOOLEAN, message TEXT) -AS :TSL_MODULE_PATHNAME, 'ts_test_deparse_func' LANGUAGE C VOLATILE; - -SET ROLE :ROLE_DEFAULT_PERM_USER; diff --git a/tsl/test/sql/include/deparse_recreate.sql b/tsl/test/sql/include/deparse_recreate.sql deleted file mode 100644 index e1d9e6192b2..00000000000 --- a/tsl/test/sql/include/deparse_recreate.sql +++ /dev/null @@ -1,24 +0,0 @@ --- 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. - --- deparse each table%, drop it and recreate it using deparse result - -DO -$$DECLARE - tables CURSOR FOR - SELECT tablename, schemaname - FROM pg_tables - WHERE tablename LIKE 'table%' AND (schemaname = 'public' OR schemaname = 'myschema') - ORDER BY tablename; - deparse_stmt text; - tablename text; -BEGIN - FOR table_record IN tables - LOOP - tablename := format('%I.%I', table_record.schemaname, table_record.tablename); - EXECUTE format('SELECT _timescaledb_internal.get_tabledef(%L)', tablename) INTO deparse_stmt; - EXECUTE 'DROP TABLE ' || tablename; - EXECUTE deparse_stmt; - END LOOP; -END$$;