From 8c0bc94b5166d3592bbebbdf6bd7bdb21ce1653c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C3=96nder=20Kalac=C4=B1?= Date: Mon, 15 Nov 2021 13:10:18 +0100 Subject: [PATCH] Enable replication factor > 1 in metadata syncing (#5392) - [x] Add some more regression test coverage - [x] Make sure returning works fine in case of local execution + remote execution (task->partiallyLocalOrRemote works as expected, already added tests) - [x] Implement locking properly (and add isolation tests) - [x] We do #shardcount round-trips on `SerializeNonCommutativeWrites`. We made it a single round-trip. - [x] Acquire locks for subselects on the workers & add isolation tests - [x] Add a GUC to prevent modification from the workers, hence increase the coordinator-only throughput - The performance slightly drops (~%15), unless `citus.allow_modifications_from_workers_to_replicated_tables` is set to false --- .../distributed/executor/adaptive_executor.c | 80 +- .../distributed/executor/multi_executor.c | 1 + .../distributed/metadata/metadata_cache.c | 41 +- .../distributed/metadata/metadata_sync.c | 22 +- src/backend/distributed/shared_library_init.c | 17 + src/backend/distributed/utils/resource_lock.c | 139 +- src/include/distributed/multi_executor.h | 1 + src/include/distributed/resource_lock.h | 3 +- src/test/regress/citus_tests/config.py | 2 +- .../expected/isolation_concurrent_dml.out | 25 + .../isolation_dump_global_wait_edges.out | 10 + .../expected/isolation_hash_copy_vs_all.out | 250 +- .../isolation_insert_select_conflict.out | 316 +- .../isolation_modify_with_subquery_vs_dml.out | 62 +- ...licate_reference_tables_to_coordinator.out | 20 +- .../isolation_replicated_dist_on_mx.out | 2843 +++++++++++++++++ .../expected/isolation_undistribute_table.out | 50 + .../local_shard_execution_replicated.out | 2388 ++++++++++++++ .../expected/master_copy_shard_placement.out | 10 +- .../expected/metadata_sync_helpers.out | 12 - .../expected/multi_colocation_utils.out | 2 + .../expected/multi_distribution_metadata.out | 9 +- .../expected/multi_generate_ddl_commands.out | 1 + .../multi_mx_truncate_from_worker.out | 89 +- .../regress/expected/multi_repair_shards.out | 3 + .../mx_coordinator_shouldhaveshards.out | 139 + .../expected/start_stop_metadata_sync.out | 110 +- .../regress/input/multi_mx_copy_data.source | 10 + src/test/regress/isolation_schedule | 1 + src/test/regress/multi_mx_schedule | 2 +- .../regress/output/multi_mx_copy_data.source | 84 +- .../spec/isolation_concurrent_dml.spec | 3 + .../isolation_dump_global_wait_edges.spec | 3 + .../spec/isolation_hash_copy_vs_all.spec | 3 + .../isolation_insert_select_conflict.spec | 3 + ...isolation_modify_with_subquery_vs_dml.spec | 4 +- .../spec/isolation_replicated_dist_on_mx.spec | 220 ++ .../spec/isolation_undistribute_table.spec | 4 + .../sql/local_shard_execution_replicated.sql | 1090 +++++++ .../sql/master_copy_shard_placement.sql | 9 +- .../regress/sql/metadata_sync_helpers.sql | 8 - .../regress/sql/multi_colocation_utils.sql | 3 +- .../sql/multi_distribution_metadata.sql | 12 +- .../sql/multi_mx_truncate_from_worker.sql | 56 +- src/test/regress/sql/multi_repair_shards.sql | 3 + .../sql/mx_coordinator_shouldhaveshards.sql | 68 + .../regress/sql/start_stop_metadata_sync.sql | 58 + 47 files changed, 7956 insertions(+), 333 deletions(-) create mode 100644 src/test/regress/expected/isolation_replicated_dist_on_mx.out create mode 100644 src/test/regress/expected/local_shard_execution_replicated.out create mode 100644 src/test/regress/spec/isolation_replicated_dist_on_mx.spec create mode 100644 src/test/regress/sql/local_shard_execution_replicated.sql diff --git a/src/backend/distributed/executor/adaptive_executor.c b/src/backend/distributed/executor/adaptive_executor.c index 70960d92c..0e9d96fd5 100644 --- a/src/backend/distributed/executor/adaptive_executor.c +++ b/src/backend/distributed/executor/adaptive_executor.c @@ -1721,21 +1721,31 @@ AcquireExecutorShardLocksForExecution(DistributedExecution *execution) } /* now, iterate on the tasks and acquire the executor locks on the shards */ + List *anchorShardIntervalList = NIL; + List *relationRowLockList = NIL; + List *requiresConsistentSnapshotRelationShardList = NIL; + Task *task = NULL; foreach_ptr(task, taskList) { - /* - * If we are dealing with a partition we are also taking locks on parent table - * to prevent deadlocks on concurrent operations on a partition and its parent. - */ - LockParentShardResourceIfPartition(task->anchorShardId, lockMode); - ShardInterval *anchorShardInterval = LoadShardInterval(task->anchorShardId); - SerializeNonCommutativeWrites(list_make1(anchorShardInterval), lockMode); + anchorShardIntervalList = lappend(anchorShardIntervalList, anchorShardInterval); /* Acquire additional locks for SELECT .. FOR UPDATE on reference tables */ AcquireExecutorShardLocksForRelationRowLockList(task->relationRowLockList); + /* + * Due to PG commit 5ee190f8ec37c1bbfb3061e18304e155d600bc8e we copy the + * second parameter in pre-13. + */ + relationRowLockList = + list_concat(relationRowLockList, +#if (PG_VERSION_NUM >= PG_VERSION_12) && (PG_VERSION_NUM < PG_VERSION_13) + list_copy(task->relationRowLockList)); +#else + task->relationRowLockList); +#endif + /* * If the task has a subselect, then we may need to lock the shards from which * the query selects as well to prevent the subselects from seeing different @@ -1749,9 +1759,63 @@ AcquireExecutorShardLocksForExecution(DistributedExecution *execution) * concurrently. */ - LockRelationShardResources(task->relationShardList, ExclusiveLock); + /* + * Due to PG commit 5ee190f8ec37c1bbfb3061e18304e155d600bc8e we copy the + * second parameter in pre-13. + */ + requiresConsistentSnapshotRelationShardList = + list_concat(requiresConsistentSnapshotRelationShardList, +#if (PG_VERSION_NUM >= PG_VERSION_12) && (PG_VERSION_NUM < PG_VERSION_13) + + list_copy(task->relationShardList)); +#else + task->relationShardList); +#endif } } + + /* + * Acquire the locks in a sorted way to avoid deadlocks due to lock + * ordering across concurrent sessions. + */ + anchorShardIntervalList = + SortList(anchorShardIntervalList, CompareShardIntervalsById); + + /* + * If we are dealing with a partition we are also taking locks on parent table + * to prevent deadlocks on concurrent operations on a partition and its parent. + * + * Note that this function currently does not acquire any remote locks as that + * is necessary to control the concurrency across multiple nodes for replicated + * tables. That is because Citus currently does not allow modifications to + * partitions from any node other than the coordinator. + */ + LockParentShardResourceIfPartition(anchorShardIntervalList, lockMode); + + /* Acquire distribution execution locks on the affected shards */ + SerializeNonCommutativeWrites(anchorShardIntervalList, lockMode); + + if (relationRowLockList != NIL) + { + /* Acquire additional locks for SELECT .. FOR UPDATE on reference tables */ + AcquireExecutorShardLocksForRelationRowLockList(relationRowLockList); + } + + + if (requiresConsistentSnapshotRelationShardList != NIL) + { + /* + * If the task has a subselect, then we may need to lock the shards from which + * the query selects as well to prevent the subselects from seeing different + * results on different replicas. + * + * ExclusiveLock conflicts with all lock types used by modifications + * and therefore prevents other modifications from running + * concurrently. + */ + LockRelationShardResources(requiresConsistentSnapshotRelationShardList, + ExclusiveLock); + } } diff --git a/src/backend/distributed/executor/multi_executor.c b/src/backend/distributed/executor/multi_executor.c index 5c36d5bba..5a535043d 100644 --- a/src/backend/distributed/executor/multi_executor.c +++ b/src/backend/distributed/executor/multi_executor.c @@ -56,6 +56,7 @@ */ int MultiShardConnectionType = PARALLEL_CONNECTION; bool WritableStandbyCoordinator = false; +bool AllowModificationsFromWorkersToReplicatedTables = true; /* * Pointer to bound parameters of the current ongoing call to ExecutorRun. diff --git a/src/backend/distributed/metadata/metadata_cache.c b/src/backend/distributed/metadata/metadata_cache.c index 789601d44..c4bfce2f6 100644 --- a/src/backend/distributed/metadata/metadata_cache.c +++ b/src/backend/distributed/metadata/metadata_cache.c @@ -34,6 +34,7 @@ #include "distributed/colocation_utils.h" #include "distributed/connection_management.h" #include "distributed/citus_ruleutils.h" +#include "distributed/multi_executor.h" #include "distributed/function_utils.h" #include "distributed/foreign_key_relationship.h" #include "distributed/listutils.h" @@ -309,6 +310,32 @@ EnsureModificationsCanRunOnRelation(Oid relationId) { EnsureModificationsCanRun(); + if (!OidIsValid(relationId) || !IsCitusTable(relationId)) + { + /* we are not interested in PG tables */ + return; + } + + bool modifiedTableReplicated = + IsCitusTableType(relationId, REFERENCE_TABLE) || + !SingleReplicatedTable(relationId); + + if (!IsCoordinator() && !AllowModificationsFromWorkersToReplicatedTables && + modifiedTableReplicated) + { + ereport(ERROR, (errmsg("modifications via the worker nodes are not " + "allowed for replicated tables such as reference " + "tables or hash distributed tables with replication " + "factor greater than 1."), + errhint("All modifications to replicated tables should " + "happen via the coordinator unless " + "citus.allow_modifications_from_workers_to_replicated_tables " + " = true."), + errdetail("Allowing modifications from the worker nodes " + "requires extra locking which might decrease " + "the throughput."))); + } + /* * Even if user allows writes from standby, we should not allow for * replicated tables as they require 2PC. And, 2PC needs to write a log @@ -319,21 +346,15 @@ EnsureModificationsCanRunOnRelation(Oid relationId) return; } - if (!OidIsValid(relationId) || !IsCitusTable(relationId)) - { - /* we are not interested in PG tables */ - return; - } - - if (IsCitusTableType(relationId, REFERENCE_TABLE) || - !SingleReplicatedTable(relationId)) + if (modifiedTableReplicated) { ereport(ERROR, (errmsg("writing to worker nodes is not currently " "allowed for replicated tables such as reference " "tables or hash distributed tables with replication " "factor greater than 1."), - errhint("All modifications to replicated tables happen via 2PC, " - "and 2PC requires the database to be in a writable state."), + errhint("All modifications to replicated tables " + "happen via 2PC, and 2PC requires the " + "database to be in a writable state."), errdetail("the database is read-only"))); } } diff --git a/src/backend/distributed/metadata/metadata_sync.c b/src/backend/distributed/metadata/metadata_sync.c index 95a061510..37e2c0b26 100644 --- a/src/backend/distributed/metadata/metadata_sync.c +++ b/src/backend/distributed/metadata/metadata_sync.c @@ -367,9 +367,8 @@ ClusterHasKnownMetadataWorkers() /* * ShouldSyncTableMetadata checks if the metadata of a distributed table should be - * propagated to metadata workers, i.e. the table is an MX table or reference table. - * Tables with streaming replication model (which means RF=1) and hash distribution are - * considered as MX tables while tables with none distribution are reference tables. + * propagated to metadata workers, i.e. the table is a hash distributed table or + * reference/citus local table. */ bool ShouldSyncTableMetadata(Oid relationId) @@ -381,12 +380,8 @@ ShouldSyncTableMetadata(Oid relationId) CitusTableCacheEntry *tableEntry = GetCitusTableCacheEntry(relationId); - bool streamingReplicated = - (tableEntry->replicationModel == REPLICATION_MODEL_STREAMING); - - bool mxTable = (streamingReplicated && IsCitusTableTypeCacheEntry(tableEntry, - HASH_DISTRIBUTED)); - if (mxTable || IsCitusTableTypeCacheEntry(tableEntry, CITUS_TABLE_WITH_NO_DIST_KEY)) + if (IsCitusTableTypeCacheEntry(tableEntry, HASH_DISTRIBUTED) || + IsCitusTableTypeCacheEntry(tableEntry, CITUS_TABLE_WITH_NO_DIST_KEY)) { return true; } @@ -2182,15 +2177,6 @@ EnsurePartitionMetadataIsSane(Oid relationId, char distributionMethod, int coloc "known replication models."))); } - if (distributionMethod == DISTRIBUTE_BY_HASH && - replicationModel != REPLICATION_MODEL_STREAMING) - { - ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE), - errmsg("Hash distributed tables can only have '%c' " - "as the replication model.", - REPLICATION_MODEL_STREAMING))); - } - if (distributionMethod == DISTRIBUTE_BY_NONE && !(replicationModel == REPLICATION_MODEL_STREAMING || replicationModel == REPLICATION_MODEL_2PC)) diff --git a/src/backend/distributed/shared_library_init.c b/src/backend/distributed/shared_library_init.c index fb195fc2e..d6a1005e9 100644 --- a/src/backend/distributed/shared_library_init.c +++ b/src/backend/distributed/shared_library_init.c @@ -562,6 +562,23 @@ RegisterCitusConfigVariables(void) GUC_STANDARD, NULL, NULL, NULL); + DefineCustomBoolVariable( + "citus.allow_modifications_from_workers_to_replicated_tables", + gettext_noop("Enables modifications from workers to replicated " + "tables such as reference tables or hash " + "distributed tables with replication factor " + "greater than 1."), + gettext_noop("Allowing modifications from the worker nodes " + "requires extra locking which might decrease " + "the throughput. Disabling this GUC skips the " + "extra locking and prevents modifications from " + "worker nodes."), + &AllowModificationsFromWorkersToReplicatedTables, + true, + PGC_USERSET, + GUC_NO_SHOW_ALL, + NULL, NULL, NULL); + DefineCustomBoolVariable( "citus.binary_worker_copy_format", gettext_noop("Use the binary worker copy format."), diff --git a/src/backend/distributed/utils/resource_lock.c b/src/backend/distributed/utils/resource_lock.c index 3d99e8366..1c005e582 100644 --- a/src/backend/distributed/utils/resource_lock.c +++ b/src/backend/distributed/utils/resource_lock.c @@ -26,6 +26,7 @@ #include "distributed/coordinator_protocol.h" #include "distributed/metadata_cache.h" #include "distributed/metadata_sync.h" +#include "distributed/multi_executor.h" #include "distributed/multi_join_order.h" #include "distributed/multi_partitioning_utils.h" #include "distributed/distributed_planner.h" @@ -71,6 +72,8 @@ static const int lock_mode_to_string_map_count = sizeof(lockmode_to_string_map) /* local function forward declarations */ static LOCKMODE IntToLockMode(int mode); static void LockReferencedReferenceShardResources(uint64 shardId, LOCKMODE lockMode); +static bool AnyTableReplicated(List *shardIntervalList, + List **replicatedShardIntervalList); static void LockShardListResources(List *shardIntervalList, LOCKMODE lockMode); static void LockShardListResourcesOnFirstWorker(LOCKMODE lockmode, List *shardIntervalList); @@ -244,12 +247,27 @@ lock_shard_resources(PG_FUNCTION_ARGS) static void LockShardListResourcesOnFirstWorker(LOCKMODE lockmode, List *shardIntervalList) { + if (!AllowModificationsFromWorkersToReplicatedTables) + { + /* + * Allowing modifications from worker nodes for replicated tables requires + * to serialize modifications, see AcquireExecutorShardLocksForExecution() + * for the details. + * + * If the user opted for disabling modifications from the workers, we do not + * need to acquire these remote locks. Returning early saves us from an additional + * network round-trip. + */ + Assert(AnyTableReplicated(shardIntervalList, NULL)); + return; + } + StringInfo lockCommand = makeStringInfo(); int processedShardIntervalCount = 0; int totalShardIntervalCount = list_length(shardIntervalList); WorkerNode *firstWorkerNode = GetFirstPrimaryWorkerNode(); int connectionFlags = 0; - const char *superuser = CurrentUserName(); + const char *currentUser = CurrentUserName(); appendStringInfo(lockCommand, "SELECT lock_shard_resources(%d, ARRAY[", lockmode); @@ -282,7 +300,7 @@ LockShardListResourcesOnFirstWorker(LOCKMODE lockmode, List *shardIntervalList) ->workerName, firstWorkerNode ->workerPort, - superuser, + currentUser, NULL); /* the SELECT .. FOR UPDATE breaks if we lose the connection */ @@ -717,28 +735,81 @@ LockShardsInPlacementListMetadata(List *shardPlacementList, LOCKMODE lockMode) void SerializeNonCommutativeWrites(List *shardIntervalList, LOCKMODE lockMode) { - ShardInterval *firstShardInterval = (ShardInterval *) linitial(shardIntervalList); - int64 firstShardId = firstShardInterval->shardId; + if (shardIntervalList == NIL) + { + return; + } - if (ReferenceTableShardId(firstShardId)) + List *replicatedShardList = NIL; + if (AnyTableReplicated(shardIntervalList, &replicatedShardList)) { if (ClusterHasKnownMetadataWorkers() && !IsFirstWorkerNode()) { - LockShardListResourcesOnFirstWorker(lockMode, shardIntervalList); + LockShardListResourcesOnFirstWorker(lockMode, replicatedShardList); } - /* - * Referenced tables can cascade their changes to this table, and we - * want to serialize changes to keep different replicas consistent. - */ - LockReferencedReferenceShardResources(firstShardId, lockMode); + ShardInterval *firstShardInterval = + (ShardInterval *) linitial(replicatedShardList); + if (ReferenceTableShardId(firstShardInterval->shardId)) + { + /* + * Referenced tables can cascade their changes to this table, and we + * want to serialize changes to keep different replicas consistent. + * + * We currently only support foreign keys to reference tables, which are + * single shard. So, getting the first shard should be sufficient here. + */ + LockReferencedReferenceShardResources(firstShardInterval->shardId, lockMode); + } } - LockShardListResources(shardIntervalList, lockMode); } +/* + * AnyTableReplicated iterates on the shard list and returns true + * if any of the shard is a replicated table. We qualify replicated + * tables as any reference table or any distributed table with + * replication factor > 1. + * + * If the optional replicatedShardIntervalList is passed, the function + * fills it with the replicated shard intervals. + */ +static bool +AnyTableReplicated(List *shardIntervalList, List **replicatedShardIntervalList) +{ + if (replicatedShardIntervalList == NULL) + { + /* the caller is not interested in the replicatedShardIntervalList */ + List *localList = NIL; + replicatedShardIntervalList = &localList; + } + + *replicatedShardIntervalList = NIL; + + ShardInterval *shardInterval = NULL; + foreach_ptr(shardInterval, shardIntervalList) + { + int64 shardId = shardInterval->shardId; + + Oid relationId = RelationIdForShard(shardId); + if (ReferenceTableShardId(shardId)) + { + *replicatedShardIntervalList = + lappend(*replicatedShardIntervalList, LoadShardInterval(shardId)); + } + else if (!SingleReplicatedTable(relationId)) + { + *replicatedShardIntervalList = + lappend(*replicatedShardIntervalList, LoadShardInterval(shardId)); + } + } + + return list_length(*replicatedShardIntervalList) > 0; +} + + /* * LockShardListResources takes locks on all shards in shardIntervalList to * prevent concurrent DML statements on those shards. @@ -766,19 +837,25 @@ LockShardListResources(List *shardIntervalList, LOCKMODE lockMode) void LockRelationShardResources(List *relationShardList, LOCKMODE lockMode) { - /* lock shards in a consistent order to prevent deadlock */ - relationShardList = SortList(relationShardList, CompareRelationShards); + if (relationShardList == NIL) + { + return; + } + List *shardIntervalList = NIL; RelationShard *relationShard = NULL; foreach_ptr(relationShard, relationShardList) { uint64 shardId = relationShard->shardId; - if (shardId != INVALID_SHARD_ID) - { - LockShardResource(shardId, lockMode); - } + ShardInterval *shardInterval = LoadShardInterval(shardId); + + shardIntervalList = lappend(shardIntervalList, shardInterval); } + + /* lock shards in a consistent order to prevent deadlock */ + shardIntervalList = SortList(shardIntervalList, CompareShardIntervalsById); + SerializeNonCommutativeWrites(shardIntervalList, lockMode); } @@ -788,19 +865,29 @@ LockRelationShardResources(List *relationShardList, LOCKMODE lockMode) * shard resource lock on the colocated shard of the parent table. */ void -LockParentShardResourceIfPartition(uint64 shardId, LOCKMODE lockMode) +LockParentShardResourceIfPartition(List *shardIntervalList, LOCKMODE lockMode) { - ShardInterval *shardInterval = LoadShardInterval(shardId); - Oid relationId = shardInterval->relationId; + List *parentShardIntervalList = NIL; - if (PartitionTable(relationId)) + ShardInterval *shardInterval = NULL; + foreach_ptr(shardInterval, shardIntervalList) { - int shardIndex = ShardIndex(shardInterval); - Oid parentRelationId = PartitionParentOid(relationId); - uint64 parentShardId = ColocatedShardIdInRelation(parentRelationId, shardIndex); + Oid relationId = shardInterval->relationId; - LockShardResource(parentShardId, lockMode); + if (PartitionTable(relationId)) + { + int shardIndex = ShardIndex(shardInterval); + Oid parentRelationId = PartitionParentOid(relationId); + uint64 parentShardId = ColocatedShardIdInRelation(parentRelationId, + shardIndex); + + ShardInterval *parentShardInterval = LoadShardInterval(parentShardId); + parentShardIntervalList = lappend(parentShardIntervalList, + parentShardInterval); + } } + + LockShardListResources(parentShardIntervalList, lockMode); } diff --git a/src/include/distributed/multi_executor.h b/src/include/distributed/multi_executor.h index 14d280e9e..143f5a1c7 100644 --- a/src/include/distributed/multi_executor.h +++ b/src/include/distributed/multi_executor.h @@ -63,6 +63,7 @@ typedef struct TransactionProperties extern int MultiShardConnectionType; extern bool WritableStandbyCoordinator; +extern bool AllowModificationsFromWorkersToReplicatedTables; extern bool ForceMaxQueryParallelization; extern int MaxAdaptiveExecutorPoolSize; extern int ExecutorSlowStartInterval; diff --git a/src/include/distributed/resource_lock.h b/src/include/distributed/resource_lock.h index 072591cda..4fa53144c 100644 --- a/src/include/distributed/resource_lock.h +++ b/src/include/distributed/resource_lock.h @@ -145,7 +145,8 @@ extern void LockRelationShardResources(List *relationShardList, LOCKMODE lockMod extern List * GetSortedReferenceShardIntervals(List *relationList); /* Lock parent table's colocated shard resource */ -extern void LockParentShardResourceIfPartition(uint64 shardId, LOCKMODE lockMode); +extern void LockParentShardResourceIfPartition(List *shardIntervalList, + LOCKMODE lockMode); /* Lock mode translation between text and enum */ extern LOCKMODE LockModeTextToLockMode(const char *lockModeName); diff --git a/src/test/regress/citus_tests/config.py b/src/test/regress/citus_tests/config.py index 41aa2bfd0..c80b3d14c 100644 --- a/src/test/regress/citus_tests/config.py +++ b/src/test/regress/citus_tests/config.py @@ -316,7 +316,7 @@ class CitusSingleNodeSingleShardClusterConfig(CitusDefaultClusterConfig): common.coordinator_should_haveshards(self.bindir, self.coordinator_port()) -class CitusShardReplicationFactorClusterConfig(CitusDefaultClusterConfig): +class CitusShardReplicationFactorClusterConfig(CitusMXBaseClusterConfig): def __init__(self, arguments): super().__init__(arguments) self.new_settings = {"citus.shard_replication_factor": 2} diff --git a/src/test/regress/expected/isolation_concurrent_dml.out b/src/test/regress/expected/isolation_concurrent_dml.out index 32f3c6246..112f8a4b8 100644 --- a/src/test/regress/expected/isolation_concurrent_dml.out +++ b/src/test/regress/expected/isolation_concurrent_dml.out @@ -19,6 +19,11 @@ step s1-commit: COMMIT; step s2-update: <... completed> +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-insert s2-update master_create_worker_shards @@ -32,6 +37,11 @@ step s1-insert: step s2-update: UPDATE test_concurrent_dml SET data = 'blarg' WHERE test_id = 1; +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-begin s1-multi-insert s2-update s1-commit master_create_worker_shards @@ -52,6 +62,11 @@ step s1-commit: COMMIT; step s2-update: <... completed> +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-begin s1-multi-insert s2-multi-insert-overlap s1-commit master_create_worker_shards @@ -71,6 +86,11 @@ step s2-multi-insert-overlap: step s1-commit: COMMIT; +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-begin s2-begin s1-multi-insert s2-multi-insert s1-commit s2-commit master_create_worker_shards @@ -96,3 +116,8 @@ step s1-commit: step s2-commit: COMMIT; +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + diff --git a/src/test/regress/expected/isolation_dump_global_wait_edges.out b/src/test/regress/expected/isolation_dump_global_wait_edges.out index 574b397ff..e71a3085d 100644 --- a/src/test/regress/expected/isolation_dump_global_wait_edges.out +++ b/src/test/regress/expected/isolation_dump_global_wait_edges.out @@ -44,6 +44,11 @@ step s2-update: <... completed> step s2-abort: ABORT; +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-begin s2-begin s3-begin s1-update s2-update s3-update detector-dump-wait-edges s1-abort s2-abort s3-abort step s1-begin: @@ -102,3 +107,8 @@ step s3-update: <... completed> step s3-abort: ABORT; +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + diff --git a/src/test/regress/expected/isolation_hash_copy_vs_all.out b/src/test/regress/expected/isolation_hash_copy_vs_all.out index 51a687dc2..9e2393ae0 100644 --- a/src/test/regress/expected/isolation_hash_copy_vs_all.out +++ b/src/test/regress/expected/isolation_hash_copy_vs_all.out @@ -17,6 +17,11 @@ count 15 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-copy s2-router-select s1-commit s1-select-count create_distributed_table @@ -40,6 +45,11 @@ count 10 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-copy s2-real-time-select s1-commit s1-select-count create_distributed_table @@ -67,6 +77,11 @@ count 10 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-copy s2-adaptive-select s1-commit s1-select-count create_distributed_table @@ -78,8 +93,8 @@ step s1-initialize: COPY hash_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && step s1-begin: BEGIN; step s1-copy: COPY hash_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; step s2-adaptive-select: - SET citus.enable_repartition_joins TO ON; - SELECT * FROM hash_copy AS t1 JOIN hash_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; + SET citus.enable_repartition_joins TO ON; + SELECT * FROM hash_copy AS t1 JOIN hash_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id|data|int_data|id|data|int_data --------------------------------------------------------------------- @@ -97,6 +112,11 @@ count 10 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-copy s2-insert s1-commit s1-select-count create_distributed_table @@ -115,6 +135,11 @@ count 11 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-copy s2-insert-select s1-commit s1-select-count create_distributed_table @@ -133,6 +158,11 @@ count 15 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-copy s2-update s1-commit s1-select-count create_distributed_table @@ -151,6 +181,11 @@ count 10 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-copy s2-delete s1-commit s1-select-count create_distributed_table @@ -169,6 +204,11 @@ count 9 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-copy s2-truncate s1-commit s1-select-count create_distributed_table @@ -188,6 +228,11 @@ count 0 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-copy s2-drop s1-commit s1-select-count create_distributed_table @@ -203,6 +248,11 @@ step s1-commit: COMMIT; step s2-drop: <... completed> step s1-select-count: SELECT COUNT(*) FROM hash_copy; ERROR: relation "hash_copy" does not exist +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-copy s2-ddl-create-index s1-commit s1-select-count s1-show-indexes create_distributed_table @@ -229,6 +279,11 @@ run_command_on_workers (localhost,57638,t,2) (2 rows) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-ddl-create-index s1-begin s1-copy s2-ddl-drop-index s1-commit s1-select-count s1-show-indexes create_distributed_table @@ -256,6 +311,11 @@ run_command_on_workers (localhost,57638,t,0) (2 rows) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-copy s2-ddl-create-index-concurrently s1-commit s1-select-count s1-show-indexes create_distributed_table @@ -282,6 +342,11 @@ run_command_on_workers (localhost,57638,t,2) (2 rows) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-copy s2-ddl-add-column s1-commit s1-select-count s1-show-columns create_distributed_table @@ -308,6 +373,11 @@ run_command_on_workers (localhost,57638,t,new_column) (2 rows) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-ddl-add-column s1-begin s1-copy-additional-column s2-ddl-drop-column s1-commit s1-select-count s1-show-columns create_distributed_table @@ -335,6 +405,11 @@ run_command_on_workers (localhost,57638,t,"") (2 rows) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-copy s2-ddl-rename-column s1-commit s1-select-count s1-show-columns create_distributed_table @@ -361,6 +436,11 @@ run_command_on_workers (localhost,57638,t,new_column) (2 rows) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-copy s2-table-size s1-commit s1-select-count create_distributed_table @@ -384,6 +464,11 @@ count 10 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-copy s2-master-modify-multiple-shards s1-commit s1-select-count create_distributed_table @@ -402,6 +487,11 @@ count 5 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-copy s2-master-drop-all-shards s1-commit s1-select-count create_distributed_table @@ -426,6 +516,11 @@ count 0 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-drop s1-create-non-distributed-table s1-initialize s1-begin s1-copy s2-distribute-table s1-commit s1-select-count create_distributed_table @@ -452,6 +547,11 @@ count 15 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-recreate-with-replication-2 s1-initialize s1-begin s1-copy s2-update s1-commit s1-select-count create_distributed_table @@ -460,10 +560,10 @@ create_distributed_table (1 row) step s1-recreate-with-replication-2: - DROP TABLE hash_copy; - SET citus.shard_replication_factor TO 2; - CREATE TABLE hash_copy(id integer, data text, int_data int); - SELECT create_distributed_table('hash_copy', 'id'); + DROP TABLE hash_copy; + SET citus.shard_replication_factor TO 2; + CREATE TABLE hash_copy(id integer, data text, int_data int); + SELECT create_distributed_table('hash_copy', 'id'); create_distributed_table --------------------------------------------------------------------- @@ -482,6 +582,11 @@ count 10 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-recreate-with-replication-2 s1-initialize s1-begin s1-copy s2-delete s1-commit s1-select-count create_distributed_table @@ -490,10 +595,10 @@ create_distributed_table (1 row) step s1-recreate-with-replication-2: - DROP TABLE hash_copy; - SET citus.shard_replication_factor TO 2; - CREATE TABLE hash_copy(id integer, data text, int_data int); - SELECT create_distributed_table('hash_copy', 'id'); + DROP TABLE hash_copy; + SET citus.shard_replication_factor TO 2; + CREATE TABLE hash_copy(id integer, data text, int_data int); + SELECT create_distributed_table('hash_copy', 'id'); create_distributed_table --------------------------------------------------------------------- @@ -512,6 +617,11 @@ count 9 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-recreate-with-replication-2 s1-initialize s1-begin s1-copy s2-insert-select s1-commit s1-select-count create_distributed_table @@ -520,10 +630,10 @@ create_distributed_table (1 row) step s1-recreate-with-replication-2: - DROP TABLE hash_copy; - SET citus.shard_replication_factor TO 2; - CREATE TABLE hash_copy(id integer, data text, int_data int); - SELECT create_distributed_table('hash_copy', 'id'); + DROP TABLE hash_copy; + SET citus.shard_replication_factor TO 2; + CREATE TABLE hash_copy(id integer, data text, int_data int); + SELECT create_distributed_table('hash_copy', 'id'); create_distributed_table --------------------------------------------------------------------- @@ -542,6 +652,11 @@ count 20 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-recreate-with-replication-2 s1-initialize s1-begin s1-copy s2-master-modify-multiple-shards s1-commit s1-select-count create_distributed_table @@ -550,10 +665,10 @@ create_distributed_table (1 row) step s1-recreate-with-replication-2: - DROP TABLE hash_copy; - SET citus.shard_replication_factor TO 2; - CREATE TABLE hash_copy(id integer, data text, int_data int); - SELECT create_distributed_table('hash_copy', 'id'); + DROP TABLE hash_copy; + SET citus.shard_replication_factor TO 2; + CREATE TABLE hash_copy(id integer, data text, int_data int); + SELECT create_distributed_table('hash_copy', 'id'); create_distributed_table --------------------------------------------------------------------- @@ -572,6 +687,11 @@ count 0 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-router-select s2-copy s1-commit s1-select-count create_distributed_table @@ -595,6 +715,11 @@ count 10 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-real-time-select s2-copy s1-commit s1-select-count create_distributed_table @@ -622,6 +747,11 @@ count 10 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-adaptive-select s2-copy s1-commit s1-select-count create_distributed_table @@ -632,8 +762,8 @@ create_distributed_table step s1-initialize: COPY hash_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; step s1-begin: BEGIN; step s1-adaptive-select: - SET citus.enable_repartition_joins TO ON; - SELECT * FROM hash_copy AS t1 JOIN hash_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; + SET citus.enable_repartition_joins TO ON; + SELECT * FROM hash_copy AS t1 JOIN hash_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id|data|int_data|id|data|int_data --------------------------------------------------------------------- @@ -652,6 +782,11 @@ count 10 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-insert s2-copy s1-commit s1-select-count create_distributed_table @@ -670,6 +805,11 @@ count 11 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-insert-select s2-copy s1-commit s1-select-count create_distributed_table @@ -688,6 +828,11 @@ count 15 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-update s2-copy s1-commit s1-select-count create_distributed_table @@ -706,6 +851,11 @@ count 10 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-delete s2-copy s1-commit s1-select-count create_distributed_table @@ -724,6 +874,11 @@ count 9 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-truncate s2-copy s1-commit s1-select-count create_distributed_table @@ -743,6 +898,11 @@ count 5 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-drop s2-copy s1-commit s1-select-count create_distributed_table @@ -759,6 +919,11 @@ step s2-copy: <... completed> ERROR: relation "hash_copy" does not exist step s1-select-count: SELECT COUNT(*) FROM hash_copy; ERROR: relation "hash_copy" does not exist +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-ddl-create-index s2-copy s1-commit s1-select-count s1-show-indexes create_distributed_table @@ -785,6 +950,11 @@ run_command_on_workers (localhost,57638,t,2) (2 rows) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-ddl-create-index s1-begin s1-ddl-drop-index s2-copy s1-commit s1-select-count s1-show-indexes create_distributed_table @@ -812,6 +982,11 @@ run_command_on_workers (localhost,57638,t,0) (2 rows) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-ddl-add-column s2-copy s1-commit s1-select-count s1-show-columns create_distributed_table @@ -839,6 +1014,11 @@ run_command_on_workers (localhost,57638,t,new_column) (2 rows) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-ddl-add-column s1-begin s1-ddl-drop-column s2-copy s1-commit s1-select-count s1-show-columns create_distributed_table @@ -866,6 +1046,11 @@ run_command_on_workers (localhost,57638,t,"") (2 rows) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-ddl-rename-column s2-copy s1-commit s1-select-count s1-show-columns create_distributed_table @@ -892,6 +1077,11 @@ run_command_on_workers (localhost,57638,t,new_column) (2 rows) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-table-size s2-copy s1-commit s1-select-count create_distributed_table @@ -915,6 +1105,11 @@ count 10 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-master-modify-multiple-shards s2-copy s1-commit s1-select-count create_distributed_table @@ -933,6 +1128,11 @@ count 5 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-initialize s1-begin s1-master-drop-all-shards s2-copy s1-commit s1-select-count create_distributed_table @@ -958,6 +1158,11 @@ count 0 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-drop s1-create-non-distributed-table s1-initialize s1-begin s1-distribute-table s2-copy s1-commit s1-select-count create_distributed_table @@ -984,3 +1189,8 @@ count 15 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + diff --git a/src/test/regress/expected/isolation_insert_select_conflict.out b/src/test/regress/expected/isolation_insert_select_conflict.out index 9c5411727..2a375574a 100644 --- a/src/test/regress/expected/isolation_insert_select_conflict.out +++ b/src/test/regress/expected/isolation_insert_select_conflict.out @@ -7,21 +7,21 @@ create_distributed_table (1 row) step s1-begin: - SET citus.shard_replication_factor to 1; - BEGIN; + SET citus.shard_replication_factor to 1; + BEGIN; step s1-insert-into-select-conflict-update: - INSERT INTO target_table - SELECT - col_1, col_2 - FROM ( - SELECT - col_1, col_2, col_3 - FROM - source_table - LIMIT 5 - ) as foo - ON CONFLICT(col_1) DO UPDATE SET col_2 = EXCLUDED.col_2 RETURNING *; + INSERT INTO target_table + SELECT + col_1, col_2 + FROM ( + SELECT + col_1, col_2, col_3 + FROM + source_table + LIMIT 5 + ) as foo + ON CONFLICT(col_1) DO UPDATE SET col_2 = EXCLUDED.col_2 RETURNING *; col_1|col_2 --------------------------------------------------------------------- @@ -33,17 +33,22 @@ col_1|col_2 (5 rows) step s2-begin: - BEGIN; + BEGIN; step s2-update: - UPDATE target_table SET col_2 = 5; + UPDATE target_table SET col_2 = 5; step s1-commit: - COMMIT; + COMMIT; step s2-update: <... completed> step s2-commit: - COMMIT; + COMMIT; + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) starting permutation: s1-begin s1-insert-into-select-conflict-do-nothing s2-begin s2-delete s1-commit s2-commit @@ -53,34 +58,39 @@ create_distributed_table (1 row) step s1-begin: - SET citus.shard_replication_factor to 1; - BEGIN; + SET citus.shard_replication_factor to 1; + BEGIN; step s1-insert-into-select-conflict-do-nothing: - INSERT INTO target_table - SELECT - col_1, col_2 - FROM ( - SELECT - col_1, col_2, col_3 - FROM - source_table - LIMIT 5 - ) as foo - ON CONFLICT DO NOTHING; + INSERT INTO target_table + SELECT + col_1, col_2 + FROM ( + SELECT + col_1, col_2, col_3 + FROM + source_table + LIMIT 5 + ) as foo + ON CONFLICT DO NOTHING; step s2-begin: - BEGIN; + BEGIN; step s2-delete: - DELETE FROM target_table; + DELETE FROM target_table; step s1-commit: - COMMIT; + COMMIT; step s2-delete: <... completed> step s2-commit: - COMMIT; + COMMIT; + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) starting permutation: s1-begin s1-insert-into-select-conflict-do-nothing s2-begin s2-insert-into-select-conflict-update s1-commit s2-commit @@ -90,40 +100,40 @@ create_distributed_table (1 row) step s1-begin: - SET citus.shard_replication_factor to 1; - BEGIN; + SET citus.shard_replication_factor to 1; + BEGIN; step s1-insert-into-select-conflict-do-nothing: - INSERT INTO target_table - SELECT - col_1, col_2 - FROM ( - SELECT - col_1, col_2, col_3 - FROM - source_table - LIMIT 5 - ) as foo - ON CONFLICT DO NOTHING; + INSERT INTO target_table + SELECT + col_1, col_2 + FROM ( + SELECT + col_1, col_2, col_3 + FROM + source_table + LIMIT 5 + ) as foo + ON CONFLICT DO NOTHING; step s2-begin: - BEGIN; + BEGIN; step s2-insert-into-select-conflict-update: - INSERT INTO target_table - SELECT - col_1, col_2 - FROM ( - SELECT - col_1, col_2, col_3 - FROM - source_table - LIMIT 5 - ) as foo - ON CONFLICT(col_1) DO UPDATE SET col_2 = EXCLUDED.col_2 RETURNING *; + INSERT INTO target_table + SELECT + col_1, col_2 + FROM ( + SELECT + col_1, col_2, col_3 + FROM + source_table + LIMIT 5 + ) as foo + ON CONFLICT(col_1) DO UPDATE SET col_2 = EXCLUDED.col_2 RETURNING *; step s1-commit: - COMMIT; + COMMIT; step s2-insert-into-select-conflict-update: <... completed> col_1|col_2 @@ -136,7 +146,12 @@ col_1|col_2 (5 rows) step s2-commit: - COMMIT; + COMMIT; + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) starting permutation: s1-begin s1-insert-into-select-conflict-update s2-begin s2-insert-into-select-conflict-update s1-commit s2-commit @@ -146,21 +161,21 @@ create_distributed_table (1 row) step s1-begin: - SET citus.shard_replication_factor to 1; - BEGIN; + SET citus.shard_replication_factor to 1; + BEGIN; step s1-insert-into-select-conflict-update: - INSERT INTO target_table - SELECT - col_1, col_2 - FROM ( - SELECT - col_1, col_2, col_3 - FROM - source_table - LIMIT 5 - ) as foo - ON CONFLICT(col_1) DO UPDATE SET col_2 = EXCLUDED.col_2 RETURNING *; + INSERT INTO target_table + SELECT + col_1, col_2 + FROM ( + SELECT + col_1, col_2, col_3 + FROM + source_table + LIMIT 5 + ) as foo + ON CONFLICT(col_1) DO UPDATE SET col_2 = EXCLUDED.col_2 RETURNING *; col_1|col_2 --------------------------------------------------------------------- @@ -172,23 +187,23 @@ col_1|col_2 (5 rows) step s2-begin: - BEGIN; + BEGIN; step s2-insert-into-select-conflict-update: - INSERT INTO target_table - SELECT - col_1, col_2 - FROM ( - SELECT - col_1, col_2, col_3 - FROM - source_table - LIMIT 5 - ) as foo - ON CONFLICT(col_1) DO UPDATE SET col_2 = EXCLUDED.col_2 RETURNING *; + INSERT INTO target_table + SELECT + col_1, col_2 + FROM ( + SELECT + col_1, col_2, col_3 + FROM + source_table + LIMIT 5 + ) as foo + ON CONFLICT(col_1) DO UPDATE SET col_2 = EXCLUDED.col_2 RETURNING *; step s1-commit: - COMMIT; + COMMIT; step s2-insert-into-select-conflict-update: <... completed> col_1|col_2 @@ -201,7 +216,12 @@ col_1|col_2 (5 rows) step s2-commit: - COMMIT; + COMMIT; + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) starting permutation: s1-begin s1-insert-into-select-conflict-update s2-begin s2-insert-into-select-conflict-do-nothing s1-commit s2-commit @@ -211,21 +231,21 @@ create_distributed_table (1 row) step s1-begin: - SET citus.shard_replication_factor to 1; - BEGIN; + SET citus.shard_replication_factor to 1; + BEGIN; step s1-insert-into-select-conflict-update: - INSERT INTO target_table - SELECT - col_1, col_2 - FROM ( - SELECT - col_1, col_2, col_3 - FROM - source_table - LIMIT 5 - ) as foo - ON CONFLICT(col_1) DO UPDATE SET col_2 = EXCLUDED.col_2 RETURNING *; + INSERT INTO target_table + SELECT + col_1, col_2 + FROM ( + SELECT + col_1, col_2, col_3 + FROM + source_table + LIMIT 5 + ) as foo + ON CONFLICT(col_1) DO UPDATE SET col_2 = EXCLUDED.col_2 RETURNING *; col_1|col_2 --------------------------------------------------------------------- @@ -237,27 +257,32 @@ col_1|col_2 (5 rows) step s2-begin: - BEGIN; + BEGIN; step s2-insert-into-select-conflict-do-nothing: - INSERT INTO target_table - SELECT - col_1, col_2 - FROM ( - SELECT - col_1, col_2, col_3 - FROM - source_table - LIMIT 5 - ) as foo - ON CONFLICT DO NOTHING; + INSERT INTO target_table + SELECT + col_1, col_2 + FROM ( + SELECT + col_1, col_2, col_3 + FROM + source_table + LIMIT 5 + ) as foo + ON CONFLICT DO NOTHING; step s1-commit: - COMMIT; + COMMIT; step s2-insert-into-select-conflict-do-nothing: <... completed> step s2-commit: - COMMIT; + COMMIT; + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) starting permutation: s1-begin-replication-factor-2 s1-insert-into-select-conflict-update-replication-factor-2 s2-begin-replication-factor-2 s2-insert-into-select-conflict-update-replication-factor-2 s1-commit s2-commit @@ -267,21 +292,21 @@ create_distributed_table (1 row) step s1-begin-replication-factor-2: - SET citus.shard_replication_factor to 2; - BEGIN; + SET citus.shard_replication_factor to 2; + BEGIN; step s1-insert-into-select-conflict-update-replication-factor-2: - INSERT INTO target_table_2 - SELECT - col_1, col_2 - FROM ( - SELECT - col_1, col_2, col_3 - FROM - source_table - LIMIT 5 - ) as foo - ON CONFLICT(col_1) DO UPDATE SET col_2 = EXCLUDED.col_2 RETURNING *; + INSERT INTO target_table_2 + SELECT + col_1, col_2 + FROM ( + SELECT + col_1, col_2, col_3 + FROM + source_table + LIMIT 5 + ) as foo + ON CONFLICT(col_1) DO UPDATE SET col_2 = EXCLUDED.col_2 RETURNING *; col_1|col_2|col_3 --------------------------------------------------------------------- @@ -293,24 +318,24 @@ col_1|col_2|col_3 (5 rows) step s2-begin-replication-factor-2: - SET citus.shard_replication_factor to 2; - BEGIN; + SET citus.shard_replication_factor to 2; + BEGIN; step s2-insert-into-select-conflict-update-replication-factor-2: - INSERT INTO target_table_2 - SELECT - col_1, col_2 - FROM ( - SELECT - col_1, col_2, col_3 - FROM - source_table - LIMIT 5 - ) as foo - ON CONFLICT(col_1) DO UPDATE SET col_2 = EXCLUDED.col_2 RETURNING *; + INSERT INTO target_table_2 + SELECT + col_1, col_2 + FROM ( + SELECT + col_1, col_2, col_3 + FROM + source_table + LIMIT 5 + ) as foo + ON CONFLICT(col_1) DO UPDATE SET col_2 = EXCLUDED.col_2 RETURNING *; step s1-commit: - COMMIT; + COMMIT; step s2-insert-into-select-conflict-update-replication-factor-2: <... completed> col_1|col_2|col_3 @@ -323,5 +348,10 @@ col_1|col_2|col_3 (5 rows) step s2-commit: - COMMIT; + COMMIT; + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) diff --git a/src/test/regress/expected/isolation_modify_with_subquery_vs_dml.out b/src/test/regress/expected/isolation_modify_with_subquery_vs_dml.out index 15328aab6..5e6f9fdba 100644 --- a/src/test/regress/expected/isolation_modify_with_subquery_vs_dml.out +++ b/src/test/regress/expected/isolation_modify_with_subquery_vs_dml.out @@ -5,7 +5,7 @@ step s1-begin: BEGIN; step s2-begin: - BEGIN; + BEGIN; step s2-modify_with_subquery_v1: UPDATE users_test_table SET value_2 = 5 FROM events_test_table WHERE users_test_table.user_id = events_test_table.user_id; @@ -14,61 +14,76 @@ step s1-insert_to_events_test_table: INSERT INTO events_test_table VALUES(4,6,8,10); step s2-commit: - COMMIT; + COMMIT; step s1-insert_to_events_test_table: <... completed> step s1-commit: COMMIT; +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-begin s2-begin s2-modify_with_subquery_v1 s1-update_events_test_table s2-commit s1-commit step s1-begin: BEGIN; step s2-begin: - BEGIN; + BEGIN; step s2-modify_with_subquery_v1: UPDATE users_test_table SET value_2 = 5 FROM events_test_table WHERE users_test_table.user_id = events_test_table.user_id; step s1-update_events_test_table: - UPDATE users_test_table SET value_1 = 3; + UPDATE users_test_table SET value_1 = 3; step s2-commit: - COMMIT; + COMMIT; step s1-update_events_test_table: <... completed> step s1-commit: COMMIT; +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-begin s2-begin s2-modify_with_subquery_v1 s1-delete_events_test_table s2-commit s1-commit step s1-begin: BEGIN; step s2-begin: - BEGIN; + BEGIN; step s2-modify_with_subquery_v1: UPDATE users_test_table SET value_2 = 5 FROM events_test_table WHERE users_test_table.user_id = events_test_table.user_id; step s1-delete_events_test_table: - DELETE FROM events_test_table WHERE user_id = 1 or user_id = 3; + DELETE FROM events_test_table WHERE user_id = 1 or user_id = 3; step s2-commit: - COMMIT; + COMMIT; step s1-delete_events_test_table: <... completed> step s1-commit: COMMIT; +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-begin s2-begin s1-insert_to_events_test_table s2-modify_with_subquery_v1 s1-commit s2-commit step s1-begin: BEGIN; step s2-begin: - BEGIN; + BEGIN; step s1-insert_to_events_test_table: INSERT INTO events_test_table VALUES(4,6,8,10); @@ -81,7 +96,12 @@ step s1-commit: step s2-modify_with_subquery_v1: <... completed> step s2-commit: - COMMIT; + COMMIT; + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) starting permutation: s1-begin s2-begin s1-update_events_test_table s2-modify_with_subquery_v1 s1-commit s2-commit @@ -89,10 +109,10 @@ step s1-begin: BEGIN; step s2-begin: - BEGIN; + BEGIN; step s1-update_events_test_table: - UPDATE users_test_table SET value_1 = 3; + UPDATE users_test_table SET value_1 = 3; step s2-modify_with_subquery_v1: UPDATE users_test_table SET value_2 = 5 FROM events_test_table WHERE users_test_table.user_id = events_test_table.user_id; @@ -102,7 +122,12 @@ step s1-commit: step s2-modify_with_subquery_v1: <... completed> step s2-commit: - COMMIT; + COMMIT; + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) starting permutation: s1-begin s2-begin s1-delete_events_test_table s2-modify_with_subquery_v1 s1-commit s2-commit @@ -110,10 +135,10 @@ step s1-begin: BEGIN; step s2-begin: - BEGIN; + BEGIN; step s1-delete_events_test_table: - DELETE FROM events_test_table WHERE user_id = 1 or user_id = 3; + DELETE FROM events_test_table WHERE user_id = 1 or user_id = 3; step s2-modify_with_subquery_v1: UPDATE users_test_table SET value_2 = 5 FROM events_test_table WHERE users_test_table.user_id = events_test_table.user_id; @@ -123,5 +148,10 @@ step s1-commit: step s2-modify_with_subquery_v1: <... completed> step s2-commit: - COMMIT; + COMMIT; + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) diff --git a/src/test/regress/expected/isolation_replicate_reference_tables_to_coordinator.out b/src/test/regress/expected/isolation_replicate_reference_tables_to_coordinator.out index fd24672e1..c416e62d6 100644 --- a/src/test/regress/expected/isolation_replicate_reference_tables_to_coordinator.out +++ b/src/test/regress/expected/isolation_replicate_reference_tables_to_coordinator.out @@ -75,7 +75,7 @@ step s1-update-ref-table: update ref_table set a = a + 1; step s2-sleep: - SELECT pg_sleep(0.5); + SELECT pg_sleep(0.5); pg_sleep --------------------------------------------------------------------- @@ -83,7 +83,7 @@ pg_sleep (1 row) step s2-view-dist: - SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC; + SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC; query |query_hostname |query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname --------------------------------------------------------------------- @@ -96,7 +96,7 @@ query |query_hostname |query_hostport|distribute (2 rows) step s2-view-worker: - SELECT query, query_hostname, query_hostport, distributed_query_host_name, + SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND @@ -106,10 +106,10 @@ step s2-view-worker: query |query_hostname |query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname --------------------------------------------------------------------- -UPDATE public.ref_table_1500379 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57638|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression -UPDATE public.ref_table_1500379 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57637|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression -UPDATE public.ref_table_1500379 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|coordinator_host| 57636|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression -UPDATE public.ref_table_1500379 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57636|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression +UPDATE public.ref_table_1500775 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57638|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression +UPDATE public.ref_table_1500775 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57637|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression +UPDATE public.ref_table_1500775 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|coordinator_host| 57636|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression +UPDATE public.ref_table_1500775 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57636|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression (4 rows) step s2-end: @@ -140,9 +140,9 @@ step s1-update-ref-table: update ref_table set a = a + 1; step s2-active-transactions: - -- Admin should be able to see all transactions - SELECT count(*) FROM get_all_active_transactions(); - SELECT count(*) FROM get_global_active_transactions(); + -- Admin should be able to see all transactions + SELECT count(*) FROM get_all_active_transactions(); + SELECT count(*) FROM get_global_active_transactions(); count --------------------------------------------------------------------- diff --git a/src/test/regress/expected/isolation_replicated_dist_on_mx.out b/src/test/regress/expected/isolation_replicated_dist_on_mx.out new file mode 100644 index 000000000..646783bae --- /dev/null +++ b/src/test/regress/expected/isolation_replicated_dist_on_mx.out @@ -0,0 +1,2843 @@ +Parsed test spec with 2 sessions + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-update-1-rep-table s2-start-session-level-connection s2-begin-on-worker s2-update-1-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-update-1-rep-table: + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12 WHERE user_id = 1'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-1-rep-table: + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12 WHERE user_id = 1'); + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-1-rep-table: <... completed> +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-update-1-rep-table s2-start-session-level-connection s2-begin-on-worker s2-update-all-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-update-1-rep-table: + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12 WHERE user_id = 1'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-all-rep-table: + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12'); + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-all-rep-table: <... completed> +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-update-all-rep-table s2-start-session-level-connection s2-begin-on-worker s2-update-all-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-update-all-rep-table: + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-all-rep-table: + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12'); + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-all-rep-table: <... completed> +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-update-all-rep-table s2-start-session-level-connection s2-begin-on-worker s2-update-1-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-update-all-rep-table: + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-1-rep-table: + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12 WHERE user_id = 1'); + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-1-rep-table: <... completed> +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-delete-1-from-rep-table s2-start-session-level-connection s2-begin-on-worker s2-update-1-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-delete-1-from-rep-table: + SELECT run_commands_on_session_level_connection_to_node('DELETE FROM replicated_table WHERE user_id = 1'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-1-rep-table: + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12 WHERE user_id = 1'); + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-1-rep-table: <... completed> +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-delete-all-from-rep-table s2-start-session-level-connection s2-begin-on-worker s2-update-1-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-delete-all-from-rep-table: + SELECT run_commands_on_session_level_connection_to_node('DELETE FROM replicated_table'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-1-rep-table: + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12 WHERE user_id = 1'); + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-1-rep-table: <... completed> +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-delete-1-from-rep-table s2-start-session-level-connection s2-begin-on-worker s2-update-all-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-delete-1-from-rep-table: + SELECT run_commands_on_session_level_connection_to_node('DELETE FROM replicated_table WHERE user_id = 1'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-all-rep-table: + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12'); + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-all-rep-table: <... completed> +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-delete-all-from-rep-table s2-start-session-level-connection s2-begin-on-worker s2-update-1-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-delete-all-from-rep-table: + SELECT run_commands_on_session_level_connection_to_node('DELETE FROM replicated_table'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-1-rep-table: + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12 WHERE user_id = 1'); + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-1-rep-table: <... completed> +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-insert-into-1-rep-table s2-start-session-level-connection s2-begin-on-worker s2-update-1-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-insert-into-1-rep-table: + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table VALUES(1,81)'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-1-rep-table: + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12 WHERE user_id = 1'); + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-1-rep-table: <... completed> +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-insert-into-1-rep-table s2-start-session-level-connection s2-begin-on-worker s2-update-all-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-insert-into-1-rep-table: + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table VALUES(1,81)'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-all-rep-table: + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12'); + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-all-rep-table: <... completed> +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-insert-into-all-rep-table s2-start-session-level-connection s2-begin-on-worker s2-update-1-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-insert-into-all-rep-table: + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table VALUES(8,81),(9,91),(10,91),(11,91),(12,91), (13,91), (14,91), (15,91), (16,91), (17,91), (18,91), (19,91), (20,91)'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-1-rep-table: + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12 WHERE user_id = 1'); + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-1-rep-table: <... completed> +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-insert-into-all-rep-table s2-start-session-level-connection s2-begin-on-worker s2-update-all-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-insert-into-all-rep-table: + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table VALUES(8,81),(9,91),(10,91),(11,91),(12,91), (13,91), (14,91), (15,91), (16,91), (17,91), (18,91), (19,91), (20,91)'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-all-rep-table: + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12'); + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-all-rep-table: <... completed> +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-update-1-rep-table s2-start-session-level-connection s2-begin-on-worker s2-insert-into-1-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-update-1-rep-table: + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12 WHERE user_id = 1'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-insert-into-1-rep-table: + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table VALUES(1,81)'); + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-insert-into-1-rep-table: <... completed> +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-update-1-rep-table s2-start-session-level-connection s2-begin-on-worker s2-insert-into-all-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-update-1-rep-table: + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12 WHERE user_id = 1'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-insert-into-all-rep-table: + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table VALUES(8,81),(9,91),(10,91),(11,91),(12,91), (13,91), (14,91), (15,91), (16,91), (17,91), (18,91), (19,91), (20,91)'); + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-insert-into-all-rep-table: <... completed> +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-update-all-rep-table s2-start-session-level-connection s2-begin-on-worker s2-insert-into-1-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-update-all-rep-table: + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-insert-into-1-rep-table: + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table VALUES(1,81)'); + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-insert-into-1-rep-table: <... completed> +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-update-all-rep-table s2-start-session-level-connection s2-begin-on-worker s2-insert-into-all-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-update-all-rep-table: + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-insert-into-all-rep-table: + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table VALUES(8,81),(9,91),(10,91),(11,91),(12,91), (13,91), (14,91), (15,91), (16,91), (17,91), (18,91), (19,91), (20,91)'); + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-insert-into-all-rep-table: <... completed> +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-insert-into-1-rep-table s2-start-session-level-connection s2-begin-on-worker s2-insert-into-1-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-insert-into-1-rep-table: + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table VALUES(1,81)'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-insert-into-1-rep-table: + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table VALUES(1,81)'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-insert-into-1-rep-table s2-start-session-level-connection s2-begin-on-worker s2-insert-into-all-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-insert-into-1-rep-table: + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table VALUES(1,81)'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-insert-into-all-rep-table: + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table VALUES(8,81),(9,91),(10,91),(11,91),(12,91), (13,91), (14,91), (15,91), (16,91), (17,91), (18,91), (19,91), (20,91)'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-insert-into-all-rep-table s2-start-session-level-connection s2-begin-on-worker s2-insert-into-1-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-insert-into-all-rep-table: + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table VALUES(8,81),(9,91),(10,91),(11,91),(12,91), (13,91), (14,91), (15,91), (16,91), (17,91), (18,91), (19,91), (20,91)'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-insert-into-1-rep-table: + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table VALUES(1,81)'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-insert-into-all-rep-table s2-start-session-level-connection s2-begin-on-worker s2-insert-into-all-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-insert-into-all-rep-table: + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table VALUES(8,81),(9,91),(10,91),(11,91),(12,91), (13,91), (14,91), (15,91), (16,91), (17,91), (18,91), (19,91), (20,91)'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-insert-into-all-rep-table: + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table VALUES(8,81),(9,91),(10,91),(11,91),(12,91), (13,91), (14,91), (15,91), (16,91), (17,91), (18,91), (19,91), (20,91)'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-copy-1-to-rep-table s2-start-session-level-connection s2-begin-on-worker s2-update-1-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-copy-1-to-rep-table: + SELECT run_commands_on_session_level_connection_to_node('COPY replicated_table FROM PROGRAM ''echo 1, 101 && echo 1, 111 && echo 1,1111'' WITH CSV'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-1-rep-table: + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12 WHERE user_id = 1'); + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-1-rep-table: <... completed> +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-copy-1-to-rep-table s2-start-session-level-connection s2-begin-on-worker s2-update-all-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-copy-1-to-rep-table: + SELECT run_commands_on_session_level_connection_to_node('COPY replicated_table FROM PROGRAM ''echo 1, 101 && echo 1, 111 && echo 1,1111'' WITH CSV'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-all-rep-table: + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12'); + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-all-rep-table: <... completed> +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-copy-all-to-rep-table s2-start-session-level-connection s2-begin-on-worker s2-update-1-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-copy-all-to-rep-table: + SELECT run_commands_on_session_level_connection_to_node('COPY replicated_table FROM PROGRAM ''echo 10, 101 && echo 11, 111 && echo 11, 111 && echo 12, 111 && echo 13, 111 && echo 14, 111 && echo 15, 111 && echo 16, 111 && echo 17, 111 && echo 18, 111 && echo 19, 111 && echo 20, 111 && echo 21, 111 && echo 22, 111 && echo 23, 111'' WITH CSV'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-1-rep-table: + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12 WHERE user_id = 1'); + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-1-rep-table: <... completed> +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-copy-all-to-rep-table s2-start-session-level-connection s2-begin-on-worker s2-update-all-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-copy-all-to-rep-table: + SELECT run_commands_on_session_level_connection_to_node('COPY replicated_table FROM PROGRAM ''echo 10, 101 && echo 11, 111 && echo 11, 111 && echo 12, 111 && echo 13, 111 && echo 14, 111 && echo 15, 111 && echo 16, 111 && echo 17, 111 && echo 18, 111 && echo 19, 111 && echo 20, 111 && echo 21, 111 && echo 22, 111 && echo 23, 111'' WITH CSV'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-all-rep-table: + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12'); + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-all-rep-table: <... completed> +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-copy-all-to-rep-table s2-start-session-level-connection s2-begin-on-worker s2-insert-into-all-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-copy-all-to-rep-table: + SELECT run_commands_on_session_level_connection_to_node('COPY replicated_table FROM PROGRAM ''echo 10, 101 && echo 11, 111 && echo 11, 111 && echo 12, 111 && echo 13, 111 && echo 14, 111 && echo 15, 111 && echo 16, 111 && echo 17, 111 && echo 18, 111 && echo 19, 111 && echo 20, 111 && echo 21, 111 && echo 22, 111 && echo 23, 111'' WITH CSV'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-insert-into-all-rep-table: + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table VALUES(8,81),(9,91),(10,91),(11,91),(12,91), (13,91), (14,91), (15,91), (16,91), (17,91), (18,91), (19,91), (20,91)'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-copy-1-to-rep-table s2-start-session-level-connection s2-begin-on-worker s2-insert-into-1-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-copy-1-to-rep-table: + SELECT run_commands_on_session_level_connection_to_node('COPY replicated_table FROM PROGRAM ''echo 1, 101 && echo 1, 111 && echo 1,1111'' WITH CSV'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-insert-into-1-rep-table: + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table VALUES(1,81)'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-copy-all-to-rep-table s2-start-session-level-connection s2-begin-on-worker s2-copy-all-to-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-copy-all-to-rep-table: + SELECT run_commands_on_session_level_connection_to_node('COPY replicated_table FROM PROGRAM ''echo 10, 101 && echo 11, 111 && echo 11, 111 && echo 12, 111 && echo 13, 111 && echo 14, 111 && echo 15, 111 && echo 16, 111 && echo 17, 111 && echo 18, 111 && echo 19, 111 && echo 20, 111 && echo 21, 111 && echo 22, 111 && echo 23, 111'' WITH CSV'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-copy-all-to-rep-table: + SELECT run_commands_on_session_level_connection_to_node('COPY replicated_table FROM PROGRAM ''echo 10, 101 && echo 11, 111 && echo 11, 111 && echo 12, 111 && echo 13, 111 && echo 14, 111 && echo 15, 111 && echo 16, 111 && echo 17, 111 && echo 18, 111 && echo 19, 111 && echo 20, 111 && echo 21, 111 && echo 22, 111 && echo 23, 111'' WITH CSV'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-copy-all-to-rep-table s2-start-session-level-connection s2-begin-on-worker s2-select-from-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-copy-all-to-rep-table: + SELECT run_commands_on_session_level_connection_to_node('COPY replicated_table FROM PROGRAM ''echo 10, 101 && echo 11, 111 && echo 11, 111 && echo 12, 111 && echo 13, 111 && echo 14, 111 && echo 15, 111 && echo 16, 111 && echo 17, 111 && echo 18, 111 && echo 19, 111 && echo 20, 111 && echo 21, 111 && echo 22, 111 && echo 23, 111'' WITH CSV'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-select-from-rep-table: + SELECT run_commands_on_session_level_connection_to_node('SELECT count(*) FROM replicated_table'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s2-start-session-level-connection s2-begin-on-worker s2-insert-into-1-rep-table s1-begin s1-alter-table s2-commit-worker s1-commit s2-stop-connection +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-insert-into-1-rep-table: + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table VALUES(1,81)'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin: + BEGIN; + +step s1-alter-table: + ALTER TABLE replicated_table ADD COLUMN x INT; + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-alter-table: <... completed> +step s1-commit: + COMMIT; + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s2-start-session-level-connection s2-begin-on-worker s2-select-from-rep-table s1-begin s1-alter-table s2-commit-worker s1-commit s2-stop-connection +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-select-from-rep-table: + SELECT run_commands_on_session_level_connection_to_node('SELECT count(*) FROM replicated_table'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin: + BEGIN; + +step s1-alter-table: + ALTER TABLE replicated_table ADD COLUMN x INT; + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-alter-table: <... completed> +step s1-commit: + COMMIT; + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-insert-into-select s2-start-session-level-connection s2-begin-on-worker s2-insert-into-all-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-insert-into-select: + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table SELECT * FROM replicated_table_2'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-insert-into-all-rep-table: + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table VALUES(8,81),(9,91),(10,91),(11,91),(12,91), (13,91), (14,91), (15,91), (16,91), (17,91), (18,91), (19,91), (20,91)'); + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-insert-into-all-rep-table: <... completed> +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-insert-into-select s2-start-session-level-connection s2-begin-on-worker s2-insert-into-1-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-insert-into-select: + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table SELECT * FROM replicated_table_2'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-insert-into-1-rep-table: + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table VALUES(1,81)'); + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-insert-into-1-rep-table: <... completed> +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + + +starting permutation: s1-start-session-level-connection s1-begin-on-worker s1-insert-into-select-from-single-rep s2-start-session-level-connection s2-begin-on-worker s2-update-all-single-rep-table s2-update-all-rep-table s1-commit-worker s2-commit-worker s1-stop-connection s2-stop-connection +step s1-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57637); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-insert-into-select-from-single-rep: + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table SELECT * FROM single_replicated_table LIMIT 10'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-start-session-level-connection: + SELECT start_session_level_connection_to_node('localhost', 57638); + +start_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-begin-on-worker: + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-all-single-rep-table: + SELECT run_commands_on_session_level_connection_to_node('UPDATE single_replicated_table SET value_1 = 12'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-all-rep-table: + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12'); + +step s1-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-update-all-rep-table: <... completed> +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-commit-worker: + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); + +run_commands_on_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s1-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +step s2-stop-connection: + SELECT stop_session_level_connection_to_node(); + +stop_session_level_connection_to_node +--------------------------------------------------------------------- + +(1 row) + +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + diff --git a/src/test/regress/expected/isolation_undistribute_table.out b/src/test/regress/expected/isolation_undistribute_table.out index 185af05c5..d861b5af8 100644 --- a/src/test/regress/expected/isolation_undistribute_table.out +++ b/src/test/regress/expected/isolation_undistribute_table.out @@ -20,6 +20,11 @@ step s1-commit: step s2-undistribute: <... completed> ERROR: cannot complete operation because no such table exists +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-begin s1-undistribute s2-select s1-commit step s1-begin: @@ -47,6 +52,11 @@ a|b 5|6 (3 rows) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-begin s1-undistribute s2-insert s1-commit s2-select step s1-begin: @@ -79,6 +89,11 @@ a| b 9|10 (5 rows) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-begin s1-undistribute s2-insert-select s1-commit s2-select step s1-begin: @@ -112,6 +127,11 @@ a|b 5|6 (6 rows) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-begin s1-undistribute s2-delete s1-commit s2-select step s1-begin: @@ -141,6 +161,11 @@ a|b 5|6 (2 rows) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-begin s1-undistribute s2-copy s1-commit s2-select step s1-begin: @@ -173,6 +198,11 @@ step s2-select: 13|14 (5 rows) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-begin s1-undistribute s2-drop s1-commit s2-select step s1-begin: @@ -197,6 +227,11 @@ step s2-select: SELECT * FROM dist_table ORDER BY 1, 2; ERROR: relation "dist_table" does not exist +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-begin s1-undistribute s2-truncate s1-commit s2-select step s1-begin: @@ -224,6 +259,11 @@ a|b --------------------------------------------------------------------- (0 rows) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-begin s1-undistribute s2-select-for-update s1-commit step s1-begin: @@ -249,6 +289,11 @@ a|b 5|6 (1 row) +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + starting permutation: s1-begin s1-undistribute s2-create-index-concurrently s1-commit step s1-begin: @@ -269,3 +314,8 @@ step s1-commit: COMMIT; step s2-create-index-concurrently: <... completed> +restore_isolation_tester_func +--------------------------------------------------------------------- + +(1 row) + diff --git a/src/test/regress/expected/local_shard_execution_replicated.out b/src/test/regress/expected/local_shard_execution_replicated.out new file mode 100644 index 000000000..7e352b097 --- /dev/null +++ b/src/test/regress/expected/local_shard_execution_replicated.out @@ -0,0 +1,2388 @@ +CREATE SCHEMA local_shard_execution_replicated; +SET search_path TO local_shard_execution_replicated; +SET citus.shard_count TO 4; +SET citus.shard_replication_factor TO 2; +SET citus.next_shard_id TO 1500000; +CREATE TABLE reference_table (key int PRIMARY KEY); +SELECT create_reference_table('reference_table'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE distributed_table (key int PRIMARY KEY , value text, age bigint CHECK (age > 10)); +SELECT create_distributed_table('distributed_table','key'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE second_distributed_table (key int PRIMARY KEY , value text); +SELECT create_distributed_table('second_distributed_table','key'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- ingest some data to enable some tests with data +INSERT INTO distributed_table VALUES (1, '1', 20); +-- This GUC prevents to acquire the remote lock for replicated +-- tables +BEGIN; + SET LOCAL citus.allow_modifications_from_workers_to_replicated_tables TO false; + INSERT INTO second_distributed_table VALUES (1, '1'); + INSERT INTO reference_table VALUES (1); +COMMIT; +-- a simple test for +CREATE TABLE collections_list ( + key bigserial, + ser bigserial, + ts timestamptz, + collection_id integer, + value numeric, + PRIMARY KEY(key, collection_id) +) PARTITION BY LIST (collection_id ); +SELECT create_distributed_table('collections_list', 'key'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE collections_list_0 + PARTITION OF collections_list (key, ser, ts, collection_id, value) + FOR VALUES IN ( 0 ); +-- create a volatile function that returns the local node id +CREATE OR REPLACE FUNCTION get_local_node_id_volatile() +RETURNS INT AS $$ +DECLARE localGroupId int; +BEGIN + SELECT groupid INTO localGroupId FROM pg_dist_local_group; + RETURN localGroupId; +END; $$ language plpgsql VOLATILE; +SELECT create_distributed_function('get_local_node_id_volatile()'); + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +-- test case for issue #3556 +CREATE TABLE accounts (id text PRIMARY KEY); +CREATE TABLE stats (account_id text PRIMARY KEY, spent int); +SELECT create_distributed_table('accounts', 'id', colocate_with => 'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('stats', 'account_id', colocate_with => 'accounts'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO accounts (id) VALUES ('foo'); +INSERT INTO stats (account_id, spent) VALUES ('foo', 100); +CREATE TABLE abcd(a int, b int, c int, d int); +SELECT create_distributed_table('abcd', 'b'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO abcd VALUES (1,2,3,4); +INSERT INTO abcd VALUES (2,3,4,5); +INSERT INTO abcd VALUES (3,4,5,6); +ALTER TABLE abcd DROP COLUMN a; +-- connection worker and get ready for the tests +\c - - - :worker_1_port +SET search_path TO local_shard_execution_replicated; +-- test case for issue #3556 +SET citus.log_intermediate_results TO TRUE; +SET client_min_messages TO DEBUG1; +SELECT * +FROM +( + WITH accounts_cte AS ( + SELECT id AS account_id + FROM accounts + ), + joined_stats_cte_1 AS ( + SELECT spent, account_id + FROM stats + INNER JOIN accounts_cte USING (account_id) + ), + joined_stats_cte_2 AS ( + SELECT spent, account_id + FROM joined_stats_cte_1 + INNER JOIN accounts_cte USING (account_id) + ) + SELECT SUM(spent) OVER (PARTITION BY coalesce(account_id, NULL)) + FROM accounts_cte + INNER JOIN joined_stats_cte_2 USING (account_id) +) inner_query; +DEBUG: CTE joined_stats_cte_1 is going to be inlined via distributed planning +DEBUG: CTE joined_stats_cte_2 is going to be inlined via distributed planning +DEBUG: generating subplan XXX_1 for CTE accounts_cte: SELECT id AS account_id FROM local_shard_execution_replicated.accounts +DEBUG: generating subplan XXX_2 for subquery SELECT sum(joined_stats_cte_2.spent) OVER (PARTITION BY COALESCE(accounts_cte.account_id, NULL::text)) AS sum FROM ((SELECT intermediate_result.account_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(account_id text)) accounts_cte JOIN (SELECT joined_stats_cte_1.spent, joined_stats_cte_1.account_id FROM ((SELECT stats.spent, stats.account_id FROM (local_shard_execution_replicated.stats JOIN (SELECT intermediate_result.account_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(account_id text)) accounts_cte_2 USING (account_id))) joined_stats_cte_1 JOIN (SELECT intermediate_result.account_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(account_id text)) accounts_cte_1 USING (account_id))) joined_stats_cte_2 USING (account_id)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT sum FROM (SELECT intermediate_result.sum FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(sum bigint)) inner_query +DEBUG: Subplan XXX_1 will be sent to localhost:xxxxx +DEBUG: Subplan XXX_1 will be sent to localhost:xxxxx +DEBUG: Subplan XXX_2 will be written to local file + sum +--------------------------------------------------------------------- + 100 +(1 row) + +SET citus.log_intermediate_results TO DEFAULT; +SET client_min_messages TO DEFAULT; +--- enable logging to see which tasks are executed locally +SET citus.log_local_commands TO ON; +-- first, make sure that local execution works fine +-- with simple queries that are not in transcation blocks +SELECT count(*) FROM distributed_table WHERE key = 1; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) + count +--------------------------------------------------------------------- + 1 +(1 row) + +-- multiple tasks both of which are local should NOT use local execution +-- because local execution means executing the tasks locally, so the executor +-- favors parallel execution even if everyting is local to node +SELECT count(*) FROM distributed_table WHERE key IN (1,6); + count +--------------------------------------------------------------------- + 1 +(1 row) + +-- queries that hit any remote shards should NOT use local execution +SELECT count(*) FROM distributed_table WHERE key IN (1,11); + count +--------------------------------------------------------------------- + 1 +(1 row) + +SELECT count(*) FROM distributed_table; + count +--------------------------------------------------------------------- + 1 +(1 row) + +-- modifications also follow the same rules +INSERT INTO reference_table VALUES (1) ON CONFLICT DO NOTHING; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.reference_table_1500000 AS citus_table_alias (key) VALUES (1) ON CONFLICT DO NOTHING +INSERT INTO distributed_table VALUES (1, '1', 21) ON CONFLICT DO NOTHING; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1, '1'::text, 21) ON CONFLICT DO NOTHING +-- local query +DELETE FROM distributed_table WHERE key = 1 AND age = 21; +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE ((key OPERATOR(pg_catalog.=) 1) AND (age OPERATOR(pg_catalog.=) 21)) +-- hitting multiple shards, so should be a distributed execution +DELETE FROM distributed_table WHERE age = 21; +-- although both shards are local, the executor choose the parallel execution +-- over the wire because as noted above local execution is sequential +DELETE FROM second_distributed_table WHERE key IN (1,6); +-- similarly, any multi-shard query just follows distributed execution +DELETE FROM second_distributed_table; +-- load some more data for the following tests +INSERT INTO second_distributed_table VALUES (1, '1'); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.second_distributed_table_1500005 (key, value) VALUES (1, '1'::text) +-- INSERT .. SELECT hitting a single single (co-located) shard(s) should +-- be executed locally +INSERT INTO distributed_table +SELECT + distributed_table.* +FROM + distributed_table, second_distributed_table +WHERE + distributed_table.key = 1 and distributed_table.key=second_distributed_table.key +ON CONFLICT(key) DO UPDATE SET value = '22' +RETURNING *; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) SELECT distributed_table.key, distributed_table.value, distributed_table.age FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table, local_shard_execution_replicated.second_distributed_table_1500005 second_distributed_table WHERE (((distributed_table.key OPERATOR(pg_catalog.=) 1) AND (distributed_table.key OPERATOR(pg_catalog.=) second_distributed_table.key)) AND (distributed_table.key IS NOT NULL)) ON CONFLICT(key) DO UPDATE SET value = '22'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age + key | value | age +--------------------------------------------------------------------- + 1 | 22 | 20 +(1 row) + +-- INSERT .. SELECT hitting multi-shards should go thourgh distributed execution +INSERT INTO distributed_table +SELECT + distributed_table.* +FROM + distributed_table, second_distributed_table +WHERE + distributed_table.key != 1 and distributed_table.key=second_distributed_table.key +ON CONFLICT(key) DO UPDATE SET value = '22' +RETURNING *; + key | value | age +--------------------------------------------------------------------- +(0 rows) + +-- INSERT..SELECT via coordinator consists of two steps, select + COPY +-- that's why it is disallowed to use local execution even if the SELECT +-- can be executed locally +INSERT INTO distributed_table SELECT * FROM distributed_table WHERE key = 1 OFFSET 0 ON CONFLICT DO NOTHING; +NOTICE: executing the command locally: SELECT key, value, age FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) OFFSET 0 +NOTICE: executing the copy locally for colocated file with shard xxxxx +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) SELECT key, value, age FROM read_intermediate_result('insert_select_XXX_1500001'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, age bigint) ON CONFLICT DO NOTHING +INSERT INTO distributed_table SELECT 1, '1',15 FROM distributed_table WHERE key = 2 LIMIT 1 ON CONFLICT DO NOTHING; +NOTICE: executing the command locally: SELECT 1 AS key, '1'::text AS value, int8(15) AS age FROM local_shard_execution_replicated.distributed_table_1500004 distributed_table WHERE (key OPERATOR(pg_catalog.=) 2) LIMIT 1 +-- sanity check: multi-shard INSERT..SELECT pushdown goes through distributed execution +INSERT INTO distributed_table SELECT * FROM distributed_table ON CONFLICT DO NOTHING; +-- Ensure tuple data in explain analyze output is the same on all PG versions +SET citus.enable_binary_protocol = TRUE; +-- EXPLAIN for local execution just works fine +-- though going through distributed execution +EXPLAIN (COSTS OFF) SELECT * FROM distributed_table WHERE key = 1 AND age = 20; + QUERY PLAN +--------------------------------------------------------------------- + Custom Scan (Citus Adaptive) + Task Count: 1 + Tasks Shown: All + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> Index Scan using distributed_table_pkey_1500001 on distributed_table_1500001 distributed_table + Index Cond: (key = 1) + Filter: (age = 20) +(8 rows) + +EXPLAIN (ANALYZE, COSTS OFF, SUMMARY OFF, TIMING OFF) SELECT * FROM distributed_table WHERE key = 1 AND age = 20; + QUERY PLAN +--------------------------------------------------------------------- + Custom Scan (Citus Adaptive) (actual rows=1 loops=1) + Task Count: 1 + Tuple data received from nodes: 14 bytes + Tasks Shown: All + -> Task + Tuple data received from node: 14 bytes + Node: host=localhost port=xxxxx dbname=regression + -> Index Scan using distributed_table_pkey_1500001 on distributed_table_1500001 distributed_table (actual rows=1 loops=1) + Index Cond: (key = 1) + Filter: (age = 20) +(10 rows) + +EXPLAIN (ANALYZE ON, COSTS OFF, SUMMARY OFF, TIMING OFF) +WITH r AS ( SELECT GREATEST(random(), 2) z,* FROM distributed_table) +SELECT 1 FROM r WHERE z < 3; + QUERY PLAN +--------------------------------------------------------------------- + Custom Scan (Citus Adaptive) (actual rows=1 loops=1) + -> Distributed Subplan XXX_1 + Intermediate Data Size: 40 bytes + Result destination: Write locally + -> Custom Scan (Citus Adaptive) (actual rows=1 loops=1) + Task Count: 4 + Tuple data received from nodes: 22 bytes + Tasks Shown: One of 4 + -> Task + Tuple data received from node: 22 bytes + Node: host=localhost port=xxxxx dbname=regression + -> Seq Scan on distributed_table_1500001 distributed_table (actual rows=1 loops=1) + Task Count: 1 + Tuple data received from nodes: 4 bytes + Tasks Shown: All + -> Task + Tuple data received from node: 4 bytes + Node: host=localhost port=xxxxx dbname=regression + -> Function Scan on read_intermediate_result intermediate_result (actual rows=1 loops=1) + Filter: (z < '3'::double precision) +(20 rows) + +EXPLAIN (COSTS OFF) DELETE FROM distributed_table WHERE key = 1 AND age = 20; + QUERY PLAN +--------------------------------------------------------------------- + Custom Scan (Citus Adaptive) + Task Count: 1 + Tasks Shown: All + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> Delete on distributed_table_1500001 distributed_table + -> Index Scan using distributed_table_pkey_1500001 on distributed_table_1500001 distributed_table + Index Cond: (key = 1) + Filter: (age = 20) +(9 rows) + +EXPLAIN (ANALYZE, COSTS OFF, SUMMARY OFF, TIMING OFF) DELETE FROM distributed_table WHERE key = 1 AND age = 20; + QUERY PLAN +--------------------------------------------------------------------- + Custom Scan (Citus Adaptive) (actual rows=0 loops=1) + Task Count: 1 + Tasks Shown: All + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> Delete on distributed_table_1500001 distributed_table (actual rows=0 loops=1) + -> Index Scan using distributed_table_pkey_1500001 on distributed_table_1500001 distributed_table (actual rows=1 loops=1) + Index Cond: (key = 1) + Filter: (age = 20) +(9 rows) + +-- show that EXPLAIN ANALYZE deleted the row +SELECT * FROM distributed_table WHERE key = 1 AND age = 20 ORDER BY 1,2,3; +NOTICE: executing the command locally: SELECT key, value, age FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE ((key OPERATOR(pg_catalog.=) 1) AND (age OPERATOR(pg_catalog.=) 20)) ORDER BY key, value, age + key | value | age +--------------------------------------------------------------------- +(0 rows) + +SELECT * FROM second_distributed_table WHERE key = 1 ORDER BY 1,2; +NOTICE: executing the command locally: SELECT key, value FROM local_shard_execution_replicated.second_distributed_table_1500005 second_distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) ORDER BY key, value + key | value +--------------------------------------------------------------------- + 1 | 1 +(1 row) + +-- Put row back for other tests +INSERT INTO distributed_table VALUES (1, '22', 20); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 (key, value, age) VALUES (1, '22'::text, 20) +CREATE VIEW abcd_view AS SELECT * FROM abcd; +SELECT * FROM abcd first join abcd second on first.b = second.b ORDER BY 1,2,3,4; + b | c | d | b | c | d +--------------------------------------------------------------------- + 2 | 3 | 4 | 2 | 3 | 4 + 3 | 4 | 5 | 3 | 4 | 5 + 4 | 5 | 6 | 4 | 5 | 6 +(3 rows) + +BEGIN; +SELECT * FROM abcd first join abcd second on first.b = second.b ORDER BY 1,2,3,4; +NOTICE: executing the command locally: SELECT first.b, first.c, first.d, second.b, second.c, second.d FROM (local_shard_execution_replicated.abcd_1500025 first JOIN local_shard_execution_replicated.abcd_1500025 second ON ((first.b OPERATOR(pg_catalog.=) second.b))) WHERE true +NOTICE: executing the command locally: SELECT first.b, first.c, first.d, second.b, second.c, second.d FROM (local_shard_execution_replicated.abcd_1500026 first JOIN local_shard_execution_replicated.abcd_1500026 second ON ((first.b OPERATOR(pg_catalog.=) second.b))) WHERE true +NOTICE: executing the command locally: SELECT first.b, first.c, first.d, second.b, second.c, second.d FROM (local_shard_execution_replicated.abcd_1500027 first JOIN local_shard_execution_replicated.abcd_1500027 second ON ((first.b OPERATOR(pg_catalog.=) second.b))) WHERE true +NOTICE: executing the command locally: SELECT first.b, first.c, first.d, second.b, second.c, second.d FROM (local_shard_execution_replicated.abcd_1500028 first JOIN local_shard_execution_replicated.abcd_1500028 second ON ((first.b OPERATOR(pg_catalog.=) second.b))) WHERE true + b | c | d | b | c | d +--------------------------------------------------------------------- + 2 | 3 | 4 | 2 | 3 | 4 + 3 | 4 | 5 | 3 | 4 | 5 + 4 | 5 | 6 | 4 | 5 | 6 +(3 rows) + +END; +BEGIN; +SELECT * FROM abcd_view first join abcd_view second on first.b = second.b ORDER BY 1,2,3,4; +NOTICE: executing the command locally: SELECT abcd.b, abcd.c, abcd.d, abcd_1.b, abcd_1.c, abcd_1.d FROM (local_shard_execution_replicated.abcd_1500025 abcd JOIN local_shard_execution_replicated.abcd_1500025 abcd_1 ON ((abcd.b OPERATOR(pg_catalog.=) abcd_1.b))) WHERE true +NOTICE: executing the command locally: SELECT abcd.b, abcd.c, abcd.d, abcd_1.b, abcd_1.c, abcd_1.d FROM (local_shard_execution_replicated.abcd_1500026 abcd JOIN local_shard_execution_replicated.abcd_1500026 abcd_1 ON ((abcd.b OPERATOR(pg_catalog.=) abcd_1.b))) WHERE true +NOTICE: executing the command locally: SELECT abcd.b, abcd.c, abcd.d, abcd_1.b, abcd_1.c, abcd_1.d FROM (local_shard_execution_replicated.abcd_1500027 abcd JOIN local_shard_execution_replicated.abcd_1500027 abcd_1 ON ((abcd.b OPERATOR(pg_catalog.=) abcd_1.b))) WHERE true +NOTICE: executing the command locally: SELECT abcd.b, abcd.c, abcd.d, abcd_1.b, abcd_1.c, abcd_1.d FROM (local_shard_execution_replicated.abcd_1500028 abcd JOIN local_shard_execution_replicated.abcd_1500028 abcd_1 ON ((abcd.b OPERATOR(pg_catalog.=) abcd_1.b))) WHERE true + b | c | d | b | c | d +--------------------------------------------------------------------- + 2 | 3 | 4 | 2 | 3 | 4 + 3 | 4 | 5 | 3 | 4 | 5 + 4 | 5 | 6 | 4 | 5 | 6 +(3 rows) + +END; +BEGIN; +SELECT * FROM abcd first full join abcd second on first.b = second.b ORDER BY 1,2,3,4; +NOTICE: executing the command locally: SELECT worker_column_1 AS b, worker_column_2 AS c, worker_column_3 AS d, worker_column_4 AS b, worker_column_5 AS c, worker_column_6 AS d FROM (SELECT first.b AS worker_column_1, first.c AS worker_column_2, first.d AS worker_column_3, second.b AS worker_column_4, second.c AS worker_column_5, second.d AS worker_column_6 FROM (local_shard_execution_replicated.abcd_1500025 first FULL JOIN local_shard_execution_replicated.abcd_1500025 second ON ((first.b OPERATOR(pg_catalog.=) second.b)))) worker_subquery +NOTICE: executing the command locally: SELECT worker_column_1 AS b, worker_column_2 AS c, worker_column_3 AS d, worker_column_4 AS b, worker_column_5 AS c, worker_column_6 AS d FROM (SELECT first.b AS worker_column_1, first.c AS worker_column_2, first.d AS worker_column_3, second.b AS worker_column_4, second.c AS worker_column_5, second.d AS worker_column_6 FROM (local_shard_execution_replicated.abcd_1500026 first FULL JOIN local_shard_execution_replicated.abcd_1500026 second ON ((first.b OPERATOR(pg_catalog.=) second.b)))) worker_subquery +NOTICE: executing the command locally: SELECT worker_column_1 AS b, worker_column_2 AS c, worker_column_3 AS d, worker_column_4 AS b, worker_column_5 AS c, worker_column_6 AS d FROM (SELECT first.b AS worker_column_1, first.c AS worker_column_2, first.d AS worker_column_3, second.b AS worker_column_4, second.c AS worker_column_5, second.d AS worker_column_6 FROM (local_shard_execution_replicated.abcd_1500027 first FULL JOIN local_shard_execution_replicated.abcd_1500027 second ON ((first.b OPERATOR(pg_catalog.=) second.b)))) worker_subquery +NOTICE: executing the command locally: SELECT worker_column_1 AS b, worker_column_2 AS c, worker_column_3 AS d, worker_column_4 AS b, worker_column_5 AS c, worker_column_6 AS d FROM (SELECT first.b AS worker_column_1, first.c AS worker_column_2, first.d AS worker_column_3, second.b AS worker_column_4, second.c AS worker_column_5, second.d AS worker_column_6 FROM (local_shard_execution_replicated.abcd_1500028 first FULL JOIN local_shard_execution_replicated.abcd_1500028 second ON ((first.b OPERATOR(pg_catalog.=) second.b)))) worker_subquery + b | c | d | b | c | d +--------------------------------------------------------------------- + 2 | 3 | 4 | 2 | 3 | 4 + 3 | 4 | 5 | 3 | 4 | 5 + 4 | 5 | 6 | 4 | 5 | 6 +(3 rows) + +END; +BEGIN; +SELECT * FROM abcd first join abcd second USING(b) ORDER BY 1,2,3,4; +NOTICE: executing the command locally: SELECT first.b, first.c, first.d, second.c, second.d FROM (local_shard_execution_replicated.abcd_1500025 first JOIN local_shard_execution_replicated.abcd_1500025 second ON ((first.b OPERATOR(pg_catalog.=) second.b))) WHERE true +NOTICE: executing the command locally: SELECT first.b, first.c, first.d, second.c, second.d FROM (local_shard_execution_replicated.abcd_1500026 first JOIN local_shard_execution_replicated.abcd_1500026 second ON ((first.b OPERATOR(pg_catalog.=) second.b))) WHERE true +NOTICE: executing the command locally: SELECT first.b, first.c, first.d, second.c, second.d FROM (local_shard_execution_replicated.abcd_1500027 first JOIN local_shard_execution_replicated.abcd_1500027 second ON ((first.b OPERATOR(pg_catalog.=) second.b))) WHERE true +NOTICE: executing the command locally: SELECT first.b, first.c, first.d, second.c, second.d FROM (local_shard_execution_replicated.abcd_1500028 first JOIN local_shard_execution_replicated.abcd_1500028 second ON ((first.b OPERATOR(pg_catalog.=) second.b))) WHERE true + b | c | d | c | d +--------------------------------------------------------------------- + 2 | 3 | 4 | 3 | 4 + 3 | 4 | 5 | 4 | 5 + 4 | 5 | 6 | 5 | 6 +(3 rows) + +END; +BEGIN; +SELECT * FROM abcd first join abcd second USING(b) join abcd third on first.b=third.b ORDER BY 1,2,3,4; +NOTICE: executing the command locally: SELECT first.b, first.c, first.d, second.c, second.d, third.b, third.c, third.d FROM ((local_shard_execution_replicated.abcd_1500025 first JOIN local_shard_execution_replicated.abcd_1500025 second ON ((first.b OPERATOR(pg_catalog.=) second.b))) JOIN local_shard_execution_replicated.abcd_1500025 third ON ((first.b OPERATOR(pg_catalog.=) third.b))) WHERE true +NOTICE: executing the command locally: SELECT first.b, first.c, first.d, second.c, second.d, third.b, third.c, third.d FROM ((local_shard_execution_replicated.abcd_1500026 first JOIN local_shard_execution_replicated.abcd_1500026 second ON ((first.b OPERATOR(pg_catalog.=) second.b))) JOIN local_shard_execution_replicated.abcd_1500026 third ON ((first.b OPERATOR(pg_catalog.=) third.b))) WHERE true +NOTICE: executing the command locally: SELECT first.b, first.c, first.d, second.c, second.d, third.b, third.c, third.d FROM ((local_shard_execution_replicated.abcd_1500027 first JOIN local_shard_execution_replicated.abcd_1500027 second ON ((first.b OPERATOR(pg_catalog.=) second.b))) JOIN local_shard_execution_replicated.abcd_1500027 third ON ((first.b OPERATOR(pg_catalog.=) third.b))) WHERE true +NOTICE: executing the command locally: SELECT first.b, first.c, first.d, second.c, second.d, third.b, third.c, third.d FROM ((local_shard_execution_replicated.abcd_1500028 first JOIN local_shard_execution_replicated.abcd_1500028 second ON ((first.b OPERATOR(pg_catalog.=) second.b))) JOIN local_shard_execution_replicated.abcd_1500028 third ON ((first.b OPERATOR(pg_catalog.=) third.b))) WHERE true + b | c | d | c | d | b | c | d +--------------------------------------------------------------------- + 2 | 3 | 4 | 3 | 4 | 2 | 3 | 4 + 3 | 4 | 5 | 4 | 5 | 3 | 4 | 5 + 4 | 5 | 6 | 5 | 6 | 4 | 5 | 6 +(3 rows) + +END; +-- copy always happens via distributed execution irrespective of the +-- shards that are accessed +COPY reference_table FROM STDIN; +COPY distributed_table FROM STDIN WITH CSV; +COPY second_distributed_table FROM STDIN WITH CSV; +-- the behaviour in transaction blocks is the following: + -- (a) Unless the first query is a local query, always use distributed execution. + -- (b) If the executor has used local execution, it has to use local execution + -- for the remaining of the transaction block. If that's not possible, the + -- executor has to error out +-- rollback should be able to rollback local execution +BEGIN; + INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '29' RETURNING *; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1, '11'::text, 21) ON CONFLICT(key) DO UPDATE SET value = '29'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age + key | value | age +--------------------------------------------------------------------- + 1 | 29 | 20 +(1 row) + + SELECT * FROM distributed_table WHERE key = 1 ORDER BY 1,2,3; +NOTICE: executing the command locally: SELECT key, value, age FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) ORDER BY key, value, age + key | value | age +--------------------------------------------------------------------- + 1 | 29 | 20 +(1 row) + +ROLLBACK; +-- make sure that the value is rollbacked +SELECT * FROM distributed_table WHERE key = 1 ORDER BY 1,2,3; +NOTICE: executing the command locally: SELECT key, value, age FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) ORDER BY key, value, age + key | value | age +--------------------------------------------------------------------- + 1 | 22 | 20 +(1 row) + +-- rollback should be able to rollback both the local and distributed executions +BEGIN; + INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '29' RETURNING *; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1, '11'::text, 21) ON CONFLICT(key) DO UPDATE SET value = '29'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age + key | value | age +--------------------------------------------------------------------- + 1 | 29 | 20 +(1 row) + + DELETE FROM distributed_table; +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500004 distributed_table + SELECT count(*) FROM second_distributed_table; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.second_distributed_table_1500005 second_distributed_table WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.second_distributed_table_1500006 second_distributed_table WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.second_distributed_table_1500007 second_distributed_table WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.second_distributed_table_1500008 second_distributed_table WHERE true + count +--------------------------------------------------------------------- + 2 +(1 row) + +ROLLBACK; +-- make sure that everything is rollbacked +SELECT * FROM distributed_table WHERE key = 1 ORDER BY 1,2,3; +NOTICE: executing the command locally: SELECT key, value, age FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) ORDER BY key, value, age + key | value | age +--------------------------------------------------------------------- + 1 | 22 | 20 +(1 row) + +SELECT count(*) FROM second_distributed_table; + count +--------------------------------------------------------------------- + 2 +(1 row) + +SELECT * FROM second_distributed_table; + key | value +--------------------------------------------------------------------- + 1 | 1 + 6 | '6' +(2 rows) + +-- very simple examples, an SELECTs should see the modifications +-- that has done before +BEGIN; + -- INSERT is executed locally + INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '23' RETURNING *; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1, '11'::text, 21) ON CONFLICT(key) DO UPDATE SET value = '23'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age + key | value | age +--------------------------------------------------------------------- + 1 | 23 | 20 +(1 row) + + -- since the INSERT is executed locally, the SELECT should also be + -- executed locally and see the changes + SELECT * FROM distributed_table WHERE key = 1 ORDER BY 1,2,3; +NOTICE: executing the command locally: SELECT key, value, age FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) ORDER BY key, value, age + key | value | age +--------------------------------------------------------------------- + 1 | 23 | 20 +(1 row) + + -- multi-shard SELECTs are now forced to use local execution on + -- the shards that reside on this node + SELECT * FROM distributed_table WHERE value = '23' ORDER BY 1,2,3; +NOTICE: executing the command locally: SELECT key, value, age FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (value OPERATOR(pg_catalog.=) '23'::text) +NOTICE: executing the command locally: SELECT key, value, age FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table WHERE (value OPERATOR(pg_catalog.=) '23'::text) +NOTICE: executing the command locally: SELECT key, value, age FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE (value OPERATOR(pg_catalog.=) '23'::text) +NOTICE: executing the command locally: SELECT key, value, age FROM local_shard_execution_replicated.distributed_table_1500004 distributed_table WHERE (value OPERATOR(pg_catalog.=) '23'::text) + key | value | age +--------------------------------------------------------------------- + 1 | 23 | 20 +(1 row) + + -- similarly, multi-shard modifications should use local exection + -- on the shards that reside on this node + DELETE FROM distributed_table WHERE value = '23'; +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (value OPERATOR(pg_catalog.=) '23'::text) +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table WHERE (value OPERATOR(pg_catalog.=) '23'::text) +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE (value OPERATOR(pg_catalog.=) '23'::text) +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500004 distributed_table WHERE (value OPERATOR(pg_catalog.=) '23'::text) + -- make sure that the value is deleted + SELECT * FROM distributed_table WHERE value = '23' ORDER BY 1,2,3; +NOTICE: executing the command locally: SELECT key, value, age FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (value OPERATOR(pg_catalog.=) '23'::text) +NOTICE: executing the command locally: SELECT key, value, age FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table WHERE (value OPERATOR(pg_catalog.=) '23'::text) +NOTICE: executing the command locally: SELECT key, value, age FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE (value OPERATOR(pg_catalog.=) '23'::text) +NOTICE: executing the command locally: SELECT key, value, age FROM local_shard_execution_replicated.distributed_table_1500004 distributed_table WHERE (value OPERATOR(pg_catalog.=) '23'::text) + key | value | age +--------------------------------------------------------------------- +(0 rows) + +COMMIT; +-- make sure that we've committed everything +SELECT * FROM distributed_table WHERE key = 1 ORDER BY 1,2,3; +NOTICE: executing the command locally: SELECT key, value, age FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) ORDER BY key, value, age + key | value | age +--------------------------------------------------------------------- +(0 rows) + +-- if we start with a distributed execution, we should keep +-- using that and never switch back to local execution +BEGIN; + DELETE FROM distributed_table WHERE value = '11'; +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (value OPERATOR(pg_catalog.=) '11'::text) +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table WHERE (value OPERATOR(pg_catalog.=) '11'::text) +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE (value OPERATOR(pg_catalog.=) '11'::text) +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500004 distributed_table WHERE (value OPERATOR(pg_catalog.=) '11'::text) + -- although this command could have been executed + -- locally, it is not going to be executed locally + SELECT * FROM distributed_table WHERE key = 1 ORDER BY 1,2,3; +NOTICE: executing the command locally: SELECT key, value, age FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) ORDER BY key, value, age + key | value | age +--------------------------------------------------------------------- +(0 rows) + + -- but we can still execute parallel queries, even if + -- they are utility commands + TRUNCATE distributed_table CASCADE; +NOTICE: executing the command locally: TRUNCATE TABLE local_shard_execution_replicated.distributed_table_xxxxx CASCADE +NOTICE: executing the command locally: TRUNCATE TABLE local_shard_execution_replicated.distributed_table_xxxxx CASCADE +NOTICE: executing the command locally: TRUNCATE TABLE local_shard_execution_replicated.distributed_table_xxxxx CASCADE +NOTICE: executing the command locally: TRUNCATE TABLE local_shard_execution_replicated.distributed_table_xxxxx CASCADE + -- TRUNCATE didn't cascade into second_distributed_table + SELECT count(*) FROM second_distributed_table; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.second_distributed_table_1500005 second_distributed_table WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.second_distributed_table_1500006 second_distributed_table WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.second_distributed_table_1500007 second_distributed_table WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.second_distributed_table_1500008 second_distributed_table WHERE true + count +--------------------------------------------------------------------- + 2 +(1 row) + +ROLLBACK; +-- load some data +INSERT INTO reference_table SELECT i FROM generate_series(500, 600) i; +NOTICE: executing the copy locally for shard xxxxx +-- show that complex tx blocks work fine +BEGIN; + INSERT INTO reference_table VALUES (701); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.reference_table_1500000 (key) VALUES (701) + INSERT INTO distributed_table VALUES (701, '701', 701); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 (key, value, age) VALUES (701, '701'::text, 701) + INSERT INTO second_distributed_table VALUES (701, '701'); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.second_distributed_table_1500005 (key, value) VALUES (701, '701'::text) + DELETE FROM reference_table WHERE key = 701; +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.reference_table_1500000 reference_table WHERE (key OPERATOR(pg_catalog.=) 701) + SELECT count(*) FROM distributed_table WHERE key = 701; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 701) + count +--------------------------------------------------------------------- + 1 +(1 row) + + SELECT count(*) FROM second_distributed_table WHERE key = 701; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.second_distributed_table_1500005 second_distributed_table WHERE (key OPERATOR(pg_catalog.=) 701) + count +--------------------------------------------------------------------- + 1 +(1 row) + + -- multi-shard commands should also see the changes + SELECT count(*) FROM distributed_table WHERE key > 700; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.>) 700) +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table WHERE (key OPERATOR(pg_catalog.>) 700) +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE (key OPERATOR(pg_catalog.>) 700) +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500004 distributed_table WHERE (key OPERATOR(pg_catalog.>) 700) + count +--------------------------------------------------------------------- + 1 +(1 row) + + -- we can still do multi-shard commands + DELETE FROM distributed_table; +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500004 distributed_table +ROLLBACK; +-- multiple queries hitting different shards can be executed locally +BEGIN; + SELECT count(*) FROM distributed_table WHERE key = 1; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) + count +--------------------------------------------------------------------- + 0 +(1 row) + + SELECT count(*) FROM distributed_table WHERE key = 6; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE (key OPERATOR(pg_catalog.=) 6) + count +--------------------------------------------------------------------- + 1 +(1 row) + + SELECT count(*) FROM distributed_table WHERE key = 500; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE (key OPERATOR(pg_catalog.=) 500) + count +--------------------------------------------------------------------- + 0 +(1 row) + +ROLLBACK; +-- a local query followed by TRUNCATE command can be executed locally +BEGIN; + SELECT count(*) FROM distributed_table WHERE key = 1; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) + count +--------------------------------------------------------------------- + 0 +(1 row) + + TRUNCATE distributed_table CASCADE; +NOTICE: executing the command locally: TRUNCATE TABLE local_shard_execution_replicated.distributed_table_xxxxx CASCADE +NOTICE: executing the command locally: TRUNCATE TABLE local_shard_execution_replicated.distributed_table_xxxxx CASCADE +NOTICE: executing the command locally: TRUNCATE TABLE local_shard_execution_replicated.distributed_table_xxxxx CASCADE +NOTICE: executing the command locally: TRUNCATE TABLE local_shard_execution_replicated.distributed_table_xxxxx CASCADE +ROLLBACK; +-- a local query is followed by an INSERT..SELECT via the coordinator +BEGIN; + SELECT count(*) FROM distributed_table WHERE key = 1; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) + count +--------------------------------------------------------------------- + 0 +(1 row) + + INSERT INTO distributed_table (key) SELECT i FROM generate_series(1,1) i; +NOTICE: executing the copy locally for shard xxxxx +ROLLBACK; +BEGIN; +SET citus.enable_repartition_joins TO ON; +SELECT count(*) FROM distributed_table; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500004 distributed_table WHERE true + count +--------------------------------------------------------------------- + 2 +(1 row) + +SELECT count(*) FROM distributed_table d1 join distributed_table d2 using(age); +ERROR: cannot execute command because a local execution has accessed a placement in the transaction +DETAIL: Some parallel commands cannot be executed if a previous command has already been executed locally +HINT: Try re-running the transaction with "SET LOCAL citus.enable_local_execution TO OFF;" +ROLLBACK; +-- a local query is followed by an INSERT..SELECT with re-partitioning +BEGIN; + SELECT count(*) FROM distributed_table WHERE key = 6; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE (key OPERATOR(pg_catalog.=) 6) + count +--------------------------------------------------------------------- + 1 +(1 row) + + INSERT INTO reference_table (key) SELECT -key FROM distributed_table; +NOTICE: executing the command locally: SELECT (OPERATOR(pg_catalog.-) key) AS key FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE true +NOTICE: executing the command locally: SELECT (OPERATOR(pg_catalog.-) key) AS key FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table WHERE true +NOTICE: executing the command locally: SELECT (OPERATOR(pg_catalog.-) key) AS key FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE true +NOTICE: executing the command locally: SELECT (OPERATOR(pg_catalog.-) key) AS key FROM local_shard_execution_replicated.distributed_table_1500004 distributed_table WHERE true +NOTICE: executing the copy locally for shard xxxxx + INSERT INTO distributed_table (key) SELECT -key FROM distributed_table; +NOTICE: executing the command locally: SELECT partition_index, 'repartitioned_results_xxxxx_from_1500001_to' || '_' || partition_index::text , rows_written FROM worker_partition_query_result('repartitioned_results_xxxxx_from_1500001_to','SELECT (OPERATOR(pg_catalog.-) key) AS key FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE true',0,'hash','{-2147483648,-1073741824,0,1073741824}'::text[],'{-1073741825,-1,1073741823,2147483647}'::text[],true) WHERE rows_written > 0 +NOTICE: executing the command locally: SELECT partition_index, 'repartitioned_results_xxxxx_from_1500002_to' || '_' || partition_index::text , rows_written FROM worker_partition_query_result('repartitioned_results_xxxxx_from_1500002_to','SELECT (OPERATOR(pg_catalog.-) key) AS key FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table WHERE true',0,'hash','{-2147483648,-1073741824,0,1073741824}'::text[],'{-1073741825,-1,1073741823,2147483647}'::text[],true) WHERE rows_written > 0 +NOTICE: executing the command locally: SELECT partition_index, 'repartitioned_results_xxxxx_from_1500003_to' || '_' || partition_index::text , rows_written FROM worker_partition_query_result('repartitioned_results_xxxxx_from_1500003_to','SELECT (OPERATOR(pg_catalog.-) key) AS key FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE true',0,'hash','{-2147483648,-1073741824,0,1073741824}'::text[],'{-1073741825,-1,1073741823,2147483647}'::text[],true) WHERE rows_written > 0 +NOTICE: executing the command locally: SELECT partition_index, 'repartitioned_results_xxxxx_from_1500004_to' || '_' || partition_index::text , rows_written FROM worker_partition_query_result('repartitioned_results_xxxxx_from_1500004_to','SELECT (OPERATOR(pg_catalog.-) key) AS key FROM local_shard_execution_replicated.distributed_table_1500004 distributed_table WHERE true',0,'hash','{-2147483648,-1073741824,0,1073741824}'::text[],'{-1073741825,-1,1073741823,2147483647}'::text[],true) WHERE rows_written > 0 +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key) SELECT key FROM read_intermediate_results('{repartitioned_results_xxxxx_from_1500003_to_0}'::text[], 'binary'::citus_copy_format) intermediate_result(key integer) +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500004 AS citus_table_alias (key) SELECT key FROM read_intermediate_results('{repartitioned_results_xxxxx_from_1500004_to_3}'::text[], 'binary'::citus_copy_format) intermediate_result(key integer) + SELECT count(*) FROM distributed_table WHERE key = -6; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) '-6'::integer) + count +--------------------------------------------------------------------- + 1 +(1 row) + +ROLLBACK; +INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '29' RETURNING *; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1, '11'::text, 21) ON CONFLICT(key) DO UPDATE SET value = '29'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age + key | value | age +--------------------------------------------------------------------- + 1 | 11 | 21 +(1 row) + +BEGIN; + DELETE FROM distributed_table WHERE key = 1; +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) + EXPLAIN ANALYZE DELETE FROM distributed_table WHERE key = 1; +ERROR: cannot execute command because a local execution has accessed a placement in the transaction +DETAIL: Some parallel commands cannot be executed if a previous command has already been executed locally +HINT: Try re-running the transaction with "SET LOCAL citus.enable_local_execution TO OFF;" +ROLLBACK; +BEGIN; + INSERT INTO distributed_table VALUES (11, '111',29) ON CONFLICT(key) DO UPDATE SET value = '29' RETURNING *; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500004 AS citus_table_alias (key, value, age) VALUES (11, '111'::text, 29) ON CONFLICT(key) DO UPDATE SET value = '29'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age + key | value | age +--------------------------------------------------------------------- + 11 | 29 | 121 +(1 row) + + -- this is already disallowed on the nodes, adding it in case we + -- support DDLs from the worker nodes in the future + ALTER TABLE distributed_table ADD COLUMN x INT; +ERROR: operation is not allowed on this node +HINT: Connect to the coordinator and run it again. +ROLLBACK; +BEGIN; + INSERT INTO distributed_table VALUES (11, '111',29) ON CONFLICT(key) DO UPDATE SET value = '29' RETURNING *; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500004 AS citus_table_alias (key, value, age) VALUES (11, '111'::text, 29) ON CONFLICT(key) DO UPDATE SET value = '29'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age + key | value | age +--------------------------------------------------------------------- + 11 | 29 | 121 +(1 row) + + -- this is already disallowed because VACUUM cannot be executed in tx block + -- adding in case this is supported some day + VACUUM second_distributed_table; +ERROR: VACUUM cannot run inside a transaction block +ROLLBACK; +-- make sure that functions can use local execution +CREATE OR REPLACE PROCEDURE only_local_execution() AS $$ + DECLARE cnt INT; + BEGIN + INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '29'; + SELECT count(*) INTO cnt FROM distributed_table WHERE key = 1; + DELETE FROM distributed_table WHERE key = 1; + END; +$$ LANGUAGE plpgsql; +CALL only_local_execution(); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1, '11'::text, 21) ON CONFLICT(key) DO UPDATE SET value = '29'::text +CONTEXT: SQL statement "INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '29'" +PL/pgSQL function only_local_execution() line XX at SQL statement +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) +CONTEXT: SQL statement "SELECT count(*) FROM distributed_table WHERE key = 1" +PL/pgSQL function only_local_execution() line XX at SQL statement +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) +CONTEXT: SQL statement "DELETE FROM distributed_table WHERE key = 1" +PL/pgSQL function only_local_execution() line XX at SQL statement +-- insert a row that we need in the next tests +INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '29'; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1, '11'::text, 21) ON CONFLICT(key) DO UPDATE SET value = '29'::text +-- make sure that functions can use local execution +CREATE OR REPLACE PROCEDURE only_local_execution_with_function_evaluation() AS $$ + DECLARE nodeId INT; + BEGIN + -- fast path router + SELECT get_local_node_id_volatile() INTO nodeId FROM distributed_table WHERE key = 1; + IF nodeId <= 0 THEN + RAISE NOTICE 'unexpected node id'; + END IF; + + -- regular router + SELECT get_local_node_id_volatile() INTO nodeId FROM distributed_table d1 JOIN distributed_table d2 USING (key) WHERE d1.key = 1; + IF nodeId <= 0 THEN + RAISE NOTICE 'unexpected node id'; + END IF; + END; +$$ LANGUAGE plpgsql; +CALL only_local_execution_with_function_evaluation(); +NOTICE: executing the command locally: SELECT local_shard_execution_replicated.get_local_node_id_volatile() AS get_local_node_id_volatile FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) +CONTEXT: SQL statement "SELECT get_local_node_id_volatile() FROM distributed_table WHERE key = 1" +PL/pgSQL function only_local_execution_with_function_evaluation() line XX at SQL statement +NOTICE: executing the command locally: SELECT local_shard_execution_replicated.get_local_node_id_volatile() AS get_local_node_id_volatile FROM (local_shard_execution_replicated.distributed_table_1500001 d1(key, value, age) JOIN local_shard_execution_replicated.distributed_table_1500001 d2(key, value, age) USING (key)) WHERE (d1.key OPERATOR(pg_catalog.=) 1) +CONTEXT: SQL statement "SELECT get_local_node_id_volatile() FROM distributed_table d1 JOIN distributed_table d2 USING (key) WHERE d1.key = 1" +PL/pgSQL function only_local_execution_with_function_evaluation() line XX at SQL statement +CREATE OR REPLACE PROCEDURE only_local_execution_with_params(int) AS $$ + DECLARE cnt INT; + BEGIN + INSERT INTO distributed_table VALUES ($1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '29'; + SELECT count(*) INTO cnt FROM distributed_table WHERE key = $1; + DELETE FROM distributed_table WHERE key = $1; + END; +$$ LANGUAGE plpgsql; +CALL only_local_execution_with_params(1); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1, '11'::text, '21'::bigint) ON CONFLICT(key) DO UPDATE SET value = '29'::text +CONTEXT: SQL statement "INSERT INTO distributed_table VALUES ($1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '29'" +PL/pgSQL function only_local_execution_with_params(integer) line XX at SQL statement +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) +CONTEXT: SQL statement "SELECT count(*) FROM distributed_table WHERE key = $1" +PL/pgSQL function only_local_execution_with_params(integer) line XX at SQL statement +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) +CONTEXT: SQL statement "DELETE FROM distributed_table WHERE key = $1" +PL/pgSQL function only_local_execution_with_params(integer) line XX at SQL statement +CREATE OR REPLACE PROCEDURE only_local_execution_with_function_evaluation_param(int) AS $$ + DECLARE nodeId INT; + BEGIN + -- fast path router + SELECT get_local_node_id_volatile() INTO nodeId FROM distributed_table WHERE key = $1; + IF nodeId <= 0 THEN + RAISE NOTICE 'unexpected node id'; + END IF; + + -- regular router + SELECT get_local_node_id_volatile() INTO nodeId FROM distributed_table d1 JOIN distributed_table d2 USING (key) WHERE d1.key = $1; + IF nodeId <= 0 THEN + RAISE NOTICE 'unexpected node id'; + END IF; + END; +$$ LANGUAGE plpgsql; +CALL only_local_execution_with_function_evaluation_param(1); +NOTICE: executing the command locally: SELECT local_shard_execution_replicated.get_local_node_id_volatile() AS get_local_node_id_volatile FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) +CONTEXT: SQL statement "SELECT get_local_node_id_volatile() FROM distributed_table WHERE key = $1" +PL/pgSQL function only_local_execution_with_function_evaluation_param(integer) line XX at SQL statement +NOTICE: executing the command locally: SELECT local_shard_execution_replicated.get_local_node_id_volatile() AS get_local_node_id_volatile FROM (local_shard_execution_replicated.distributed_table_1500001 d1(key, value, age) JOIN local_shard_execution_replicated.distributed_table_1500001 d2(key, value, age) USING (key)) WHERE (d1.key OPERATOR(pg_catalog.=) $1) +CONTEXT: SQL statement "SELECT get_local_node_id_volatile() FROM distributed_table d1 JOIN distributed_table d2 USING (key) WHERE d1.key = $1" +PL/pgSQL function only_local_execution_with_function_evaluation_param(integer) line XX at SQL statement +CREATE OR REPLACE PROCEDURE local_execution_followed_by_dist() AS $$ + DECLARE cnt INT; + BEGIN + INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '29'; + SELECT count(*) INTO cnt FROM distributed_table WHERE key = 1; + DELETE FROM distributed_table; + SELECT count(*) INTO cnt FROM distributed_table; + END; +$$ LANGUAGE plpgsql; +CALL local_execution_followed_by_dist(); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1, '11'::text, 21) ON CONFLICT(key) DO UPDATE SET value = '29'::text +CONTEXT: SQL statement "INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '29'" +PL/pgSQL function local_execution_followed_by_dist() line XX at SQL statement +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) +CONTEXT: SQL statement "SELECT count(*) FROM distributed_table WHERE key = 1" +PL/pgSQL function local_execution_followed_by_dist() line XX at SQL statement +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table +CONTEXT: SQL statement "DELETE FROM distributed_table" +PL/pgSQL function local_execution_followed_by_dist() line XX at SQL statement +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table +CONTEXT: SQL statement "DELETE FROM distributed_table" +PL/pgSQL function local_execution_followed_by_dist() line XX at SQL statement +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table +CONTEXT: SQL statement "DELETE FROM distributed_table" +PL/pgSQL function local_execution_followed_by_dist() line XX at SQL statement +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500004 distributed_table +CONTEXT: SQL statement "DELETE FROM distributed_table" +PL/pgSQL function local_execution_followed_by_dist() line XX at SQL statement +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE true +CONTEXT: SQL statement "SELECT count(*) FROM distributed_table" +PL/pgSQL function local_execution_followed_by_dist() line XX at SQL statement +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table WHERE true +CONTEXT: SQL statement "SELECT count(*) FROM distributed_table" +PL/pgSQL function local_execution_followed_by_dist() line XX at SQL statement +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE true +CONTEXT: SQL statement "SELECT count(*) FROM distributed_table" +PL/pgSQL function local_execution_followed_by_dist() line XX at SQL statement +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500004 distributed_table WHERE true +CONTEXT: SQL statement "SELECT count(*) FROM distributed_table" +PL/pgSQL function local_execution_followed_by_dist() line XX at SQL statement +-- test CTEs, including modifying CTEs +WITH local_insert AS (INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '29' RETURNING *), +distributed_local_mixed AS (SELECT * FROM reference_table WHERE key IN (SELECT key FROM local_insert)) +SELECT * FROM local_insert, distributed_local_mixed; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1, '11'::text, 21) ON CONFLICT(key) DO UPDATE SET value = '29'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age +NOTICE: executing the command locally: SELECT key FROM local_shard_execution_replicated.reference_table_1500000 reference_table WHERE (key OPERATOR(pg_catalog.=) ANY (SELECT local_insert.key FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.age FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, age bigint)) local_insert)) +NOTICE: executing the command locally: SELECT local_insert.key, local_insert.value, local_insert.age, distributed_local_mixed.key FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.age FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, age bigint)) local_insert, (SELECT intermediate_result.key FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_local_mixed + key | value | age | key +--------------------------------------------------------------------- + 1 | 11 | 21 | 1 +(1 row) + +-- since we start with parallel execution, we do not switch back to local execution in the +-- latter CTEs +WITH distributed_local_mixed AS (SELECT * FROM distributed_table), +local_insert AS (INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '29' RETURNING *) +SELECT * FROM local_insert, distributed_local_mixed ORDER BY 1,2,3,4,5; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1, '11'::text, 21) ON CONFLICT(key) DO UPDATE SET value = '29'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age +NOTICE: executing the command locally: SELECT worker_column_1 AS key, worker_column_2 AS value, worker_column_3 AS age, worker_column_4 AS key, worker_column_5 AS value, worker_column_6 AS age FROM (SELECT local_insert.key AS worker_column_1, local_insert.value AS worker_column_2, local_insert.age AS worker_column_3, distributed_local_mixed.key AS worker_column_4, distributed_local_mixed.value AS worker_column_5, distributed_local_mixed.age AS worker_column_6 FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.age FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, age bigint)) local_insert, (SELECT distributed_table.key, distributed_table.value, distributed_table.age FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table) distributed_local_mixed) worker_subquery +NOTICE: executing the command locally: SELECT worker_column_1 AS key, worker_column_2 AS value, worker_column_3 AS age, worker_column_4 AS key, worker_column_5 AS value, worker_column_6 AS age FROM (SELECT local_insert.key AS worker_column_1, local_insert.value AS worker_column_2, local_insert.age AS worker_column_3, distributed_local_mixed.key AS worker_column_4, distributed_local_mixed.value AS worker_column_5, distributed_local_mixed.age AS worker_column_6 FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.age FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, age bigint)) local_insert, (SELECT distributed_table.key, distributed_table.value, distributed_table.age FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table) distributed_local_mixed) worker_subquery +NOTICE: executing the command locally: SELECT worker_column_1 AS key, worker_column_2 AS value, worker_column_3 AS age, worker_column_4 AS key, worker_column_5 AS value, worker_column_6 AS age FROM (SELECT local_insert.key AS worker_column_1, local_insert.value AS worker_column_2, local_insert.age AS worker_column_3, distributed_local_mixed.key AS worker_column_4, distributed_local_mixed.value AS worker_column_5, distributed_local_mixed.age AS worker_column_6 FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.age FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, age bigint)) local_insert, (SELECT distributed_table.key, distributed_table.value, distributed_table.age FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table) distributed_local_mixed) worker_subquery +NOTICE: executing the command locally: SELECT worker_column_1 AS key, worker_column_2 AS value, worker_column_3 AS age, worker_column_4 AS key, worker_column_5 AS value, worker_column_6 AS age FROM (SELECT local_insert.key AS worker_column_1, local_insert.value AS worker_column_2, local_insert.age AS worker_column_3, distributed_local_mixed.key AS worker_column_4, distributed_local_mixed.value AS worker_column_5, distributed_local_mixed.age AS worker_column_6 FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.age FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, age bigint)) local_insert, (SELECT distributed_table.key, distributed_table.value, distributed_table.age FROM local_shard_execution_replicated.distributed_table_1500004 distributed_table) distributed_local_mixed) worker_subquery + key | value | age | key | value | age +--------------------------------------------------------------------- + 1 | 29 | 21 | 1 | 11 | 21 +(1 row) + +-- router CTE pushdown +WITH all_data AS (SELECT * FROM distributed_table WHERE key = 1) +SELECT + count(*) +FROM + distributed_table, all_data +WHERE + distributed_table.key = all_data.key AND distributed_table.key = 1; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table, (SELECT distributed_table_1.key, distributed_table_1.value, distributed_table_1.age FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table_1 WHERE (distributed_table_1.key OPERATOR(pg_catalog.=) 1)) all_data WHERE ((distributed_table.key OPERATOR(pg_catalog.=) all_data.key) AND (distributed_table.key OPERATOR(pg_catalog.=) 1)) + count +--------------------------------------------------------------------- + 1 +(1 row) + +INSERT INTO reference_table VALUES (2); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.reference_table_1500000 (key) VALUES (2) +INSERT INTO distributed_table VALUES (2, '29', 29); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500004 (key, value, age) VALUES (2, '29'::text, 29) +INSERT INTO second_distributed_table VALUES (2, '29'); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.second_distributed_table_1500008 (key, value) VALUES (2, '29'::text) +-- single shard that is not a local query followed by a local query +WITH all_data AS (SELECT * FROM second_distributed_table WHERE key = 2) +SELECT + distributed_table.key +FROM + distributed_table, all_data +WHERE + distributed_table.value = all_data.value AND distributed_table.key = 1 +ORDER BY + 1 DESC; +NOTICE: executing the command locally: SELECT distributed_table.key FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table, (SELECT second_distributed_table.key, second_distributed_table.value FROM local_shard_execution_replicated.second_distributed_table_1500008 second_distributed_table WHERE (second_distributed_table.key OPERATOR(pg_catalog.=) 2)) all_data WHERE ((distributed_table.value OPERATOR(pg_catalog.=) all_data.value) AND (distributed_table.key OPERATOR(pg_catalog.=) 1)) ORDER BY distributed_table.key DESC + key +--------------------------------------------------------------------- + 1 +(1 row) + +-- multi-shard CTE is followed by a query which could be executed locally, but +-- since the query started with a parallel query, it doesn't use local execution +-- note that if we allow Postgres to inline the CTE (e.g., not have the EXISTS +-- subquery), then it'd pushdown the filters and the query becomes single-shard, +-- locally executable query +WITH all_data AS (SELECT * FROM distributed_table) +SELECT + count(*) +FROM + distributed_table, all_data +WHERE + distributed_table.key = all_data.key AND distributed_table.key = 1 + AND EXISTS (SELECT * FROM all_data); +NOTICE: executing the command locally: SELECT key, value, age FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE true +NOTICE: executing the command locally: SELECT key, value, age FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table WHERE true +NOTICE: executing the command locally: SELECT key, value, age FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE true +NOTICE: executing the command locally: SELECT key, value, age FROM local_shard_execution_replicated.distributed_table_1500004 distributed_table WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table, (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.age FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, age bigint)) all_data WHERE ((distributed_table.key OPERATOR(pg_catalog.=) all_data.key) AND (distributed_table.key OPERATOR(pg_catalog.=) 1) AND (EXISTS (SELECT all_data_1.key, all_data_1.value, all_data_1.age FROM (SELECT intermediate_result.key, intermediate_result.value, intermediate_result.age FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text, age bigint)) all_data_1))) + count +--------------------------------------------------------------------- + 1 +(1 row) + +-- in pg12, the following CTE can be inlined, still the query becomes +-- a subquery that needs to be recursively planned and a parallel +-- query, so do not use local execution +WITH all_data AS (SELECT age FROM distributed_table) +SELECT + count(*) +FROM + distributed_table, all_data +WHERE + distributed_table.key = all_data.age AND distributed_table.key = 1; +NOTICE: executing the command locally: SELECT age FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE true +NOTICE: executing the command locally: SELECT age FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table WHERE true +NOTICE: executing the command locally: SELECT age FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE true +NOTICE: executing the command locally: SELECT age FROM local_shard_execution_replicated.distributed_table_1500004 distributed_table WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table, (SELECT intermediate_result.age FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(age bigint)) all_data WHERE ((distributed_table.key OPERATOR(pg_catalog.=) all_data.age) AND (distributed_table.key OPERATOR(pg_catalog.=) 1)) + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- get ready for the next commands +TRUNCATE reference_table, distributed_table, second_distributed_table; +NOTICE: executing the command locally: TRUNCATE TABLE local_shard_execution_replicated.reference_table_xxxxx CASCADE +NOTICE: executing the command locally: TRUNCATE TABLE local_shard_execution_replicated.distributed_table_xxxxx CASCADE +NOTICE: executing the command locally: TRUNCATE TABLE local_shard_execution_replicated.distributed_table_xxxxx CASCADE +NOTICE: executing the command locally: TRUNCATE TABLE local_shard_execution_replicated.distributed_table_xxxxx CASCADE +NOTICE: executing the command locally: TRUNCATE TABLE local_shard_execution_replicated.distributed_table_xxxxx CASCADE +NOTICE: executing the command locally: TRUNCATE TABLE local_shard_execution_replicated.second_distributed_table_xxxxx CASCADE +NOTICE: executing the command locally: TRUNCATE TABLE local_shard_execution_replicated.second_distributed_table_xxxxx CASCADE +NOTICE: executing the command locally: TRUNCATE TABLE local_shard_execution_replicated.second_distributed_table_xxxxx CASCADE +NOTICE: executing the command locally: TRUNCATE TABLE local_shard_execution_replicated.second_distributed_table_xxxxx CASCADE +-- local execution of returning of reference tables +INSERT INTO reference_table VALUES (1),(2),(3),(4),(5),(6) RETURNING *; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.reference_table_1500000 AS citus_table_alias (key) VALUES (1), (2), (3), (4), (5), (6) RETURNING citus_table_alias.key + key +--------------------------------------------------------------------- + 1 + 2 + 3 + 4 + 5 + 6 +(6 rows) + +-- local execution of multi-row INSERTs +INSERT INTO distributed_table VALUES (1, '11',21), (5,'55',22) ON CONFLICT(key) DO UPDATE SET value = (EXCLUDED.value::int + 1)::text RETURNING *; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1,'11'::text,'21'::bigint), (5,'55'::text,'22'::bigint) ON CONFLICT(key) DO UPDATE SET value = (((excluded.value)::integer OPERATOR(pg_catalog.+) 1))::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age + key | value | age +--------------------------------------------------------------------- + 1 | 11 | 21 + 5 | 55 | 22 +(2 rows) + +-- distributed execution of multi-rows INSERTs, where executor +-- is smart enough to execute local tasks via local execution +INSERT INTO distributed_table VALUES (1, '11',21), (2,'22',22), (3,'33',33), (4,'44',44),(5,'55',55) ON CONFLICT(key) DO UPDATE SET value = (EXCLUDED.value::int + 1)::text RETURNING *; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1,'11'::text,'21'::bigint), (5,'55'::text,'55'::bigint) ON CONFLICT(key) DO UPDATE SET value = (((excluded.value)::integer OPERATOR(pg_catalog.+) 1))::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500002 AS citus_table_alias (key, value, age) VALUES (3,'33'::text,'33'::bigint), (4,'44'::text,'44'::bigint) ON CONFLICT(key) DO UPDATE SET value = (((excluded.value)::integer OPERATOR(pg_catalog.+) 1))::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500004 AS citus_table_alias (key, value, age) VALUES (2,'22'::text,'22'::bigint) ON CONFLICT(key) DO UPDATE SET value = (((excluded.value)::integer OPERATOR(pg_catalog.+) 1))::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age + key | value | age +--------------------------------------------------------------------- + 1 | 12 | 21 + 2 | 22 | 22 + 3 | 33 | 33 + 4 | 44 | 44 + 5 | 56 | 22 +(5 rows) + +PREPARE local_prepare_no_param AS SELECT count(*) FROM distributed_table WHERE key = 1; +PREPARE local_prepare_no_param_subquery AS +SELECT DISTINCT trim(value) FROM ( + SELECT value FROM distributed_table + WHERE + key IN (1, 6, 500, 701) + AND (select 2) > random() + order by 1 + limit 2 + ) t; +PREPARE local_prepare_param (int) AS SELECT count(*) FROM distributed_table WHERE key = $1; +PREPARE remote_prepare_param (int) AS SELECT count(*) FROM distributed_table WHERE key != $1; +BEGIN; + -- 8 local execution without params + EXECUTE local_prepare_no_param; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) + count +--------------------------------------------------------------------- + 1 +(1 row) + + EXECUTE local_prepare_no_param; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) + count +--------------------------------------------------------------------- + 1 +(1 row) + + EXECUTE local_prepare_no_param; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) + count +--------------------------------------------------------------------- + 1 +(1 row) + + EXECUTE local_prepare_no_param; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) + count +--------------------------------------------------------------------- + 1 +(1 row) + + EXECUTE local_prepare_no_param; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) + count +--------------------------------------------------------------------- + 1 +(1 row) + + EXECUTE local_prepare_no_param; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) + count +--------------------------------------------------------------------- + 1 +(1 row) + + EXECUTE local_prepare_no_param; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) + count +--------------------------------------------------------------------- + 1 +(1 row) + + EXECUTE local_prepare_no_param; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) + count +--------------------------------------------------------------------- + 1 +(1 row) + + -- 8 local execution without params and some subqueries + EXECUTE local_prepare_no_param_subquery; +NOTICE: executing the command locally: SELECT worker_column_1 AS value FROM (SELECT distributed_table.value AS worker_column_1 FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE ((distributed_table.key OPERATOR(pg_catalog.=) ANY (ARRAY[1, 6, 500, 701])) AND (((SELECT 2))::double precision OPERATOR(pg_catalog.>) random()))) worker_subquery ORDER BY worker_column_1 LIMIT '2'::bigint +NOTICE: executing the command locally: SELECT worker_column_1 AS value FROM (SELECT distributed_table.value AS worker_column_1 FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE ((distributed_table.key OPERATOR(pg_catalog.=) ANY (ARRAY[1, 6, 500, 701])) AND (((SELECT 2))::double precision OPERATOR(pg_catalog.>) random()))) worker_subquery ORDER BY worker_column_1 LIMIT '2'::bigint +NOTICE: executing the command locally: SELECT DISTINCT btrim(value) AS btrim FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value text)) t + btrim +--------------------------------------------------------------------- + 12 +(1 row) + + EXECUTE local_prepare_no_param_subquery; +NOTICE: executing the command locally: SELECT worker_column_1 AS value FROM (SELECT distributed_table.value AS worker_column_1 FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE ((distributed_table.key OPERATOR(pg_catalog.=) ANY (ARRAY[1, 6, 500, 701])) AND (((SELECT 2))::double precision OPERATOR(pg_catalog.>) random()))) worker_subquery ORDER BY worker_column_1 LIMIT '2'::bigint +NOTICE: executing the command locally: SELECT worker_column_1 AS value FROM (SELECT distributed_table.value AS worker_column_1 FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE ((distributed_table.key OPERATOR(pg_catalog.=) ANY (ARRAY[1, 6, 500, 701])) AND (((SELECT 2))::double precision OPERATOR(pg_catalog.>) random()))) worker_subquery ORDER BY worker_column_1 LIMIT '2'::bigint +NOTICE: executing the command locally: SELECT DISTINCT btrim(value) AS btrim FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value text)) t + btrim +--------------------------------------------------------------------- + 12 +(1 row) + + EXECUTE local_prepare_no_param_subquery; +NOTICE: executing the command locally: SELECT worker_column_1 AS value FROM (SELECT distributed_table.value AS worker_column_1 FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE ((distributed_table.key OPERATOR(pg_catalog.=) ANY (ARRAY[1, 6, 500, 701])) AND (((SELECT 2))::double precision OPERATOR(pg_catalog.>) random()))) worker_subquery ORDER BY worker_column_1 LIMIT '2'::bigint +NOTICE: executing the command locally: SELECT worker_column_1 AS value FROM (SELECT distributed_table.value AS worker_column_1 FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE ((distributed_table.key OPERATOR(pg_catalog.=) ANY (ARRAY[1, 6, 500, 701])) AND (((SELECT 2))::double precision OPERATOR(pg_catalog.>) random()))) worker_subquery ORDER BY worker_column_1 LIMIT '2'::bigint +NOTICE: executing the command locally: SELECT DISTINCT btrim(value) AS btrim FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value text)) t + btrim +--------------------------------------------------------------------- + 12 +(1 row) + + EXECUTE local_prepare_no_param_subquery; +NOTICE: executing the command locally: SELECT worker_column_1 AS value FROM (SELECT distributed_table.value AS worker_column_1 FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE ((distributed_table.key OPERATOR(pg_catalog.=) ANY (ARRAY[1, 6, 500, 701])) AND (((SELECT 2))::double precision OPERATOR(pg_catalog.>) random()))) worker_subquery ORDER BY worker_column_1 LIMIT '2'::bigint +NOTICE: executing the command locally: SELECT worker_column_1 AS value FROM (SELECT distributed_table.value AS worker_column_1 FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE ((distributed_table.key OPERATOR(pg_catalog.=) ANY (ARRAY[1, 6, 500, 701])) AND (((SELECT 2))::double precision OPERATOR(pg_catalog.>) random()))) worker_subquery ORDER BY worker_column_1 LIMIT '2'::bigint +NOTICE: executing the command locally: SELECT DISTINCT btrim(value) AS btrim FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value text)) t + btrim +--------------------------------------------------------------------- + 12 +(1 row) + + EXECUTE local_prepare_no_param_subquery; +NOTICE: executing the command locally: SELECT worker_column_1 AS value FROM (SELECT distributed_table.value AS worker_column_1 FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE ((distributed_table.key OPERATOR(pg_catalog.=) ANY (ARRAY[1, 6, 500, 701])) AND (((SELECT 2))::double precision OPERATOR(pg_catalog.>) random()))) worker_subquery ORDER BY worker_column_1 LIMIT '2'::bigint +NOTICE: executing the command locally: SELECT worker_column_1 AS value FROM (SELECT distributed_table.value AS worker_column_1 FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE ((distributed_table.key OPERATOR(pg_catalog.=) ANY (ARRAY[1, 6, 500, 701])) AND (((SELECT 2))::double precision OPERATOR(pg_catalog.>) random()))) worker_subquery ORDER BY worker_column_1 LIMIT '2'::bigint +NOTICE: executing the command locally: SELECT DISTINCT btrim(value) AS btrim FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value text)) t + btrim +--------------------------------------------------------------------- + 12 +(1 row) + + EXECUTE local_prepare_no_param_subquery; +NOTICE: executing the command locally: SELECT worker_column_1 AS value FROM (SELECT distributed_table.value AS worker_column_1 FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE ((distributed_table.key OPERATOR(pg_catalog.=) ANY (ARRAY[1, 6, 500, 701])) AND (((SELECT 2))::double precision OPERATOR(pg_catalog.>) random()))) worker_subquery ORDER BY worker_column_1 LIMIT '2'::bigint +NOTICE: executing the command locally: SELECT worker_column_1 AS value FROM (SELECT distributed_table.value AS worker_column_1 FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE ((distributed_table.key OPERATOR(pg_catalog.=) ANY (ARRAY[1, 6, 500, 701])) AND (((SELECT 2))::double precision OPERATOR(pg_catalog.>) random()))) worker_subquery ORDER BY worker_column_1 LIMIT '2'::bigint +NOTICE: executing the command locally: SELECT DISTINCT btrim(value) AS btrim FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value text)) t + btrim +--------------------------------------------------------------------- + 12 +(1 row) + + EXECUTE local_prepare_no_param_subquery; +NOTICE: executing the command locally: SELECT worker_column_1 AS value FROM (SELECT distributed_table.value AS worker_column_1 FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE ((distributed_table.key OPERATOR(pg_catalog.=) ANY (ARRAY[1, 6, 500, 701])) AND (((SELECT 2))::double precision OPERATOR(pg_catalog.>) random()))) worker_subquery ORDER BY worker_column_1 LIMIT '2'::bigint +NOTICE: executing the command locally: SELECT worker_column_1 AS value FROM (SELECT distributed_table.value AS worker_column_1 FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE ((distributed_table.key OPERATOR(pg_catalog.=) ANY (ARRAY[1, 6, 500, 701])) AND (((SELECT 2))::double precision OPERATOR(pg_catalog.>) random()))) worker_subquery ORDER BY worker_column_1 LIMIT '2'::bigint +NOTICE: executing the command locally: SELECT DISTINCT btrim(value) AS btrim FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value text)) t + btrim +--------------------------------------------------------------------- + 12 +(1 row) + + EXECUTE local_prepare_no_param_subquery; +NOTICE: executing the command locally: SELECT worker_column_1 AS value FROM (SELECT distributed_table.value AS worker_column_1 FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE ((distributed_table.key OPERATOR(pg_catalog.=) ANY (ARRAY[1, 6, 500, 701])) AND (((SELECT 2))::double precision OPERATOR(pg_catalog.>) random()))) worker_subquery ORDER BY worker_column_1 LIMIT '2'::bigint +NOTICE: executing the command locally: SELECT worker_column_1 AS value FROM (SELECT distributed_table.value AS worker_column_1 FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE ((distributed_table.key OPERATOR(pg_catalog.=) ANY (ARRAY[1, 6, 500, 701])) AND (((SELECT 2))::double precision OPERATOR(pg_catalog.>) random()))) worker_subquery ORDER BY worker_column_1 LIMIT '2'::bigint +NOTICE: executing the command locally: SELECT DISTINCT btrim(value) AS btrim FROM (SELECT intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(value text)) t + btrim +--------------------------------------------------------------------- + 12 +(1 row) + + -- 8 local executions with params + EXECUTE local_prepare_param(1); +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) + count +--------------------------------------------------------------------- + 1 +(1 row) + + EXECUTE local_prepare_param(5); +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 5) + count +--------------------------------------------------------------------- + 1 +(1 row) + + EXECUTE local_prepare_param(6); +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE (key OPERATOR(pg_catalog.=) 6) + count +--------------------------------------------------------------------- + 0 +(1 row) + + EXECUTE local_prepare_param(1); +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 1) + count +--------------------------------------------------------------------- + 1 +(1 row) + + EXECUTE local_prepare_param(5); +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.=) 5) + count +--------------------------------------------------------------------- + 1 +(1 row) + + EXECUTE local_prepare_param(6); +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE (key OPERATOR(pg_catalog.=) 6) + count +--------------------------------------------------------------------- + 0 +(1 row) + + EXECUTE local_prepare_param(6); +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE (key OPERATOR(pg_catalog.=) 6) + count +--------------------------------------------------------------------- + 0 +(1 row) + + EXECUTE local_prepare_param(6); +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE (key OPERATOR(pg_catalog.=) 6) + count +--------------------------------------------------------------------- + 0 +(1 row) + + -- followed by a non-local execution + EXECUTE remote_prepare_param(1); +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.<>) 1) +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table WHERE (key OPERATOR(pg_catalog.<>) 1) +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE (key OPERATOR(pg_catalog.<>) 1) +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500004 distributed_table WHERE (key OPERATOR(pg_catalog.<>) 1) + count +--------------------------------------------------------------------- + 4 +(1 row) + +COMMIT; +PREPARE local_insert_prepare_no_param AS INSERT INTO distributed_table VALUES (1+0*random(), '11',21::int) ON CONFLICT(key) DO UPDATE SET value = '29' || '28' RETURNING *, key + 1, value || '30', age * 15; +PREPARE local_insert_prepare_param (int) AS INSERT INTO distributed_table VALUES ($1+0*random(), '11',21::int) ON CONFLICT(key) DO UPDATE SET value = '29' || '28' RETURNING *, key + 1, value || '30', age * 15; +BEGIN; + -- 8 local execution without params + EXECUTE local_insert_prepare_no_param; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1, '11'::text, '21'::bigint) ON CONFLICT(key) DO UPDATE SET value = '2928'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age, (citus_table_alias.key OPERATOR(pg_catalog.+) 1), (citus_table_alias.value OPERATOR(pg_catalog.||) '30'::text), (citus_table_alias.age OPERATOR(pg_catalog.*) 15) + key | value | age | ?column? | ?column? | ?column? +--------------------------------------------------------------------- + 1 | 2928 | 21 | 2 | 292830 | 315 +(1 row) + + EXECUTE local_insert_prepare_no_param; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1, '11'::text, '21'::bigint) ON CONFLICT(key) DO UPDATE SET value = '2928'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age, (citus_table_alias.key OPERATOR(pg_catalog.+) 1), (citus_table_alias.value OPERATOR(pg_catalog.||) '30'::text), (citus_table_alias.age OPERATOR(pg_catalog.*) 15) + key | value | age | ?column? | ?column? | ?column? +--------------------------------------------------------------------- + 1 | 2928 | 21 | 2 | 292830 | 315 +(1 row) + + EXECUTE local_insert_prepare_no_param; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1, '11'::text, '21'::bigint) ON CONFLICT(key) DO UPDATE SET value = '2928'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age, (citus_table_alias.key OPERATOR(pg_catalog.+) 1), (citus_table_alias.value OPERATOR(pg_catalog.||) '30'::text), (citus_table_alias.age OPERATOR(pg_catalog.*) 15) + key | value | age | ?column? | ?column? | ?column? +--------------------------------------------------------------------- + 1 | 2928 | 21 | 2 | 292830 | 315 +(1 row) + + EXECUTE local_insert_prepare_no_param; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1, '11'::text, '21'::bigint) ON CONFLICT(key) DO UPDATE SET value = '2928'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age, (citus_table_alias.key OPERATOR(pg_catalog.+) 1), (citus_table_alias.value OPERATOR(pg_catalog.||) '30'::text), (citus_table_alias.age OPERATOR(pg_catalog.*) 15) + key | value | age | ?column? | ?column? | ?column? +--------------------------------------------------------------------- + 1 | 2928 | 21 | 2 | 292830 | 315 +(1 row) + + EXECUTE local_insert_prepare_no_param; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1, '11'::text, '21'::bigint) ON CONFLICT(key) DO UPDATE SET value = '2928'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age, (citus_table_alias.key OPERATOR(pg_catalog.+) 1), (citus_table_alias.value OPERATOR(pg_catalog.||) '30'::text), (citus_table_alias.age OPERATOR(pg_catalog.*) 15) + key | value | age | ?column? | ?column? | ?column? +--------------------------------------------------------------------- + 1 | 2928 | 21 | 2 | 292830 | 315 +(1 row) + + EXECUTE local_insert_prepare_no_param; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1, '11'::text, '21'::bigint) ON CONFLICT(key) DO UPDATE SET value = '2928'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age, (citus_table_alias.key OPERATOR(pg_catalog.+) 1), (citus_table_alias.value OPERATOR(pg_catalog.||) '30'::text), (citus_table_alias.age OPERATOR(pg_catalog.*) 15) + key | value | age | ?column? | ?column? | ?column? +--------------------------------------------------------------------- + 1 | 2928 | 21 | 2 | 292830 | 315 +(1 row) + + EXECUTE local_insert_prepare_no_param; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1, '11'::text, '21'::bigint) ON CONFLICT(key) DO UPDATE SET value = '2928'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age, (citus_table_alias.key OPERATOR(pg_catalog.+) 1), (citus_table_alias.value OPERATOR(pg_catalog.||) '30'::text), (citus_table_alias.age OPERATOR(pg_catalog.*) 15) + key | value | age | ?column? | ?column? | ?column? +--------------------------------------------------------------------- + 1 | 2928 | 21 | 2 | 292830 | 315 +(1 row) + + EXECUTE local_insert_prepare_no_param; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1, '11'::text, '21'::bigint) ON CONFLICT(key) DO UPDATE SET value = '2928'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age, (citus_table_alias.key OPERATOR(pg_catalog.+) 1), (citus_table_alias.value OPERATOR(pg_catalog.||) '30'::text), (citus_table_alias.age OPERATOR(pg_catalog.*) 15) + key | value | age | ?column? | ?column? | ?column? +--------------------------------------------------------------------- + 1 | 2928 | 21 | 2 | 292830 | 315 +(1 row) + + -- 8 local executions with params + EXECUTE local_insert_prepare_param(1); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1, '11'::text, '21'::bigint) ON CONFLICT(key) DO UPDATE SET value = '2928'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age, (citus_table_alias.key OPERATOR(pg_catalog.+) 1), (citus_table_alias.value OPERATOR(pg_catalog.||) '30'::text), (citus_table_alias.age OPERATOR(pg_catalog.*) 15) + key | value | age | ?column? | ?column? | ?column? +--------------------------------------------------------------------- + 1 | 2928 | 21 | 2 | 292830 | 315 +(1 row) + + EXECUTE local_insert_prepare_param(5); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (5, '11'::text, '21'::bigint) ON CONFLICT(key) DO UPDATE SET value = '2928'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age, (citus_table_alias.key OPERATOR(pg_catalog.+) 1), (citus_table_alias.value OPERATOR(pg_catalog.||) '30'::text), (citus_table_alias.age OPERATOR(pg_catalog.*) 15) + key | value | age | ?column? | ?column? | ?column? +--------------------------------------------------------------------- + 5 | 2928 | 22 | 6 | 292830 | 330 +(1 row) + + EXECUTE local_insert_prepare_param(6); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500003 AS citus_table_alias (key, value, age) VALUES (6, '11'::text, '21'::bigint) ON CONFLICT(key) DO UPDATE SET value = '2928'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age, (citus_table_alias.key OPERATOR(pg_catalog.+) 1), (citus_table_alias.value OPERATOR(pg_catalog.||) '30'::text), (citus_table_alias.age OPERATOR(pg_catalog.*) 15) + key | value | age | ?column? | ?column? | ?column? +--------------------------------------------------------------------- + 6 | 11 | 21 | 7 | 1130 | 315 +(1 row) + + EXECUTE local_insert_prepare_param(1); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1, '11'::text, '21'::bigint) ON CONFLICT(key) DO UPDATE SET value = '2928'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age, (citus_table_alias.key OPERATOR(pg_catalog.+) 1), (citus_table_alias.value OPERATOR(pg_catalog.||) '30'::text), (citus_table_alias.age OPERATOR(pg_catalog.*) 15) + key | value | age | ?column? | ?column? | ?column? +--------------------------------------------------------------------- + 1 | 2928 | 21 | 2 | 292830 | 315 +(1 row) + + EXECUTE local_insert_prepare_param(5); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (5, '11'::text, '21'::bigint) ON CONFLICT(key) DO UPDATE SET value = '2928'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age, (citus_table_alias.key OPERATOR(pg_catalog.+) 1), (citus_table_alias.value OPERATOR(pg_catalog.||) '30'::text), (citus_table_alias.age OPERATOR(pg_catalog.*) 15) + key | value | age | ?column? | ?column? | ?column? +--------------------------------------------------------------------- + 5 | 2928 | 22 | 6 | 292830 | 330 +(1 row) + + EXECUTE local_insert_prepare_param(6); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500003 AS citus_table_alias (key, value, age) VALUES (6, '11'::text, '21'::bigint) ON CONFLICT(key) DO UPDATE SET value = '2928'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age, (citus_table_alias.key OPERATOR(pg_catalog.+) 1), (citus_table_alias.value OPERATOR(pg_catalog.||) '30'::text), (citus_table_alias.age OPERATOR(pg_catalog.*) 15) + key | value | age | ?column? | ?column? | ?column? +--------------------------------------------------------------------- + 6 | 2928 | 21 | 7 | 292830 | 315 +(1 row) + + EXECUTE local_insert_prepare_param(6); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500003 AS citus_table_alias (key, value, age) VALUES (6, '11'::text, '21'::bigint) ON CONFLICT(key) DO UPDATE SET value = '2928'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age, (citus_table_alias.key OPERATOR(pg_catalog.+) 1), (citus_table_alias.value OPERATOR(pg_catalog.||) '30'::text), (citus_table_alias.age OPERATOR(pg_catalog.*) 15) + key | value | age | ?column? | ?column? | ?column? +--------------------------------------------------------------------- + 6 | 2928 | 21 | 7 | 292830 | 315 +(1 row) + + EXECUTE local_insert_prepare_param(6); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500003 AS citus_table_alias (key, value, age) VALUES (6, '11'::text, '21'::bigint) ON CONFLICT(key) DO UPDATE SET value = '2928'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age, (citus_table_alias.key OPERATOR(pg_catalog.+) 1), (citus_table_alias.value OPERATOR(pg_catalog.||) '30'::text), (citus_table_alias.age OPERATOR(pg_catalog.*) 15) + key | value | age | ?column? | ?column? | ?column? +--------------------------------------------------------------------- + 6 | 2928 | 21 | 7 | 292830 | 315 +(1 row) + + -- followed by a non-local execution + EXECUTE remote_prepare_param(2); +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (key OPERATOR(pg_catalog.<>) 2) +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table WHERE (key OPERATOR(pg_catalog.<>) 2) +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE (key OPERATOR(pg_catalog.<>) 2) +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500004 distributed_table WHERE (key OPERATOR(pg_catalog.<>) 2) + count +--------------------------------------------------------------------- + 5 +(1 row) + +COMMIT; +PREPARE local_multi_row_insert_prepare_no_param AS + INSERT INTO distributed_table VALUES (1,'55', 21), (5,'15',33) ON CONFLICT (key) WHERE key > 3 and key < 4 DO UPDATE SET value = '88' || EXCLUDED.value; +PREPARE local_multi_row_insert_prepare_no_param_multi_shard AS + INSERT INTO distributed_table VALUES (6,'55', 21), (5,'15',33) ON CONFLICT (key) WHERE key > 3 AND key < 4 DO UPDATE SET value = '88' || EXCLUDED.value;; +PREPARE local_multi_row_insert_prepare_params(int,int) AS + INSERT INTO distributed_table VALUES ($1,'55', 21), ($2,'15',33) ON CONFLICT (key) WHERE key > 3 and key < 4 DO UPDATE SET value = '88' || EXCLUDED.value;; +INSERT INTO reference_table VALUES (11); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.reference_table_1500000 (key) VALUES (11) +BEGIN; + EXECUTE local_multi_row_insert_prepare_no_param; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1,'55'::text,'21'::bigint), (5,'15'::text,'33'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) + EXECUTE local_multi_row_insert_prepare_no_param; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1,'55'::text,'21'::bigint), (5,'15'::text,'33'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) + EXECUTE local_multi_row_insert_prepare_no_param; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1,'55'::text,'21'::bigint), (5,'15'::text,'33'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) + EXECUTE local_multi_row_insert_prepare_no_param; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1,'55'::text,'21'::bigint), (5,'15'::text,'33'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) + EXECUTE local_multi_row_insert_prepare_no_param; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1,'55'::text,'21'::bigint), (5,'15'::text,'33'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) + EXECUTE local_multi_row_insert_prepare_no_param; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1,'55'::text,'21'::bigint), (5,'15'::text,'33'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) + EXECUTE local_multi_row_insert_prepare_no_param; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1,'55'::text,'21'::bigint), (5,'15'::text,'33'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) + EXECUTE local_multi_row_insert_prepare_no_param; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1,'55'::text,'21'::bigint), (5,'15'::text,'33'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) + EXECUTE local_multi_row_insert_prepare_no_param_multi_shard; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (5,'15'::text,'33'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500003 AS citus_table_alias (key, value, age) VALUES (6,'55'::text,'21'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) + EXECUTE local_multi_row_insert_prepare_no_param_multi_shard; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (5,'15'::text,'33'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500003 AS citus_table_alias (key, value, age) VALUES (6,'55'::text,'21'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) + EXECUTE local_multi_row_insert_prepare_no_param_multi_shard; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (5,'15'::text,'33'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500003 AS citus_table_alias (key, value, age) VALUES (6,'55'::text,'21'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) + EXECUTE local_multi_row_insert_prepare_no_param_multi_shard; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (5,'15'::text,'33'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500003 AS citus_table_alias (key, value, age) VALUES (6,'55'::text,'21'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) + EXECUTE local_multi_row_insert_prepare_no_param_multi_shard; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (5,'15'::text,'33'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500003 AS citus_table_alias (key, value, age) VALUES (6,'55'::text,'21'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) + EXECUTE local_multi_row_insert_prepare_no_param_multi_shard; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (5,'15'::text,'33'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500003 AS citus_table_alias (key, value, age) VALUES (6,'55'::text,'21'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) + EXECUTE local_multi_row_insert_prepare_no_param_multi_shard; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (5,'15'::text,'33'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500003 AS citus_table_alias (key, value, age) VALUES (6,'55'::text,'21'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) + EXECUTE local_multi_row_insert_prepare_no_param_multi_shard; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (5,'15'::text,'33'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500003 AS citus_table_alias (key, value, age) VALUES (6,'55'::text,'21'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) + EXECUTE local_multi_row_insert_prepare_params(1,6); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1,'55'::text,'21'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500003 AS citus_table_alias (key, value, age) VALUES (6,'15'::text,'33'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) + EXECUTE local_multi_row_insert_prepare_params(1,5); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1,'55'::text,'21'::bigint), (5,'15'::text,'33'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) + EXECUTE local_multi_row_insert_prepare_params(6,5); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (5,'15'::text,'33'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500003 AS citus_table_alias (key, value, age) VALUES (6,'55'::text,'21'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) + EXECUTE local_multi_row_insert_prepare_params(5,1); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (5,'55'::text,'21'::bigint), (1,'15'::text,'33'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) + EXECUTE local_multi_row_insert_prepare_params(5,6); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (5,'55'::text,'21'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500003 AS citus_table_alias (key, value, age) VALUES (6,'15'::text,'33'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) + EXECUTE local_multi_row_insert_prepare_params(5,1); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (5,'55'::text,'21'::bigint), (1,'15'::text,'33'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) + EXECUTE local_multi_row_insert_prepare_params(1,6); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1,'55'::text,'21'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500003 AS citus_table_alias (key, value, age) VALUES (6,'15'::text,'33'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) + EXECUTE local_multi_row_insert_prepare_params(1,5); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1,'55'::text,'21'::bigint), (5,'15'::text,'33'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) + -- one task is remote + EXECUTE local_multi_row_insert_prepare_params(5,11); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (5,'55'::text,'21'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500004 AS citus_table_alias (key, value, age) VALUES (11,'15'::text,'33'::bigint) ON CONFLICT(key) WHERE ((key OPERATOR(pg_catalog.>) 3) AND (key OPERATOR(pg_catalog.<) 4)) DO UPDATE SET value = ('88'::text OPERATOR(pg_catalog.||) excluded.value) +ROLLBACK; +-- failures of local execution should rollback both the +-- local execution and remote executions +-- fail on a local execution +BEGIN; + INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '100' RETURNING *; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1, '11'::text, 21) ON CONFLICT(key) DO UPDATE SET value = '100'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age + key | value | age +--------------------------------------------------------------------- + 1 | 100 | 21 +(1 row) + + UPDATE distributed_table SET value = '200'; +NOTICE: executing the command locally: UPDATE local_shard_execution_replicated.distributed_table_1500001 distributed_table SET value = '200'::text +NOTICE: executing the command locally: UPDATE local_shard_execution_replicated.distributed_table_1500002 distributed_table SET value = '200'::text +NOTICE: executing the command locally: UPDATE local_shard_execution_replicated.distributed_table_1500003 distributed_table SET value = '200'::text +NOTICE: executing the command locally: UPDATE local_shard_execution_replicated.distributed_table_1500004 distributed_table SET value = '200'::text + INSERT INTO distributed_table VALUES (1, '100',21) ON CONFLICT(key) DO UPDATE SET value = (1 / (100.0 - EXCLUDED.value::int))::text RETURNING *; +ERROR: division by zero +CONTEXT: while executing command on localhost:xxxxx +ROLLBACK; +-- we've rollbacked everything +SELECT count(*) FROM distributed_table WHERE value = '200'; + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- RETURNING should just work fine for reference tables +INSERT INTO reference_table VALUES (500) RETURNING *; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.reference_table_1500000 (key) VALUES (500) RETURNING key + key +--------------------------------------------------------------------- + 500 +(1 row) + +DELETE FROM reference_table WHERE key = 500 RETURNING *; +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.reference_table_1500000 reference_table WHERE (key OPERATOR(pg_catalog.=) 500) RETURNING key + key +--------------------------------------------------------------------- + 500 +(1 row) + +-- should be able to skip local execution even if in a sequential mode of execution +BEGIN; + SET LOCAL citus.multi_shard_modify_mode TO sequential ; + DELETE FROM distributed_table; +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500004 distributed_table + INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '100' RETURNING *; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1, '11'::text, 21) ON CONFLICT(key) DO UPDATE SET value = '100'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age + key | value | age +--------------------------------------------------------------------- + 1 | 11 | 21 +(1 row) + +ROLLBACK; +-- sequential execution should just work fine after a local execution +BEGIN; + SET citus.multi_shard_modify_mode TO sequential ; + INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '100' RETURNING *; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 AS citus_table_alias (key, value, age) VALUES (1, '11'::text, 21) ON CONFLICT(key) DO UPDATE SET value = '100'::text RETURNING citus_table_alias.key, citus_table_alias.value, citus_table_alias.age + key | value | age +--------------------------------------------------------------------- + 1 | 100 | 21 +(1 row) + + DELETE FROM distributed_table; +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500004 distributed_table +ROLLBACK; +-- load some data so that foreign keys won't complain with the next tests +TRUNCATE reference_table CASCADE; +NOTICE: executing the command locally: TRUNCATE TABLE local_shard_execution_replicated.reference_table_xxxxx CASCADE +INSERT INTO reference_table SELECT i FROM generate_series(500, 600) i; +NOTICE: executing the copy locally for shard xxxxx +INSERT INTO distributed_table SELECT i, i::text, i % 10 + 25 FROM generate_series(500, 600) i; +NOTICE: executing the copy locally for shard xxxxx +NOTICE: executing the copy locally for shard xxxxx +NOTICE: executing the copy locally for shard xxxxx +NOTICE: executing the copy locally for shard xxxxx +-- show that both local, and mixed local-distributed executions +-- calculate rows processed correctly +BEGIN; + DELETE FROM distributed_table WHERE key = 500; +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE (key OPERATOR(pg_catalog.=) 500) + DELETE FROM distributed_table WHERE value != '123123213123213'; +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE (value OPERATOR(pg_catalog.<>) '123123213123213'::text) +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table WHERE (value OPERATOR(pg_catalog.<>) '123123213123213'::text) +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE (value OPERATOR(pg_catalog.<>) '123123213123213'::text) +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500004 distributed_table WHERE (value OPERATOR(pg_catalog.<>) '123123213123213'::text) +ROLLBACK; +BEGIN; + DELETE FROM reference_table WHERE key = 500 RETURNING *; +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.reference_table_1500000 reference_table WHERE (key OPERATOR(pg_catalog.=) 500) RETURNING key + key +--------------------------------------------------------------------- + 500 +(1 row) + + DELETE FROM reference_table; +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.reference_table_1500000 reference_table +ROLLBACK; +BEGIN; + DELETE FROM distributed_table WHERE key = 500; +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE (key OPERATOR(pg_catalog.=) 500) + SELECT count(*) FROM distributed_table; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500004 distributed_table WHERE true + count +--------------------------------------------------------------------- + 106 +(1 row) + +ROLLBACK; +BEGIN; + SET LOCAL client_min_messages TO INFO; + SELECT count(*) FROM distributed_table; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500004 distributed_table WHERE true + count +--------------------------------------------------------------------- + 107 +(1 row) + + SET LOCAL client_min_messages TO LOG; + DELETE FROM distributed_table WHERE key = 500; +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE (key OPERATOR(pg_catalog.=) 500) +ROLLBACK; +-- probably not a realistic case since views are not very +-- well supported with MX +CREATE VIEW v_local_query_execution AS +SELECT * FROM distributed_table WHERE key = 500; +SELECT * FROM v_local_query_execution; +NOTICE: executing the command locally: SELECT key, value, age FROM (SELECT distributed_table.key, distributed_table.value, distributed_table.age FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE (distributed_table.key OPERATOR(pg_catalog.=) 500)) v_local_query_execution + key | value | age +--------------------------------------------------------------------- + 500 | 500 | 25 +(1 row) + +-- similar test, but this time the view itself is a non-local +-- query, but the query on the view is local +CREATE VIEW v_local_query_execution_2 AS +SELECT * FROM distributed_table; +SELECT * FROM v_local_query_execution_2 WHERE key = 500; +NOTICE: executing the command locally: SELECT key, value, age FROM (SELECT distributed_table.key, distributed_table.value, distributed_table.age FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table) v_local_query_execution_2 WHERE (key OPERATOR(pg_catalog.=) 500) + key | value | age +--------------------------------------------------------------------- + 500 | 500 | 25 +(1 row) + +-- even if we switch from remote execution -> local execution, +-- we are able to use remote execution after rollback +BEGIN; + SAVEPOINT my_savepoint; + SELECT count(*) FROM distributed_table; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500004 distributed_table WHERE true + count +--------------------------------------------------------------------- + 107 +(1 row) + + DELETE FROM distributed_table WHERE key = 500; +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE (key OPERATOR(pg_catalog.=) 500) + ROLLBACK TO SAVEPOINT my_savepoint; + DELETE FROM distributed_table WHERE key = 500; +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE (key OPERATOR(pg_catalog.=) 500) +COMMIT; +-- even if we switch from local execution -> remote execution, +-- we are able to use local execution after rollback +BEGIN; + SAVEPOINT my_savepoint; + DELETE FROM distributed_table WHERE key = 500; +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE (key OPERATOR(pg_catalog.=) 500) + SELECT count(*) FROM distributed_table; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500004 distributed_table WHERE true + count +--------------------------------------------------------------------- + 106 +(1 row) + + ROLLBACK TO SAVEPOINT my_savepoint; + DELETE FROM distributed_table WHERE key = 500; +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table WHERE (key OPERATOR(pg_catalog.=) 500) +COMMIT; +-- sanity check: local execution on partitions +INSERT INTO collections_list (collection_id) VALUES (0) RETURNING *; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.collections_list_1500011 (key, ser, collection_id) VALUES ('3940649673949185'::bigint, '3940649673949185'::bigint, 0) RETURNING key, ser, ts, collection_id, value + key | ser | ts | collection_id | value +--------------------------------------------------------------------- + 3940649673949185 | 3940649673949185 | | 0 | +(1 row) + +BEGIN; + INSERT INTO collections_list (key, collection_id) VALUES (1,0); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.collections_list_1500009 (key, ser, collection_id) VALUES ('1'::bigint, '3940649673949186'::bigint, 0) + SELECT count(*) FROM collections_list_0 WHERE key = 1; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.collections_list_0_1500013 collections_list_0 WHERE (key OPERATOR(pg_catalog.=) 1) + count +--------------------------------------------------------------------- + 1 +(1 row) + + SELECT count(*) FROM collections_list; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.collections_list_1500009 collections_list WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.collections_list_1500010 collections_list WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.collections_list_1500011 collections_list WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.collections_list_1500012 collections_list WHERE true + count +--------------------------------------------------------------------- + 2 +(1 row) + + SELECT * FROM collections_list ORDER BY 1,2,3,4; +NOTICE: executing the command locally: SELECT key, ser, ts, collection_id, value FROM local_shard_execution_replicated.collections_list_1500009 collections_list WHERE true +NOTICE: executing the command locally: SELECT key, ser, ts, collection_id, value FROM local_shard_execution_replicated.collections_list_1500010 collections_list WHERE true +NOTICE: executing the command locally: SELECT key, ser, ts, collection_id, value FROM local_shard_execution_replicated.collections_list_1500011 collections_list WHERE true +NOTICE: executing the command locally: SELECT key, ser, ts, collection_id, value FROM local_shard_execution_replicated.collections_list_1500012 collections_list WHERE true + key | ser | ts | collection_id | value +--------------------------------------------------------------------- + 1 | 3940649673949186 | | 0 | + 3940649673949185 | 3940649673949185 | | 0 | +(2 rows) + +COMMIT; +TRUNCATE collections_list; +-- make sure that even if local execution is used, the sequence values +-- are generated locally +ALTER SEQUENCE collections_list_key_seq NO MINVALUE NO MAXVALUE; +PREPARE serial_prepared_local AS INSERT INTO collections_list (collection_id) VALUES (0) RETURNING key, ser; +SELECT setval('collections_list_key_seq', 4); + setval +--------------------------------------------------------------------- + 4 +(1 row) + +EXECUTE serial_prepared_local; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.collections_list_1500009 (key, ser, collection_id) VALUES ('5'::bigint, '3940649673949187'::bigint, 0) RETURNING key, ser + key | ser +--------------------------------------------------------------------- + 5 | 3940649673949187 +(1 row) + +SELECT setval('collections_list_key_seq', 5); + setval +--------------------------------------------------------------------- + 5 +(1 row) + +EXECUTE serial_prepared_local; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.collections_list_1500011 (key, ser, collection_id) VALUES ('6'::bigint, '3940649673949188'::bigint, 0) RETURNING key, ser + key | ser +--------------------------------------------------------------------- + 6 | 3940649673949188 +(1 row) + +SELECT setval('collections_list_key_seq', 499); + setval +--------------------------------------------------------------------- + 499 +(1 row) + +EXECUTE serial_prepared_local; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.collections_list_1500011 (key, ser, collection_id) VALUES ('500'::bigint, '3940649673949189'::bigint, 0) RETURNING key, ser + key | ser +--------------------------------------------------------------------- + 500 | 3940649673949189 +(1 row) + +SELECT setval('collections_list_key_seq', 700); + setval +--------------------------------------------------------------------- + 700 +(1 row) + +EXECUTE serial_prepared_local; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.collections_list_1500009 (key, ser, collection_id) VALUES ('701'::bigint, '3940649673949190'::bigint, 0) RETURNING key, ser + key | ser +--------------------------------------------------------------------- + 701 | 3940649673949190 +(1 row) + +SELECT setval('collections_list_key_seq', 708); + setval +--------------------------------------------------------------------- + 708 +(1 row) + +EXECUTE serial_prepared_local; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.collections_list_1500011 (key, ser, collection_id) VALUES ('709'::bigint, '3940649673949191'::bigint, 0) RETURNING key, ser + key | ser +--------------------------------------------------------------------- + 709 | 3940649673949191 +(1 row) + +SELECT setval('collections_list_key_seq', 709); + setval +--------------------------------------------------------------------- + 709 +(1 row) + +EXECUTE serial_prepared_local; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.collections_list_1500009 (key, ser, collection_id) VALUES ('710'::bigint, '3940649673949192'::bigint, 0) RETURNING key, ser + key | ser +--------------------------------------------------------------------- + 710 | 3940649673949192 +(1 row) + +-- get ready for the next executions +DELETE FROM collections_list WHERE key IN (5,6); +SELECT setval('collections_list_key_seq', 4); + setval +--------------------------------------------------------------------- + 4 +(1 row) + +EXECUTE serial_prepared_local; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.collections_list_1500009 (key, ser, collection_id) VALUES ('5'::bigint, '3940649673949193'::bigint, 0) RETURNING key, ser + key | ser +--------------------------------------------------------------------- + 5 | 3940649673949193 +(1 row) + +SELECT setval('collections_list_key_seq', 5); + setval +--------------------------------------------------------------------- + 5 +(1 row) + +EXECUTE serial_prepared_local; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.collections_list_1500011 (key, ser, collection_id) VALUES ('6'::bigint, '3940649673949194'::bigint, 0) RETURNING key, ser + key | ser +--------------------------------------------------------------------- + 6 | 3940649673949194 +(1 row) + +-- and, one remote test +SELECT setval('collections_list_key_seq', 10); + setval +--------------------------------------------------------------------- + 10 +(1 row) + +EXECUTE serial_prepared_local; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.collections_list_1500012 (key, ser, collection_id) VALUES ('11'::bigint, '3940649673949195'::bigint, 0) RETURNING key, ser + key | ser +--------------------------------------------------------------------- + 11 | 3940649673949195 +(1 row) + +-- the final queries for the following CTEs are going to happen on the intermediate results only +-- one of them will be executed remotely, and the other is locally +-- Citus currently doesn't allow using task_assignment_policy for intermediate results +WITH distributed_local_mixed AS (INSERT INTO reference_table VALUES (1000) RETURNING *) SELECT * FROM distributed_local_mixed; +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.reference_table_1500000 (key) VALUES (1000) RETURNING key +NOTICE: executing the command locally: SELECT key FROM (SELECT intermediate_result.key FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer)) distributed_local_mixed + key +--------------------------------------------------------------------- + 1000 +(1 row) + +-- clean the table for the next tests +SET search_path TO local_shard_execution_replicated; +TRUNCATE distributed_table CASCADE; +-- load some data on a remote shard +INSERT INTO reference_table (key) VALUES (1), (2); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.reference_table_1500000 AS citus_table_alias (key) VALUES (1), (2) +INSERT INTO distributed_table (key) VALUES (2); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500004 (key) VALUES (2) +BEGIN; + -- local execution followed by a distributed query + INSERT INTO distributed_table (key) VALUES (1); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.distributed_table_1500001 (key) VALUES (1) + DELETE FROM distributed_table RETURNING key; +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table RETURNING key +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table RETURNING key +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500003 distributed_table RETURNING key +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.distributed_table_1500004 distributed_table RETURNING key + key +--------------------------------------------------------------------- + 1 + 2 +(2 rows) + +COMMIT; +-- a similar test with a reference table +TRUNCATE reference_table CASCADE; +NOTICE: executing the command locally: TRUNCATE TABLE local_shard_execution_replicated.reference_table_xxxxx CASCADE +-- load some data on a remote shard +INSERT INTO reference_table (key) VALUES (2); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.reference_table_1500000 (key) VALUES (2) +BEGIN; + -- local execution followed by a distributed query + INSERT INTO reference_table (key) VALUES (1); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.reference_table_1500000 (key) VALUES (1) + DELETE FROM reference_table RETURNING key; +NOTICE: executing the command locally: DELETE FROM local_shard_execution_replicated.reference_table_1500000 reference_table RETURNING key + key +--------------------------------------------------------------------- + 1 + 2 +(2 rows) + +COMMIT; +-- however complex the query, local execution can handle +SET client_min_messages TO LOG; +SET citus.log_local_commands TO ON; +WITH cte_1 AS + (SELECT * + FROM + (WITH cte_1 AS + (SELECT * + FROM distributed_table + WHERE key = 1) SELECT * + FROM cte_1) AS foo) +SELECT count(*) +FROM cte_1 +JOIN distributed_table USING (key) +WHERE distributed_table.key = 1 + AND distributed_table.key IN + (SELECT key + FROM distributed_table + WHERE key = 1); +NOTICE: executing the command locally: SELECT count(*) AS count FROM ((SELECT foo.key, foo.value, foo.age FROM (SELECT cte_1_1.key, cte_1_1.value, cte_1_1.age FROM (SELECT distributed_table_1.key, distributed_table_1.value, distributed_table_1.age FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table_1 WHERE (distributed_table_1.key OPERATOR(pg_catalog.=) 1)) cte_1_1) foo) cte_1 JOIN local_shard_execution_replicated.distributed_table_1500001 distributed_table(key, value, age) USING (key)) WHERE ((distributed_table.key OPERATOR(pg_catalog.=) 1) AND (distributed_table.key OPERATOR(pg_catalog.=) ANY (SELECT distributed_table_1.key FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table_1 WHERE (distributed_table_1.key OPERATOR(pg_catalog.=) 1)))) + count +--------------------------------------------------------------------- + 0 +(1 row) + +RESET client_min_messages; +RESET citus.log_local_commands; +\c - - - :master_port +SET citus.next_shard_id TO 1501000; +-- test both local and remote execution with custom type +SET citus.shard_replication_factor TO 2; +SET search_path TO local_shard_execution_replicated; +CREATE TYPE invite_resp AS ENUM ('yes', 'no', 'maybe'); +CREATE TABLE event_responses ( + event_id int, + user_id int, + response invite_resp, + primary key (event_id, user_id) +); +SELECT create_distributed_table('event_responses', 'event_id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO event_responses VALUES (1, 1, 'yes'), (2, 2, 'yes'), (3, 3, 'no'), (4, 4, 'no'); +CREATE OR REPLACE FUNCTION regular_func(p invite_resp) +RETURNS int AS $$ +DECLARE + q1Result INT; + q2Result INT; + q3Result INT; +BEGIN +SELECT count(*) INTO q1Result FROM event_responses WHERE response = $1; +SELECT count(*) INTO q2Result FROM event_responses e1 LEFT JOIN event_responses e2 USING (event_id) WHERE e2.response = $1; +SELECT count(*) INTO q3Result FROM (SELECT * FROM event_responses WHERE response = $1 LIMIT 5) as foo; +RETURN q3Result+q2Result+q1Result; +END; +$$ LANGUAGE plpgsql; +SELECT regular_func('yes'); + regular_func +--------------------------------------------------------------------- + 6 +(1 row) + +SELECT regular_func('yes'); + regular_func +--------------------------------------------------------------------- + 6 +(1 row) + +SELECT regular_func('yes'); + regular_func +--------------------------------------------------------------------- + 6 +(1 row) + +SELECT regular_func('yes'); + regular_func +--------------------------------------------------------------------- + 6 +(1 row) + +SELECT regular_func('yes'); + regular_func +--------------------------------------------------------------------- + 6 +(1 row) + +SELECT regular_func('yes'); + regular_func +--------------------------------------------------------------------- + 6 +(1 row) + +SELECT regular_func('yes'); + regular_func +--------------------------------------------------------------------- + 6 +(1 row) + +SELECT regular_func('yes'); + regular_func +--------------------------------------------------------------------- + 6 +(1 row) + +CREATE OR REPLACE PROCEDURE regular_procedure(p invite_resp) +AS $$ +BEGIN +PERFORM * FROM event_responses WHERE response = $1 ORDER BY 1 DESC, 2 DESC, 3 DESC; +PERFORM * FROM event_responses e1 LEFT JOIN event_responses e2 USING (event_id) WHERE e2.response = $1 ORDER BY 1 DESC, 2 DESC, 3 DESC, 4 DESC; +PERFORM * FROM (SELECT * FROM event_responses WHERE response = $1 LIMIT 5) as foo ORDER BY 1 DESC, 2 DESC, 3 DESC; +END; +$$ LANGUAGE plpgsql; +CALL regular_procedure('no'); +CALL regular_procedure('no'); +CALL regular_procedure('no'); +CALL regular_procedure('no'); +CALL regular_procedure('no'); +CALL regular_procedure('no'); +CALL regular_procedure('no'); +CALL regular_procedure('no'); +PREPARE multi_shard_no_dist_key(invite_resp) AS select * from event_responses where response = $1::invite_resp ORDER BY 1 DESC, 2 DESC, 3 DESC LIMIT 1; +EXECUTE multi_shard_no_dist_key('yes'); + event_id | user_id | response +--------------------------------------------------------------------- + 2 | 2 | yes +(1 row) + +EXECUTE multi_shard_no_dist_key('yes'); + event_id | user_id | response +--------------------------------------------------------------------- + 2 | 2 | yes +(1 row) + +EXECUTE multi_shard_no_dist_key('yes'); + event_id | user_id | response +--------------------------------------------------------------------- + 2 | 2 | yes +(1 row) + +EXECUTE multi_shard_no_dist_key('yes'); + event_id | user_id | response +--------------------------------------------------------------------- + 2 | 2 | yes +(1 row) + +EXECUTE multi_shard_no_dist_key('yes'); + event_id | user_id | response +--------------------------------------------------------------------- + 2 | 2 | yes +(1 row) + +EXECUTE multi_shard_no_dist_key('yes'); + event_id | user_id | response +--------------------------------------------------------------------- + 2 | 2 | yes +(1 row) + +EXECUTE multi_shard_no_dist_key('yes'); + event_id | user_id | response +--------------------------------------------------------------------- + 2 | 2 | yes +(1 row) + +EXECUTE multi_shard_no_dist_key('yes'); + event_id | user_id | response +--------------------------------------------------------------------- + 2 | 2 | yes +(1 row) + +PREPARE multi_shard_with_dist_key(int, invite_resp) AS select * from event_responses where event_id > $1 AND response = $2::invite_resp ORDER BY 1 DESC, 2 DESC, 3 DESC LIMIT 1; +EXECUTE multi_shard_with_dist_key(1, 'yes'); + event_id | user_id | response +--------------------------------------------------------------------- + 2 | 2 | yes +(1 row) + +EXECUTE multi_shard_with_dist_key(1, 'yes'); + event_id | user_id | response +--------------------------------------------------------------------- + 2 | 2 | yes +(1 row) + +EXECUTE multi_shard_with_dist_key(1, 'yes'); + event_id | user_id | response +--------------------------------------------------------------------- + 2 | 2 | yes +(1 row) + +EXECUTE multi_shard_with_dist_key(1, 'yes'); + event_id | user_id | response +--------------------------------------------------------------------- + 2 | 2 | yes +(1 row) + +EXECUTE multi_shard_with_dist_key(1, 'yes'); + event_id | user_id | response +--------------------------------------------------------------------- + 2 | 2 | yes +(1 row) + +EXECUTE multi_shard_with_dist_key(1, 'yes'); + event_id | user_id | response +--------------------------------------------------------------------- + 2 | 2 | yes +(1 row) + +EXECUTE multi_shard_with_dist_key(1, 'yes'); + event_id | user_id | response +--------------------------------------------------------------------- + 2 | 2 | yes +(1 row) + +EXECUTE multi_shard_with_dist_key(1, 'yes'); + event_id | user_id | response +--------------------------------------------------------------------- + 2 | 2 | yes +(1 row) + +PREPARE query_pushdown_no_dist_key(invite_resp) AS select * from event_responses e1 LEFT JOIN event_responses e2 USING(event_id) where e1.response = $1::invite_resp ORDER BY 1 DESC, 2 DESC, 3 DESC, 4 DESC LIMIT 1; +EXECUTE query_pushdown_no_dist_key('yes'); + event_id | user_id | response | user_id | response +--------------------------------------------------------------------- + 2 | 2 | yes | 2 | yes +(1 row) + +EXECUTE query_pushdown_no_dist_key('yes'); + event_id | user_id | response | user_id | response +--------------------------------------------------------------------- + 2 | 2 | yes | 2 | yes +(1 row) + +EXECUTE query_pushdown_no_dist_key('yes'); + event_id | user_id | response | user_id | response +--------------------------------------------------------------------- + 2 | 2 | yes | 2 | yes +(1 row) + +EXECUTE query_pushdown_no_dist_key('yes'); + event_id | user_id | response | user_id | response +--------------------------------------------------------------------- + 2 | 2 | yes | 2 | yes +(1 row) + +EXECUTE query_pushdown_no_dist_key('yes'); + event_id | user_id | response | user_id | response +--------------------------------------------------------------------- + 2 | 2 | yes | 2 | yes +(1 row) + +EXECUTE query_pushdown_no_dist_key('yes'); + event_id | user_id | response | user_id | response +--------------------------------------------------------------------- + 2 | 2 | yes | 2 | yes +(1 row) + +EXECUTE query_pushdown_no_dist_key('yes'); + event_id | user_id | response | user_id | response +--------------------------------------------------------------------- + 2 | 2 | yes | 2 | yes +(1 row) + +EXECUTE query_pushdown_no_dist_key('yes'); + event_id | user_id | response | user_id | response +--------------------------------------------------------------------- + 2 | 2 | yes | 2 | yes +(1 row) + +PREPARE insert_select_via_coord(invite_resp) AS INSERT INTO event_responses SELECT * FROM event_responses where response = $1::invite_resp LIMIT 1 ON CONFLICT (event_id, user_id) DO NOTHING ; +EXECUTE insert_select_via_coord('yes'); +EXECUTE insert_select_via_coord('yes'); +EXECUTE insert_select_via_coord('yes'); +EXECUTE insert_select_via_coord('yes'); +EXECUTE insert_select_via_coord('yes'); +EXECUTE insert_select_via_coord('yes'); +EXECUTE insert_select_via_coord('yes'); +EXECUTE insert_select_via_coord('yes'); +PREPARE insert_select_pushdown(invite_resp) AS INSERT INTO event_responses SELECT * FROM event_responses where response = $1::invite_resp ON CONFLICT (event_id, user_id) DO NOTHING; +EXECUTE insert_select_pushdown('yes'); +EXECUTE insert_select_pushdown('yes'); +EXECUTE insert_select_pushdown('yes'); +EXECUTE insert_select_pushdown('yes'); +EXECUTE insert_select_pushdown('yes'); +EXECUTE insert_select_pushdown('yes'); +EXECUTE insert_select_pushdown('yes'); +EXECUTE insert_select_pushdown('yes'); +PREPARE router_select_with_no_dist_key_filter(invite_resp) AS select * from event_responses where event_id = 1 AND response = $1::invite_resp ORDER BY 1 DESC, 2 DESC, 3 DESC LIMIT 1; +EXECUTE router_select_with_no_dist_key_filter('yes'); + event_id | user_id | response +--------------------------------------------------------------------- + 1 | 1 | yes +(1 row) + +EXECUTE router_select_with_no_dist_key_filter('yes'); + event_id | user_id | response +--------------------------------------------------------------------- + 1 | 1 | yes +(1 row) + +EXECUTE router_select_with_no_dist_key_filter('yes'); + event_id | user_id | response +--------------------------------------------------------------------- + 1 | 1 | yes +(1 row) + +EXECUTE router_select_with_no_dist_key_filter('yes'); + event_id | user_id | response +--------------------------------------------------------------------- + 1 | 1 | yes +(1 row) + +EXECUTE router_select_with_no_dist_key_filter('yes'); + event_id | user_id | response +--------------------------------------------------------------------- + 1 | 1 | yes +(1 row) + +EXECUTE router_select_with_no_dist_key_filter('yes'); + event_id | user_id | response +--------------------------------------------------------------------- + 1 | 1 | yes +(1 row) + +EXECUTE router_select_with_no_dist_key_filter('yes'); + event_id | user_id | response +--------------------------------------------------------------------- + 1 | 1 | yes +(1 row) + +EXECUTE router_select_with_no_dist_key_filter('yes'); + event_id | user_id | response +--------------------------------------------------------------------- + 1 | 1 | yes +(1 row) + +-- rest of the tests assume the table is empty +TRUNCATE event_responses; +CREATE OR REPLACE PROCEDURE register_for_event(p_event_id int, p_user_id int, p_choice invite_resp) +LANGUAGE plpgsql AS $fn$ +BEGIN + INSERT INTO local_shard_execution_replicated.event_responses VALUES (p_event_id, p_user_id, p_choice) + ON CONFLICT (event_id, user_id) + DO UPDATE SET response = EXCLUDED.response; + + PERFORM count(*) FROM local_shard_execution_replicated.event_responses WHERE event_id = p_event_id; + + PERFORM count(*) FROM local_shard_execution_replicated.event_responses WHERE event_id = p_event_id AND false; + + UPDATE local_shard_execution_replicated.event_responses SET response = p_choice WHERE event_id = p_event_id; + +END; +$fn$; +SELECT create_distributed_function('register_for_event(int,int,invite_resp)'); + create_distributed_function +--------------------------------------------------------------------- + +(1 row) + +-- call 8 times to make sure it works after the 5th time(postgres binds values after the 5th time and Citus 2nd time) +-- after 6th, the local execution caches the local plans and uses it +-- execute it both locally and remotely +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +\c - - - :worker_2_port +SET search_path TO local_shard_execution_replicated; +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +-- values 16, 17 and 19 hits the same +-- shard, so we're re-using the same cached +-- plans per statement across different distribution +-- key values +CALL register_for_event(17, 1, 'yes'); +CALL register_for_event(19, 1, 'yes'); +CALL register_for_event(17, 1, 'yes'); +CALL register_for_event(19, 1, 'yes'); +-- should work fine if the logs are enabled +\set VERBOSITY terse +SET citus.log_local_commands TO ON; +SET client_min_messages TO DEBUG2; +CALL register_for_event(19, 1, 'yes'); +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.event_responses_1501001 AS citus_table_alias (event_id, user_id, response) VALUES (19, 1, 'yes'::local_shard_execution_replicated.invite_resp) ON CONFLICT(event_id, user_id) DO UPDATE SET response = excluded.response +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.event_responses_1501001 event_responses WHERE (event_id OPERATOR(pg_catalog.=) 19) +NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT NULL::integer AS event_id, NULL::integer AS user_id, NULL::local_shard_execution_replicated.invite_resp AS response WHERE false) event_responses(event_id, user_id, response) WHERE ((event_id OPERATOR(pg_catalog.=) 19) AND false) +NOTICE: executing the command locally: UPDATE local_shard_execution_replicated.event_responses_1501001 event_responses SET response = 'yes'::local_shard_execution_replicated.invite_resp WHERE (event_id OPERATOR(pg_catalog.=) 19) +-- should be fine even if no parameters exists in the query +SELECT count(*) FROM event_responses WHERE event_id = 16; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: query has a single distribution column value: 16 +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.event_responses_1501001 event_responses WHERE (event_id OPERATOR(pg_catalog.=) 16) + count +--------------------------------------------------------------------- + 1 +(1 row) + +SELECT count(*) FROM event_responses WHERE event_id = 16; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: query has a single distribution column value: 16 +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.event_responses_1501001 event_responses WHERE (event_id OPERATOR(pg_catalog.=) 16) + count +--------------------------------------------------------------------- + 1 +(1 row) + +UPDATE event_responses SET response = 'no' WHERE event_id = 16; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: query has a single distribution column value: 16 +NOTICE: executing the command locally: UPDATE local_shard_execution_replicated.event_responses_1501001 event_responses SET response = 'no'::local_shard_execution_replicated.invite_resp WHERE (event_id OPERATOR(pg_catalog.=) 16) +INSERT INTO event_responses VALUES (16, 666, 'maybe') +ON CONFLICT (event_id, user_id) +DO UPDATE SET response = EXCLUDED.response RETURNING *; +DEBUG: Creating router plan +DEBUG: query has a single distribution column value: 16 +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.event_responses_1501001 AS citus_table_alias (event_id, user_id, response) VALUES (16, 666, 'maybe'::local_shard_execution_replicated.invite_resp) ON CONFLICT(event_id, user_id) DO UPDATE SET response = excluded.response RETURNING citus_table_alias.event_id, citus_table_alias.user_id, citus_table_alias.response + event_id | user_id | response +--------------------------------------------------------------------- + 16 | 666 | maybe +(1 row) + +-- multi row INSERTs hitting the same shard +INSERT INTO event_responses VALUES (16, 666, 'maybe'), (17, 777, 'no') +ON CONFLICT (event_id, user_id) +DO UPDATE SET response = EXCLUDED.response RETURNING *; +DEBUG: Creating router plan +NOTICE: executing the command locally: INSERT INTO local_shard_execution_replicated.event_responses_1501001 AS citus_table_alias (event_id, user_id, response) VALUES (16,666,'maybe'::local_shard_execution_replicated.invite_resp), (17,777,'no'::local_shard_execution_replicated.invite_resp) ON CONFLICT(event_id, user_id) DO UPDATE SET response = excluded.response RETURNING citus_table_alias.event_id, citus_table_alias.user_id, citus_table_alias.response + event_id | user_id | response +--------------------------------------------------------------------- + 16 | 666 | maybe + 17 | 777 | no +(2 rows) + +-- now, similar tests with some settings changed +SET citus.enable_local_execution TO false; +SET citus.enable_fast_path_router_planner TO false; +CALL register_for_event(19, 1, 'yes'); +-- should be fine even if no parameters exists in the query +SELECT count(*) FROM event_responses WHERE event_id = 16; +DEBUG: Creating router plan +DEBUG: query has a single distribution column value: 16 + count +--------------------------------------------------------------------- + 2 +(1 row) + +SELECT count(*) FROM event_responses WHERE event_id = 16; +DEBUG: Creating router plan +DEBUG: query has a single distribution column value: 16 + count +--------------------------------------------------------------------- + 2 +(1 row) + +UPDATE event_responses SET response = 'no' WHERE event_id = 16; +DEBUG: Creating router plan +DEBUG: query has a single distribution column value: 16 +INSERT INTO event_responses VALUES (16, 666, 'maybe') +ON CONFLICT (event_id, user_id) +DO UPDATE SET response = EXCLUDED.response RETURNING *; +DEBUG: Creating router plan +DEBUG: query has a single distribution column value: 16 + event_id | user_id | response +--------------------------------------------------------------------- + 16 | 666 | maybe +(1 row) + +-- multi row INSERTs hitting the same shard +INSERT INTO event_responses VALUES (16, 666, 'maybe'), (17, 777, 'no') +ON CONFLICT (event_id, user_id) +DO UPDATE SET response = EXCLUDED.response RETURNING *; +DEBUG: Creating router plan + event_id | user_id | response +--------------------------------------------------------------------- + 16 | 666 | maybe + 17 | 777 | no +(2 rows) + +-- not allow commands over the workers when user disables +SET citus.allow_modifications_from_workers_to_replicated_tables TO false; +INSERT INTO event_responses VALUES (16, 666, 'maybe'), (17, 777, 'no') +ON CONFLICT (event_id, user_id) +DO UPDATE SET response = EXCLUDED.response RETURNING *; +ERROR: modifications via the worker nodes are not allowed for replicated tables such as reference tables or hash distributed tables with replication factor greater than 1. +\c - - - :master_port +SET client_min_messages TO ERROR; +SET search_path TO public; +DROP SCHEMA local_shard_execution_replicated CASCADE; diff --git a/src/test/regress/expected/master_copy_shard_placement.out b/src/test/regress/expected/master_copy_shard_placement.out index a038046b6..aebc96922 100644 --- a/src/test/regress/expected/master_copy_shard_placement.out +++ b/src/test/regress/expected/master_copy_shard_placement.out @@ -107,15 +107,9 @@ SELECT count(*) FROM history; 2 (1 row) --- test we can not replicate MX tables +-- test we can replicate MX tables SET citus.shard_replication_factor TO 1; --- metadata sync will fail as we have a statement replicated table -SELECT start_metadata_sync_to_node('localhost', :worker_1_port); -ERROR: relation "mcsp.history" does not exist -CONTEXT: while executing command on localhost:xxxxx --- use streaming replication to enable metadata syncing -UPDATE pg_dist_partition SET repmodel='s' WHERE logicalrelid IN - ('history'::regclass); +-- metadata sync will succeed even if we have rep > 1 tables SELECT start_metadata_sync_to_node('localhost', :worker_1_port); start_metadata_sync_to_node --------------------------------------------------------------------- diff --git a/src/test/regress/expected/metadata_sync_helpers.out b/src/test/regress/expected/metadata_sync_helpers.out index 58678d8f6..06480c339 100644 --- a/src/test/regress/expected/metadata_sync_helpers.out +++ b/src/test/regress/expected/metadata_sync_helpers.out @@ -357,18 +357,6 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; SELECT citus_internal_add_partition_metadata ('test_ref'::regclass, 'n', NULL, 0, 'c'); ERROR: Local or references tables can only have 's' or 't' as the replication model. ROLLBACK; --- not-matching replication model for hash table -BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; - SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02'); - assign_distributed_transaction_id ---------------------------------------------------------------------- - -(1 row) - - SET application_name to 'citus'; - SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 0, 't'); -ERROR: Hash distributed tables can only have 's' as the replication model. -ROLLBACK; -- add entry for super user table \c - postgres - :worker_1_port SET search_path TO metadata_sync_helpers; diff --git a/src/test/regress/expected/multi_colocation_utils.out b/src/test/regress/expected/multi_colocation_utils.out index f017f740c..886638b3c 100644 --- a/src/test/regress/expected/multi_colocation_utils.out +++ b/src/test/regress/expected/multi_colocation_utils.out @@ -428,6 +428,7 @@ SELECT create_distributed_table('table_range', 'id', 'range'); -- test foreign table creation CREATE FOREIGN TABLE table3_groupD ( id int ) SERVER fake_fdw_server; SELECT create_distributed_table('table3_groupD', 'id'); +NOTICE: foreign-data wrapper "fake_fdw" does not have an extension defined NOTICE: foreign-data wrapper "fake_fdw" does not have an extension defined create_distributed_table --------------------------------------------------------------------- @@ -1356,3 +1357,4 @@ DROP TABLE range_table; DROP TABLE none; DROP TABLE ref; DROP TABLE local_table; +DROP FOREIGN TABLE table3_groupD CASCADE; diff --git a/src/test/regress/expected/multi_distribution_metadata.out b/src/test/regress/expected/multi_distribution_metadata.out index 584af69d9..ba004974b 100644 --- a/src/test/regress/expected/multi_distribution_metadata.out +++ b/src/test/regress/expected/multi_distribution_metadata.out @@ -1,6 +1,8 @@ -- =================================================================== -- create test functions -- =================================================================== +CREATE SCHEMA metadata_test; +SET search_path TO metadata_test; ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 540000; CREATE FUNCTION load_shard_id_array(regclass) RETURNS bigint[] @@ -286,6 +288,7 @@ SELECT get_shard_id_for_distribution_column('get_shardid_test_table1', 3); -- verify result of the get_shard_id_for_distribution_column \c - - - :worker_1_port +SET search_path TO metadata_test; SELECT * FROM get_shardid_test_table1_540006; column1 | column2 --------------------------------------------------------------------- @@ -305,6 +308,7 @@ SELECT * FROM get_shardid_test_table1_540007; (1 row) \c - - - :master_port +SET search_path TO metadata_test; -- test non-existing value SELECT get_shard_id_for_distribution_column('get_shardid_test_table1', 4); get_shard_id_for_distribution_column @@ -336,6 +340,7 @@ SELECT get_shard_id_for_distribution_column('get_shardid_test_table2', '{d, e, f -- verify result of the get_shard_id_for_distribution_column \c - - - :worker_1_port +SET search_path TO metadata_test; SELECT * FROM get_shardid_test_table2_540013; column1 | column2 --------------------------------------------------------------------- @@ -349,6 +354,7 @@ SELECT * FROM get_shardid_test_table2_540011; (1 row) \c - - - :master_port +SET search_path TO metadata_test; -- test mismatching data type SELECT get_shard_id_for_distribution_column('get_shardid_test_table2', 'a'); ERROR: malformed array literal: "a" @@ -578,4 +584,5 @@ ORDER BY (1 row) -- clear unnecessary tables; -DROP TABLE get_shardid_test_table1, get_shardid_test_table2, get_shardid_test_table3, get_shardid_test_table4, get_shardid_test_table5, events_table_count; +SET client_min_messages TO ERROR; +DROP SCHEMA metadata_test CASCADE; diff --git a/src/test/regress/expected/multi_generate_ddl_commands.out b/src/test/regress/expected/multi_generate_ddl_commands.out index 525df0138..f2a47e823 100644 --- a/src/test/regress/expected/multi_generate_ddl_commands.out +++ b/src/test/regress/expected/multi_generate_ddl_commands.out @@ -172,6 +172,7 @@ CREATE FOREIGN TABLE foreign_table ( full_name text not null default '' ) SERVER fake_fdw_server OPTIONS (encoding 'utf-8', compression 'true'); SELECT create_distributed_table('foreign_table', 'id'); +NOTICE: foreign-data wrapper "fake_fdw" does not have an extension defined NOTICE: foreign-data wrapper "fake_fdw" does not have an extension defined create_distributed_table --------------------------------------------------------------------- diff --git a/src/test/regress/expected/multi_mx_truncate_from_worker.out b/src/test/regress/expected/multi_mx_truncate_from_worker.out index ab87821fb..7972e251a 100644 --- a/src/test/regress/expected/multi_mx_truncate_from_worker.out +++ b/src/test/regress/expected/multi_mx_truncate_from_worker.out @@ -21,6 +21,16 @@ SELECT create_distributed_table('on_update_fkey_table', 'id'); ALTER TABLE on_update_fkey_table ADD CONSTRAINT fkey FOREIGN KEY(value_1) REFERENCES "refer'ence_table"(id) ON UPDATE CASCADE; INSERT INTO "refer'ence_table" SELECT i FROM generate_series(0, 100) i; INSERT INTO on_update_fkey_table SELECT i, i % 100 FROM generate_series(0, 1000) i; +-- also have one replicated table +SET citus.shard_replication_factor TO 2; +CREATE TABLE replicated_table(id int PRIMARY KEY, value_1 int); +SELECT create_distributed_table('replicated_table', 'id'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO replicated_table SELECT i, i % 100 FROM generate_series(0, 1000) i; -- first, make sure that truncate from the coordinator workers as expected TRUNCATE on_update_fkey_table; SELECT count(*) FROM on_update_fkey_table; @@ -29,8 +39,24 @@ SELECT count(*) FROM on_update_fkey_table; 0 (1 row) --- fill the table again +TRUNCATE replicated_table; +SELECT count(*) FROM replicated_table; + count +--------------------------------------------------------------------- + 0 +(1 row) + +SET citus.task_assignment_policy TO "round-robin"; +SELECT count(*) FROM replicated_table; + count +--------------------------------------------------------------------- + 0 +(1 row) + +RESET citus.task_assignment_policy; +-- fill the tables again INSERT INTO on_update_fkey_table SELECT i, i % 100 FROM generate_series(0, 1000) i; +INSERT INTO replicated_table SELECT i, i % 100 FROM generate_series(0, 1000) i; -- now, show that TRUNCATE CASCADE works expected from the coordinator TRUNCATE "refer'ence_table" CASCADE; NOTICE: truncate cascades to table "on_update_fkey_table" @@ -59,6 +85,16 @@ BEGIN; 0 (1 row) +ROLLBACK; +BEGIN; + ALTER TABLE replicated_table ADD COLUMN x INT; + TRUNCATE replicated_table; + SELECT count(*) FROM replicated_table; + count +--------------------------------------------------------------------- + 0 +(1 row) + ROLLBACK; \c - - - :worker_1_port SET search_path TO 'truncate_from_workers'; @@ -70,8 +106,17 @@ SELECT count(*) FROM on_update_fkey_table; 0 (1 row) +-- make sure that TRUNCATE workes expected from the worker node +TRUNCATE replicated_table; +SELECT count(*) FROM replicated_table; + count +--------------------------------------------------------------------- + 0 +(1 row) + -- load some data INSERT INTO on_update_fkey_table SELECT i, i % 100 FROM generate_series(0, 1000) i; +INSERT INTO replicated_table SELECT i, i % 100 FROM generate_series(0, 1000) i; -- now, show that TRUNCATE CASCADE works expected from the worker TRUNCATE "refer'ence_table" CASCADE; NOTICE: truncate cascades to table "on_update_fkey_table" @@ -95,6 +140,10 @@ BEGIN; TRUNCATE on_update_fkey_table; ROLLBACK; -- test within transaction blocks +BEGIN; + TRUNCATE replicated_table; +ROLLBACK; +-- test within transaction blocks BEGIN; TRUNCATE "refer'ence_table" CASCADE; NOTICE: truncate cascades to table "on_update_fkey_table" @@ -112,6 +161,11 @@ NOTICE: truncate cascades to table "on_update_fkey_table_xxxxxxx" NOTICE: truncate cascades to table "on_update_fkey_table_xxxxxxx" NOTICE: truncate cascades to table "on_update_fkey_table_xxxxxxx" ROLLBACK; +-- test with sequential mode and CASCADE +BEGIN; + SET LOCAL citus.multi_shard_modify_mode TO sequential; + TRUNCATE replicated_table CASCADE; +ROLLBACK; -- fill some data for the next test \c - - - :master_port SET search_path TO 'truncate_from_workers'; @@ -136,6 +190,26 @@ BEGIN; 0 (1 row) +ROLLBACK; +-- make sure that DMLs-SELECTs works along with TRUNCATE worker fine +TRUNCATE replicated_table; +BEGIN; + -- we can enable local execution when truncate can be executed locally. + SET citus.enable_local_execution = 'off'; + INSERT INTO replicated_table SELECT i, i % 100 FROM generate_series(0, 1000) i; + SELECT count(*) FROM replicated_table; + count +--------------------------------------------------------------------- + 1001 +(1 row) + + TRUNCATE replicated_table; + SELECT count(*) FROM replicated_table; + count +--------------------------------------------------------------------- + 0 +(1 row) + ROLLBACK; RESET client_min_messages; \c - - - :master_port @@ -162,6 +236,16 @@ BEGIN; t (1 row) +ROLLBACK; +BEGIN; + -- should work since the schema is in the search path + SET search_path TO 'truncate_from_workers'; + SELECT lock_relation_if_exists('replicated_table', 'ACCESS SHARE'); + lock_relation_if_exists +--------------------------------------------------------------------- + t +(1 row) + ROLLBACK; BEGIN; -- should return false since there is no such table @@ -249,7 +333,8 @@ BEGIN; COMMIT; DROP SCHEMA truncate_from_workers CASCADE; -NOTICE: drop cascades to 2 other objects +NOTICE: drop cascades to 3 other objects DETAIL: drop cascades to table truncate_from_workers."refer'ence_table" drop cascades to table truncate_from_workers.on_update_fkey_table +drop cascades to table truncate_from_workers.replicated_table SET search_path TO public; diff --git a/src/test/regress/expected/multi_repair_shards.out b/src/test/regress/expected/multi_repair_shards.out index 710d5a6f1..0fd4c8b82 100644 --- a/src/test/regress/expected/multi_repair_shards.out +++ b/src/test/regress/expected/multi_repair_shards.out @@ -103,6 +103,7 @@ CREATE FOREIGN TABLE remote_engagements ( SET citus.shard_count TO 1; SET citus.shard_replication_factor TO 2; SELECT create_distributed_table('remote_engagements', 'id', 'hash'); +NOTICE: foreign-data wrapper "fake_fdw" does not have an extension defined NOTICE: foreign-data wrapper "fake_fdw" does not have an extension defined create_distributed_table --------------------------------------------------------------------- @@ -118,3 +119,5 @@ UPDATE pg_dist_placement SET shardstate = 3 WHERE shardid = :remotenewshardid AN SELECT master_copy_shard_placement(:remotenewshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port); ERROR: cannot repair shard DETAIL: Table remote_engagements is a foreign table. Repairing shards backed by foreign tables is not supported. +-- clean-up +DROP FOREIGN TABLE remote_engagements CASCADE; diff --git a/src/test/regress/expected/mx_coordinator_shouldhaveshards.out b/src/test/regress/expected/mx_coordinator_shouldhaveshards.out index ddacdbd80..ba6eb8dba 100644 --- a/src/test/regress/expected/mx_coordinator_shouldhaveshards.out +++ b/src/test/regress/expected/mx_coordinator_shouldhaveshards.out @@ -33,6 +33,23 @@ SELECT create_distributed_table('table_2', 'key', colocate_with := 'none'); INSERT INTO table_1 VALUES (1, '1'), (2, '2'), (3, '3'), (4, '4'); INSERT INTO table_2 VALUES (1, '1'), (2, '2'), (3, '3'), (4, '4'), (5, '5'), (6, '6'); +SET citus.shard_replication_factor to 2; +CREATE TABLE table_1_rep (key int, value text); +SELECT create_distributed_table('table_1_rep', 'key', colocate_with := 'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE table_2_rep (key int, value text); +SELECT create_distributed_table('table_2_rep', 'key', colocate_with := 'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO table_1_rep VALUES (1, '1'), (2, '2'), (3, '3'), (4, '4'); +INSERT INTO table_2_rep VALUES (1, '1'), (2, '2'), (3, '3'), (4, '4'), (5, '5'), (6, '6'); set citus.log_intermediate_results TO ON; set client_min_messages to debug1; WITH a AS (SELECT * FROM table_1 ORDER BY 1,2 DESC LIMIT 1) @@ -96,6 +113,67 @@ DEBUG: Collecting INSERT ... SELECT results on coordinator 0 (1 row) +WITH a AS (SELECT * FROM table_1_rep ORDER BY 1,2 DESC LIMIT 1) +SELECT count(*), +key +FROM a JOIN table_2_rep USING (key) +GROUP BY key +HAVING (max(table_2_rep.value) >= (SELECT value FROM a)); +DEBUG: generating subplan XXX_1 for CTE a: SELECT key, value FROM mx_coordinator_shouldhaveshards.table_1_rep ORDER BY key, value DESC LIMIT 1 +DEBUG: push down of limit count: 1 +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count, a.key FROM ((SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text)) a JOIN mx_coordinator_shouldhaveshards.table_2_rep USING (key)) GROUP BY a.key HAVING (max(table_2_rep.value) OPERATOR(pg_catalog.>=) (SELECT a_1.value FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text)) a_1)) +DEBUG: Subplan XXX_1 will be written to local file +DEBUG: Subplan XXX_1 will be sent to localhost:xxxxx +DEBUG: Subplan XXX_1 will be sent to localhost:xxxxx + count | key +--------------------------------------------------------------------- + 1 | 1 +(1 row) + +WITH a AS (SELECT * FROM table_1_rep ORDER BY 1,2 DESC LIMIT 1) +INSERT INTO table_1_rep SELECT count(*), +key +FROM a JOIN table_2_rep USING (key) +GROUP BY key +HAVING (max(table_2_rep.value) >= (SELECT value FROM a)); +DEBUG: Group by list without distribution column is not allowed in distributed INSERT ... SELECT queries +DEBUG: generating subplan XXX_1 for CTE a: SELECT key, value FROM mx_coordinator_shouldhaveshards.table_1_rep ORDER BY key, value DESC LIMIT 1 +DEBUG: push down of limit count: 1 +DEBUG: generating subplan XXX_2 for subquery SELECT count(*) AS count, a.key FROM ((SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text)) a JOIN mx_coordinator_shouldhaveshards.table_2_rep USING (key)) GROUP BY a.key HAVING (max(table_2_rep.value) OPERATOR(pg_catalog.>=) (SELECT a_1.value FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text)) a_1)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT int4(count) AS key, (key)::text AS value FROM (SELECT intermediate_result.count, intermediate_result.key FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(count bigint, key integer)) citus_insert_select_subquery +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Subplan XXX_1 will be written to local file +DEBUG: Subplan XXX_1 will be sent to localhost:xxxxx +DEBUG: Subplan XXX_1 will be sent to localhost:xxxxx +DEBUG: Subplan XXX_2 will be written to local file +WITH stats AS ( + SELECT count(key) m FROM table_1_rep +), +inserts AS ( + INSERT INTO table_2_rep + SELECT key, count(*) + FROM table_1_rep + WHERE key >= (SELECT m FROM stats) + GROUP BY key + HAVING count(*) <= (SELECT m FROM stats) + LIMIT 1 + RETURNING * +) SELECT count(*) FROM inserts; +DEBUG: generating subplan XXX_1 for CTE stats: SELECT count(key) AS m FROM mx_coordinator_shouldhaveshards.table_1_rep +DEBUG: generating subplan XXX_2 for CTE inserts: INSERT INTO mx_coordinator_shouldhaveshards.table_2_rep (key, value) SELECT key, count(*) AS count FROM mx_coordinator_shouldhaveshards.table_1_rep WHERE (key OPERATOR(pg_catalog.>=) (SELECT stats.m FROM (SELECT intermediate_result.m FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(m bigint)) stats)) GROUP BY key HAVING (count(*) OPERATOR(pg_catalog.<=) (SELECT stats.m FROM (SELECT intermediate_result.m FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(m bigint)) stats)) LIMIT 1 RETURNING table_2_rep.key, table_2_rep.value +DEBUG: LIMIT clauses are not allowed in distributed INSERT ... SELECT queries +DEBUG: push down of limit count: 1 +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text)) inserts +DEBUG: Subplan XXX_1 will be written to local file +DEBUG: Subplan XXX_1 will be sent to localhost:xxxxx +DEBUG: Subplan XXX_1 will be sent to localhost:xxxxx +DEBUG: Subplan XXX_2 will be written to local file +DEBUG: Collecting INSERT ... SELECT results on coordinator + count +--------------------------------------------------------------------- + 0 +(1 row) + \c - - - :worker_1_port SET search_path TO mx_coordinator_shouldhaveshards; set citus.log_intermediate_results TO ON; @@ -161,6 +239,67 @@ DEBUG: Collecting INSERT ... SELECT results on coordinator 0 (1 row) +WITH a AS (SELECT * FROM table_1_rep ORDER BY 1,2 DESC LIMIT 1) +SELECT count(*), +key +FROM a JOIN table_2_rep USING (key) +GROUP BY key +HAVING (max(table_2_rep.value) >= (SELECT value FROM a)); +DEBUG: generating subplan XXX_1 for CTE a: SELECT key, value FROM mx_coordinator_shouldhaveshards.table_1_rep ORDER BY key, value DESC LIMIT 1 +DEBUG: push down of limit count: 1 +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count, a.key FROM ((SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text)) a JOIN mx_coordinator_shouldhaveshards.table_2_rep USING (key)) GROUP BY a.key HAVING (max(table_2_rep.value) OPERATOR(pg_catalog.>=) (SELECT a_1.value FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text)) a_1)) +DEBUG: Subplan XXX_1 will be written to local file +DEBUG: Subplan XXX_1 will be sent to localhost:xxxxx +DEBUG: Subplan XXX_1 will be sent to localhost:xxxxx + count | key +--------------------------------------------------------------------- + 1 | 1 +(1 row) + +WITH a AS (SELECT * FROM table_1_rep ORDER BY 1,2 DESC LIMIT 1) +INSERT INTO table_1_rep SELECT count(*), +key +FROM a JOIN table_2_rep USING (key) +GROUP BY key +HAVING (max(table_2_rep.value) >= (SELECT value FROM a)); +DEBUG: Group by list without distribution column is not allowed in distributed INSERT ... SELECT queries +DEBUG: generating subplan XXX_1 for CTE a: SELECT key, value FROM mx_coordinator_shouldhaveshards.table_1_rep ORDER BY key, value DESC LIMIT 1 +DEBUG: push down of limit count: 1 +DEBUG: generating subplan XXX_2 for subquery SELECT count(*) AS count, a.key FROM ((SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text)) a JOIN mx_coordinator_shouldhaveshards.table_2_rep USING (key)) GROUP BY a.key HAVING (max(table_2_rep.value) OPERATOR(pg_catalog.>=) (SELECT a_1.value FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text)) a_1)) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT int4(count) AS key, (key)::text AS value FROM (SELECT intermediate_result.count, intermediate_result.key FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(count bigint, key integer)) citus_insert_select_subquery +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Subplan XXX_1 will be written to local file +DEBUG: Subplan XXX_1 will be sent to localhost:xxxxx +DEBUG: Subplan XXX_1 will be sent to localhost:xxxxx +DEBUG: Subplan XXX_2 will be written to local file +WITH stats AS ( + SELECT count(key) m FROM table_1_rep +), +inserts AS ( + INSERT INTO table_2_rep + SELECT key, count(*) + FROM table_1_rep + WHERE key >= (SELECT m FROM stats) + GROUP BY key + HAVING count(*) <= (SELECT m FROM stats) + LIMIT 1 + RETURNING * +) SELECT count(*) FROM inserts; +DEBUG: generating subplan XXX_1 for CTE stats: SELECT count(key) AS m FROM mx_coordinator_shouldhaveshards.table_1_rep +DEBUG: generating subplan XXX_2 for CTE inserts: INSERT INTO mx_coordinator_shouldhaveshards.table_2_rep (key, value) SELECT key, count(*) AS count FROM mx_coordinator_shouldhaveshards.table_1_rep WHERE (key OPERATOR(pg_catalog.>=) (SELECT stats.m FROM (SELECT intermediate_result.m FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(m bigint)) stats)) GROUP BY key HAVING (count(*) OPERATOR(pg_catalog.<=) (SELECT stats.m FROM (SELECT intermediate_result.m FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(m bigint)) stats)) LIMIT 1 RETURNING table_2_rep.key, table_2_rep.value +DEBUG: LIMIT clauses are not allowed in distributed INSERT ... SELECT queries +DEBUG: push down of limit count: 1 +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT count(*) AS count FROM (SELECT intermediate_result.key, intermediate_result.value FROM read_intermediate_result('XXX_2'::text, 'binary'::citus_copy_format) intermediate_result(key integer, value text)) inserts +DEBUG: Subplan XXX_1 will be written to local file +DEBUG: Subplan XXX_1 will be sent to localhost:xxxxx +DEBUG: Subplan XXX_1 will be sent to localhost:xxxxx +DEBUG: Subplan XXX_2 will be written to local file +DEBUG: Collecting INSERT ... SELECT results on coordinator + count +--------------------------------------------------------------------- + 0 +(1 row) + \c - - - :master_port SELECT 1 FROM master_set_node_property('localhost', :master_port, 'shouldhaveshards', false); ?column? diff --git a/src/test/regress/expected/start_stop_metadata_sync.out b/src/test/regress/expected/start_stop_metadata_sync.out index a71edacf3..c9f7def08 100644 --- a/src/test/regress/expected/start_stop_metadata_sync.out +++ b/src/test/regress/expected/start_stop_metadata_sync.out @@ -91,6 +91,42 @@ SELECT alter_table_set_access_method('events_2021_jan', 'columnar'); (1 row) VACUUM (FREEZE, ANALYZE) events_2021_jan; +-- add some replicated tables +SET citus.shard_replication_factor TO 2; +-- test for hybrid partitioned table (columnar+heap) +CREATE TABLE events_replicated(ts timestamptz, i int, n numeric, s text) + PARTITION BY RANGE (ts); +CREATE TABLE events_replicated_2021_jan PARTITION OF events_replicated + FOR VALUES FROM ('2021-01-01') TO ('2021-02-01'); +CREATE TABLE events_replicated_2021_feb PARTITION OF events_replicated + FOR VALUES FROM ('2021-02-01') TO ('2021-03-01'); +INSERT INTO events_replicated SELECT + '2021-01-01'::timestamptz + '0.45 seconds'::interval * g, + g, + g*pi(), + 'number: ' || g::text + FROM generate_series(1,1000) g; +VACUUM (FREEZE, ANALYZE) events_2021_feb; +SELECT create_distributed_table('events_replicated', 'ts'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT alter_table_set_access_method('events_replicated_2021_jan', 'columnar'); + alter_table_set_access_method +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE distributed_table_replicated_1(col int unique, b tt2); +SELECT create_distributed_table('distributed_table_replicated_1', 'col'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE INDEX indrep1 ON distributed_table_replicated_1(b); -- sync metadata SELECT start_metadata_sync_to_node('localhost', :worker_1_port); start_metadata_sync_to_node @@ -120,12 +156,15 @@ SELECT * FROM test_matview; (1 row) SELECT * FROM pg_dist_partition WHERE logicalrelid::text LIKE 'events%' ORDER BY logicalrelid::text; - logicalrelid | partmethod | partkey | colocationid | repmodel | autoconverted + logicalrelid | partmethod | partkey | colocationid | repmodel | autoconverted --------------------------------------------------------------------- - events | h | {VAR :varno 1 :varattno 1 :vartype 1184 :vartypmod -1 :varcollid 0 :varlevelsup 0 :varnoold 1 :varoattno 1 :location -1} | 1390012 | s | f - events_2021_feb | h | {VAR :varno 1 :varattno 1 :vartype 1184 :vartypmod -1 :varcollid 0 :varlevelsup 0 :varnoold 1 :varoattno 1 :location -1} | 1390012 | s | f - events_2021_jan | h | {VAR :varno 1 :varattno 1 :vartype 1184 :vartypmod -1 :varcollid 0 :varlevelsup 0 :varnoold 1 :varoattno 1 :location -1} | 1390012 | s | f -(3 rows) + events | h | {VAR :varno 1 :varattno 1 :vartype 1184 :vartypmod -1 :varcollid 0 :varlevelsup 0 :varnoold 1 :varoattno 1 :location -1} | 1390012 | s | f + events_2021_feb | h | {VAR :varno 1 :varattno 1 :vartype 1184 :vartypmod -1 :varcollid 0 :varlevelsup 0 :varnoold 1 :varoattno 1 :location -1} | 1390012 | s | f + events_2021_jan | h | {VAR :varno 1 :varattno 1 :vartype 1184 :vartypmod -1 :varcollid 0 :varlevelsup 0 :varnoold 1 :varoattno 1 :location -1} | 1390012 | s | f + events_replicated | h | {VAR :varno 1 :varattno 1 :vartype 1184 :vartypmod -1 :varcollid 0 :varlevelsup 0 :varnoold 1 :varoattno 1 :location -1} | 1390013 | c | f + events_replicated_2021_feb | h | {VAR :varno 1 :varattno 1 :vartype 1184 :vartypmod -1 :varcollid 0 :varlevelsup 0 :varnoold 1 :varoattno 1 :location -1} | 1390013 | c | f + events_replicated_2021_jan | h | {VAR :varno 1 :varattno 1 :vartype 1184 :vartypmod -1 :varcollid 0 :varlevelsup 0 :varnoold 1 :varoattno 1 :location -1} | 1390013 | c | f +(6 rows) SELECT count(*) > 0 FROM pg_dist_node; ?column? @@ -542,6 +581,24 @@ BEGIN; (1 row) +ROLLBACK; +-- this is safe because start_metadata_sync_to_node already switches to +-- sequential execution +BEGIN; + SELECT start_metadata_sync_to_node('localhost', :worker_1_port); + start_metadata_sync_to_node +--------------------------------------------------------------------- + +(1 row) + + SET LOCAL citus.shard_replication_factor TO 2; + CREATE TABLE test_table_rep(a int); + SELECT create_distributed_table('test_table_rep', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + ROLLBACK; -- multi-shard commands are allowed with start_metadata_sync -- as long as the start_metadata_sync_to_node executed @@ -584,6 +641,49 @@ BEGIN; (1 row) +ROLLBACK; +-- multi-shard commands are allowed with start_metadata_sync +-- as long as the start_metadata_sync_to_node executed +-- when it is OK to switch to sequential execution +BEGIN; + -- sync at the start of the tx + SELECT start_metadata_sync_to_node('localhost', :worker_1_port); + start_metadata_sync_to_node +--------------------------------------------------------------------- + +(1 row) + + SET citus.multi_shard_modify_mode TO sequential; + SET LOCAL citus.shard_replication_factor TO 2; + CREATE TABLE test_table(a int); + SELECT create_distributed_table('test_table', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + ALTER TABLE test_table ADD COLUMN B INT; + INSERT INTO test_table SELECT i,i From generate_series(0,100)i; + SELECT count(*) FROM test_table; + count +--------------------------------------------------------------------- + 101 +(1 row) + + ALTER TABLE distributed_table_3 ADD COLUMN new_col INT DEFAULT 15; + SELECT count(*) FROM distributed_table_3; + count +--------------------------------------------------------------------- + 1 +(1 row) + + -- sync at the end of the tx + SELECT start_metadata_sync_to_node('localhost', :worker_1_port); + start_metadata_sync_to_node +--------------------------------------------------------------------- + +(1 row) + ROLLBACK; -- cleanup \c - - - :master_port diff --git a/src/test/regress/input/multi_mx_copy_data.source b/src/test/regress/input/multi_mx_copy_data.source index 22ce69e6c..957eefa42 100644 --- a/src/test/regress/input/multi_mx_copy_data.source +++ b/src/test/regress/input/multi_mx_copy_data.source @@ -10,6 +10,11 @@ SET search_path TO citus_mx_test_schema; \COPY citus_mx_test_schema_join_1.nation_hash_2 FROM '@abs_srcdir@/data/nation.data' with delimiter '|'; \COPY citus_mx_test_schema_join_2.nation_hash FROM '@abs_srcdir@/data/nation.data' with delimiter '|'; +SET citus.shard_replication_factor TO 2; +CREATE TABLE citus_mx_test_schema.nation_hash_replicated AS SELECT * FROM citus_mx_test_schema.nation_hash; +SELECT create_distributed_table('citus_mx_test_schema.nation_hash_replicated', 'n_nationkey'); +\COPY nation_hash_replicated FROM '@abs_srcdir@/data/nation.data' with delimiter '|'; + -- now try loading data from worker node \c - - - :worker_1_port SET search_path TO public; @@ -17,10 +22,13 @@ SET search_path TO public; \COPY lineitem_mx FROM '@abs_srcdir@/data/lineitem.1.data' with delimiter '|' \COPY lineitem_mx FROM '@abs_srcdir@/data/lineitem.2.data' with delimiter '|' +\COPY citus_mx_test_schema.nation_hash_replicated FROM '@abs_srcdir@/data/nation.data' with delimiter '|'; + \c - - - :worker_2_port -- and use second worker as well \COPY orders_mx FROM '@abs_srcdir@/data/orders.1.data' with delimiter '|' \COPY orders_mx FROM '@abs_srcdir@/data/orders.2.data' with delimiter '|' +\COPY citus_mx_test_schema.nation_hash_replicated FROM '@abs_srcdir@/data/nation.data' with delimiter '|'; -- get ready for the next test TRUNCATE orders_mx; @@ -35,6 +43,8 @@ show citus.local_shared_pool_size; \COPY orders_mx FROM '@abs_srcdir@/data/orders.1.data' with delimiter '|' \COPY orders_mx FROM '@abs_srcdir@/data/orders.2.data' with delimiter '|' +\COPY citus_mx_test_schema.nation_hash_replicated FROM '@abs_srcdir@/data/nation.data' with delimiter '|'; + -- set it back ALTER SYSTEM RESET citus.local_shared_pool_size; SELECT pg_reload_conf(); diff --git a/src/test/regress/isolation_schedule b/src/test/regress/isolation_schedule index 2a6a793fc..0de778929 100644 --- a/src/test/regress/isolation_schedule +++ b/src/test/regress/isolation_schedule @@ -89,6 +89,7 @@ test: isolation_ref_select_for_update_vs_all_on_mx test: isolation_ref_update_delete_upsert_vs_all_on_mx test: isolation_dis2ref_foreign_keys_on_mx test: isolation_metadata_sync_deadlock +test: isolation_replicated_dist_on_mx # MXless tests test: isolation_turn_mx_off diff --git a/src/test/regress/multi_mx_schedule b/src/test/regress/multi_mx_schedule index 57db7bd76..7a9b0bc6b 100644 --- a/src/test/regress/multi_mx_schedule +++ b/src/test/regress/multi_mx_schedule @@ -45,7 +45,7 @@ test: coordinator_evaluation_modify test: coordinator_evaluation_select test: multi_mx_call test: multi_mx_function_call_delegation -test: multi_mx_modifications local_shard_execution +test: multi_mx_modifications local_shard_execution local_shard_execution_replicated test: multi_mx_repartition_udt_w1 multi_mx_repartition_udt_w2 test: local_shard_copy test: undistribute_table_cascade_mx diff --git a/src/test/regress/output/multi_mx_copy_data.source b/src/test/regress/output/multi_mx_copy_data.source index 53a36f7dc..369222084 100644 --- a/src/test/regress/output/multi_mx_copy_data.source +++ b/src/test/regress/output/multi_mx_copy_data.source @@ -7,15 +7,30 @@ SET search_path TO citus_mx_test_schema; \COPY citus_mx_test_schema_join_1.nation_hash FROM '@abs_srcdir@/data/nation.data' with delimiter '|'; \COPY citus_mx_test_schema_join_1.nation_hash_2 FROM '@abs_srcdir@/data/nation.data' with delimiter '|'; \COPY citus_mx_test_schema_join_2.nation_hash FROM '@abs_srcdir@/data/nation.data' with delimiter '|'; +SET citus.shard_replication_factor TO 2; +CREATE TABLE citus_mx_test_schema.nation_hash_replicated AS SELECT * FROM citus_mx_test_schema.nation_hash; +SELECT create_distributed_table('citus_mx_test_schema.nation_hash_replicated', 'n_nationkey'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$citus_mx_test_schema.nation_hash_replicated$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +\COPY nation_hash_replicated FROM '@abs_srcdir@/data/nation.data' with delimiter '|'; -- now try loading data from worker node \c - - - :worker_1_port SET search_path TO public; \COPY lineitem_mx FROM '@abs_srcdir@/data/lineitem.1.data' with delimiter '|' \COPY lineitem_mx FROM '@abs_srcdir@/data/lineitem.2.data' with delimiter '|' +\COPY citus_mx_test_schema.nation_hash_replicated FROM '@abs_srcdir@/data/nation.data' with delimiter '|'; \c - - - :worker_2_port -- and use second worker as well \COPY orders_mx FROM '@abs_srcdir@/data/orders.1.data' with delimiter '|' \COPY orders_mx FROM '@abs_srcdir@/data/orders.2.data' with delimiter '|' +\COPY citus_mx_test_schema.nation_hash_replicated FROM '@abs_srcdir@/data/nation.data' with delimiter '|'; -- get ready for the next test TRUNCATE orders_mx; \c - - - :worker_2_port @@ -23,74 +38,83 @@ SET citus.log_local_commands TO ON; -- simulate the case where there is no connection slots available ALTER SYSTEM SET citus.local_shared_pool_size TO -1; SELECT pg_reload_conf(); - pg_reload_conf ----------------- + pg_reload_conf +--------------------------------------------------------------------- t (1 row) SELECT pg_sleep(0.1); - pg_sleep ----------- - + pg_sleep +--------------------------------------------------------------------- + (1 row) show citus.local_shared_pool_size; - citus.local_shared_pool_size ------------------------------- + citus.local_shared_pool_size +--------------------------------------------------------------------- -1 (1 row) \COPY orders_mx FROM '@abs_srcdir@/data/orders.1.data' with delimiter '|' -NOTICE: executing the copy locally for shard 1220075 +NOTICE: executing the copy locally for shard xxxxx CONTEXT: COPY orders_mx, line 3: "3|1234|F|205654.30|1993-10-14|5-LOW|Clerk#000000955|0|sly final accounts boost. carefully regular id..." -NOTICE: executing the copy locally for shard 1220071 +NOTICE: executing the copy locally for shard xxxxx CONTEXT: COPY orders_mx, line 5: "5|445|F|105367.67|1994-07-30|5-LOW|Clerk#000000925|0|quickly. bold deposits sleep slyly. packages us..." -NOTICE: executing the copy locally for shard 1220069 +NOTICE: executing the copy locally for shard xxxxx CONTEXT: COPY orders_mx, line 9: "33|670|F|146567.24|1993-10-27|3-MEDIUM|Clerk#000000409|0|uriously. furiously final request" -NOTICE: executing the copy locally for shard 1220079 +NOTICE: executing the copy locally for shard xxxxx CONTEXT: COPY orders_mx, line 15: "39|818|O|326565.37|1996-09-20|3-MEDIUM|Clerk#000000659|0|ole express, ironic requests: ir" -NOTICE: executing the copy locally for shard 1220083 +NOTICE: executing the copy locally for shard xxxxx CONTEXT: COPY orders_mx, line 19: "67|568|O|182481.16|1996-12-19|4-NOT SPECIFIED|Clerk#000000547|0|symptotes haggle slyly around the fu..." -NOTICE: executing the copy locally for shard 1220073 +NOTICE: executing the copy locally for shard xxxxx CONTEXT: COPY orders_mx, line 24: "96|1078|F|64364.30|1994-04-17|2-HIGH|Clerk#000000395|0|oost furiously. pinto" -NOTICE: executing the copy locally for shard 1220077 +NOTICE: executing the copy locally for shard xxxxx CONTEXT: COPY orders_mx, line 25: "97|211|F|100572.55|1993-01-29|3-MEDIUM|Clerk#000000547|0|hang blithely along the regular accounts. f..." -NOTICE: executing the copy locally for shard 1220081 +NOTICE: executing the copy locally for shard xxxxx CONTEXT: COPY orders_mx, line 38: "134|62|F|208201.46|1992-05-01|4-NOT SPECIFIED|Clerk#000000711|0|lar theodolites boos" \COPY orders_mx FROM '@abs_srcdir@/data/orders.2.data' with delimiter '|' -NOTICE: executing the copy locally for shard 1220079 +NOTICE: executing the copy locally for shard xxxxx CONTEXT: COPY orders_mx, line 2: "8998|80|F|147264.16|1993-01-04|5-LOW|Clerk#000000733|0| fluffily pending sauternes cajo" -NOTICE: executing the copy locally for shard 1220077 +NOTICE: executing the copy locally for shard xxxxx CONTEXT: COPY orders_mx, line 4: "9024|1469|F|298241.36|1992-06-03|3-MEDIUM|Clerk#000000901|0|ar the theodolites. fluffily stealthy re..." -NOTICE: executing the copy locally for shard 1220073 +NOTICE: executing the copy locally for shard xxxxx CONTEXT: COPY orders_mx, line 6: "9026|677|O|63256.87|1996-07-24|5-LOW|Clerk#000000320|0|ironic escapades would wake carefully " -NOTICE: executing the copy locally for shard 1220071 +NOTICE: executing the copy locally for shard xxxxx CONTEXT: COPY orders_mx, line 9: "9029|1213|F|78703.86|1992-11-20|3-MEDIUM|Clerk#000000965|0| excuses nag quickly carefully unusual ex..." -NOTICE: executing the copy locally for shard 1220083 +NOTICE: executing the copy locally for shard xxxxx CONTEXT: COPY orders_mx, line 14: "9058|403|F|63464.13|1993-06-29|2-HIGH|Clerk#000000376|0|ealthily special deposits. quickly regular r..." -NOTICE: executing the copy locally for shard 1220081 +NOTICE: executing the copy locally for shard xxxxx CONTEXT: COPY orders_mx, line 16: "9060|463|O|45295.71|1996-06-09|1-URGENT|Clerk#000000438|0|iously. slyly regular dol" -NOTICE: executing the copy locally for shard 1220075 +NOTICE: executing the copy locally for shard xxxxx CONTEXT: COPY orders_mx, line 43: "9159|1135|O|99594.61|1995-07-26|1-URGENT|Clerk#000000892|0|xcuses. quickly ironic deposits wake alon..." -NOTICE: executing the copy locally for shard 1220069 +NOTICE: executing the copy locally for shard xxxxx CONTEXT: COPY orders_mx, line 69: "9281|904|F|173278.28|1992-02-24|1-URGENT|Clerk#000000530|0|eep furiously according to the requests; ..." +\COPY citus_mx_test_schema.nation_hash_replicated FROM '@abs_srcdir@/data/nation.data' with delimiter '|'; +NOTICE: executing the copy locally for shard xxxxx +CONTEXT: COPY nation_hash_replicated, line 1: "0|ALGERIA|0| haggle. carefully final deposits detect slyly agai" +NOTICE: executing the copy locally for shard xxxxx +CONTEXT: COPY nation_hash_replicated, line 2: "1|ARGENTINA|1|al foxes promise slyly according to the regular accounts. bold requests alon" +NOTICE: executing the copy locally for shard xxxxx +CONTEXT: COPY nation_hash_replicated, line 3: "2|BRAZIL|1|y alongside of the pending deposits. carefully special packages are about the ironic forg..." +NOTICE: executing the copy locally for shard xxxxx +CONTEXT: COPY nation_hash_replicated, line 7: "6|FRANCE|3|refully final requests. regular, ironi" -- set it back ALTER SYSTEM RESET citus.local_shared_pool_size; SELECT pg_reload_conf(); - pg_reload_conf ----------------- + pg_reload_conf +--------------------------------------------------------------------- t (1 row) SELECT pg_sleep(0.1); - pg_sleep ----------- - + pg_sleep +--------------------------------------------------------------------- + (1 row) show citus.local_shared_pool_size; - citus.local_shared_pool_size ------------------------------- + citus.local_shared_pool_size +--------------------------------------------------------------------- 50 (1 row) diff --git a/src/test/regress/spec/isolation_concurrent_dml.spec b/src/test/regress/spec/isolation_concurrent_dml.spec index 7dc66f9d1..a2220f3df 100644 --- a/src/test/regress/spec/isolation_concurrent_dml.spec +++ b/src/test/regress/spec/isolation_concurrent_dml.spec @@ -1,5 +1,7 @@ setup { + SELECT citus_internal.replace_isolation_tester_func(); + SELECT citus_internal.refresh_isolation_tester_prepared_statement(); CREATE TABLE test_concurrent_dml (test_id integer NOT NULL, data text); SELECT master_create_distributed_table('test_concurrent_dml', 'test_id', 'hash'); SELECT master_create_worker_shards('test_concurrent_dml', 4, 2); @@ -8,6 +10,7 @@ setup teardown { DROP TABLE IF EXISTS test_concurrent_dml CASCADE; + SELECT citus_internal.restore_isolation_tester_func(); } session "s1" diff --git a/src/test/regress/spec/isolation_dump_global_wait_edges.spec b/src/test/regress/spec/isolation_dump_global_wait_edges.spec index b88dc692c..2ae1ea00b 100644 --- a/src/test/regress/spec/isolation_dump_global_wait_edges.spec +++ b/src/test/regress/spec/isolation_dump_global_wait_edges.spec @@ -1,5 +1,7 @@ setup { + SELECT citus_internal.replace_isolation_tester_func(); + SELECT citus_internal.refresh_isolation_tester_prepared_statement(); CREATE TABLE distributed_table (x int primary key, y int); SELECT create_distributed_table('distributed_table', 'x'); INSERT INTO distributed_table VALUES (1,0); @@ -15,6 +17,7 @@ setup teardown { DROP TABLE distributed_table; + SELECT citus_internal.restore_isolation_tester_func(); } session "s1" diff --git a/src/test/regress/spec/isolation_hash_copy_vs_all.spec b/src/test/regress/spec/isolation_hash_copy_vs_all.spec index c0c9c7cae..d7174b0cc 100644 --- a/src/test/regress/spec/isolation_hash_copy_vs_all.spec +++ b/src/test/regress/spec/isolation_hash_copy_vs_all.spec @@ -5,6 +5,8 @@ // create append distributed table to test behavior of COPY in concurrent operations setup { + SELECT citus_internal.replace_isolation_tester_func(); + SELECT citus_internal.refresh_isolation_tester_prepared_statement(); SET citus.shard_replication_factor TO 1; CREATE TABLE hash_copy(id integer, data text, int_data int); SELECT create_distributed_table('hash_copy', 'id'); @@ -14,6 +16,7 @@ setup teardown { DROP TABLE IF EXISTS hash_copy CASCADE; + SELECT citus_internal.restore_isolation_tester_func(); } // session 1 diff --git a/src/test/regress/spec/isolation_insert_select_conflict.spec b/src/test/regress/spec/isolation_insert_select_conflict.spec index cf548ff2d..a25f01712 100644 --- a/src/test/regress/spec/isolation_insert_select_conflict.spec +++ b/src/test/regress/spec/isolation_insert_select_conflict.spec @@ -1,5 +1,7 @@ setup { + SELECT citus_internal.replace_isolation_tester_func(); + SELECT citus_internal.refresh_isolation_tester_prepared_statement(); CREATE TABLE target_table(col_1 int primary key, col_2 int); SELECT create_distributed_table('target_table','col_1'); INSERT INTO target_table VALUES(1,2),(2,3),(3,4),(4,5),(5,6); @@ -16,6 +18,7 @@ setup teardown { DROP TABLE target_table, target_table_2, source_table; + SELECT citus_internal.restore_isolation_tester_func(); } session "s1" diff --git a/src/test/regress/spec/isolation_modify_with_subquery_vs_dml.spec b/src/test/regress/spec/isolation_modify_with_subquery_vs_dml.spec index 903be0c1b..89835dd30 100644 --- a/src/test/regress/spec/isolation_modify_with_subquery_vs_dml.spec +++ b/src/test/regress/spec/isolation_modify_with_subquery_vs_dml.spec @@ -1,7 +1,8 @@ setup { + SELECT citus_internal.replace_isolation_tester_func(); + SELECT citus_internal.refresh_isolation_tester_prepared_statement(); SET citus.shard_replication_factor to 2; - CREATE TABLE users_test_table(user_id int, value_1 int, value_2 int, value_3 int); SELECT create_distributed_table('users_test_table', 'user_id'); INSERT INTO users_test_table VALUES @@ -30,6 +31,7 @@ teardown DROP TABLE users_test_table; DROP TABLE events_test_table; SET citus.shard_replication_factor to 1; + SELECT citus_internal.restore_isolation_tester_func(); } session "s1" diff --git a/src/test/regress/spec/isolation_replicated_dist_on_mx.spec b/src/test/regress/spec/isolation_replicated_dist_on_mx.spec new file mode 100644 index 000000000..b1e59d512 --- /dev/null +++ b/src/test/regress/spec/isolation_replicated_dist_on_mx.spec @@ -0,0 +1,220 @@ +#include "isolation_mx_common.include.spec" + +setup +{ + CREATE TABLE replicated_table(user_id int, value_1 int); + SET citus.shard_replication_factor TO 2; + SELECT create_distributed_table('replicated_table', 'user_id', shard_count:=4); + INSERT INTO replicated_table VALUES (1, 11), (2, 21), (3, 31), (4, 41), (5, 51), (6, 61), (7, 71); + + CREATE TABLE replicated_table_2(user_id int, value_1 int); + SET citus.shard_replication_factor TO 2; + SELECT create_distributed_table('replicated_table_2', 'user_id', shard_count:=4); + INSERT INTO replicated_table_2 VALUES (1, 11), (2, 21), (3, 31), (4, 41), (5, 51), (6, 61), (7, 71); + + CREATE TABLE single_replicated_table(user_id int, value_1 int); + SET citus.shard_replication_factor TO 1; + SELECT create_distributed_table('single_replicated_table', 'user_id', shard_count:=4); + INSERT INTO single_replicated_table VALUES (1, 11), (2, 21), (3, 31), (4, 41), (5, 51), (6, 61), (7, 71); + + +} + +// Create and use UDF to close the connection opened in the setup step. Also return the cluster +// back to the initial state. +teardown +{ + DROP TABLE replicated_table, replicated_table_2, single_replicated_table; + SELECT citus_internal.restore_isolation_tester_func(); +} + +session "s1" + +step "s1-begin" +{ + BEGIN; +} + +// We do not need to begin a transaction on coordinator, since it will be open on workers. + +step "s1-start-session-level-connection" +{ + SELECT start_session_level_connection_to_node('localhost', 57637); +} + +step "s1-begin-on-worker" +{ + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); +} + +step "s1-update-1-rep-table" +{ + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12 WHERE user_id = 1'); +} + +step "s1-update-all-rep-table" +{ + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12'); +} + +step "s1-delete-1-from-rep-table" +{ + SELECT run_commands_on_session_level_connection_to_node('DELETE FROM replicated_table WHERE user_id = 1'); +} + +step "s1-delete-all-from-rep-table" +{ + SELECT run_commands_on_session_level_connection_to_node('DELETE FROM replicated_table'); +} + +step "s1-insert-into-1-rep-table" +{ + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table VALUES(1,81)'); +} + +step "s1-insert-into-all-rep-table" +{ + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table VALUES(8,81),(9,91),(10,91),(11,91),(12,91), (13,91), (14,91), (15,91), (16,91), (17,91), (18,91), (19,91), (20,91)'); +} + +step "s1-insert-into-select" +{ + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table SELECT * FROM replicated_table_2'); +} + +step "s1-insert-into-select-from-single-rep" +{ + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table SELECT * FROM single_replicated_table LIMIT 10'); +} + +step "s1-copy-all-to-rep-table" +{ + SELECT run_commands_on_session_level_connection_to_node('COPY replicated_table FROM PROGRAM ''echo 10, 101 && echo 11, 111 && echo 11, 111 && echo 12, 111 && echo 13, 111 && echo 14, 111 && echo 15, 111 && echo 16, 111 && echo 17, 111 && echo 18, 111 && echo 19, 111 && echo 20, 111 && echo 21, 111 && echo 22, 111 && echo 23, 111'' WITH CSV'); +} + +step "s1-copy-1-to-rep-table" +{ + SELECT run_commands_on_session_level_connection_to_node('COPY replicated_table FROM PROGRAM ''echo 1, 101 && echo 1, 111 && echo 1,1111'' WITH CSV'); +} + +step "s1-commit-worker" +{ + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); +} + +step "s1-alter-table" +{ + ALTER TABLE replicated_table ADD COLUMN x INT; +} + +step "s1-stop-connection" +{ + SELECT stop_session_level_connection_to_node(); +} + +step "s1-commit" +{ + COMMIT; +} + +session "s2" + +step "s2-start-session-level-connection" +{ + SELECT start_session_level_connection_to_node('localhost', 57638); +} + +step "s2-begin-on-worker" +{ + SELECT run_commands_on_session_level_connection_to_node('BEGIN'); +} + +step "s2-update-1-rep-table" +{ + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12 WHERE user_id = 1'); +} + +step "s2-update-all-single-rep-table" +{ + SELECT run_commands_on_session_level_connection_to_node('UPDATE single_replicated_table SET value_1 = 12'); +} + +step "s2-update-all-rep-table" +{ + SELECT run_commands_on_session_level_connection_to_node('UPDATE replicated_table SET value_1 = 12'); +} + +step "s2-select-from-rep-table" +{ + SELECT run_commands_on_session_level_connection_to_node('SELECT count(*) FROM replicated_table'); +} + +step "s2-insert-into-1-rep-table" +{ + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table VALUES(1,81)'); +} + +step "s2-insert-into-all-rep-table" +{ + SELECT run_commands_on_session_level_connection_to_node('INSERT INTO replicated_table VALUES(8,81),(9,91),(10,91),(11,91),(12,91), (13,91), (14,91), (15,91), (16,91), (17,91), (18,91), (19,91), (20,91)'); +} + +step "s2-copy-all-to-rep-table" +{ + SELECT run_commands_on_session_level_connection_to_node('COPY replicated_table FROM PROGRAM ''echo 10, 101 && echo 11, 111 && echo 11, 111 && echo 12, 111 && echo 13, 111 && echo 14, 111 && echo 15, 111 && echo 16, 111 && echo 17, 111 && echo 18, 111 && echo 19, 111 && echo 20, 111 && echo 21, 111 && echo 22, 111 && echo 23, 111'' WITH CSV'); +} + +step "s2-stop-connection" +{ + SELECT stop_session_level_connection_to_node(); +} + +step "s2-commit-worker" +{ + SELECT run_commands_on_session_level_connection_to_node('COMMIT'); +} + +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-update-1-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-1-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-update-1-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-all-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-update-all-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-all-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-update-all-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-1-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" + +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-delete-1-from-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-1-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-delete-all-from-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-1-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-delete-1-from-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-all-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-delete-all-from-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-1-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" + +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-insert-into-1-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-1-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-insert-into-1-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-all-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-insert-into-all-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-1-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-insert-into-all-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-all-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" + +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-update-1-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-insert-into-1-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-update-1-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-insert-into-all-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-update-all-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-insert-into-1-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-update-all-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-insert-into-all-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" + +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-insert-into-1-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-insert-into-1-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-insert-into-1-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-insert-into-all-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-insert-into-all-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-insert-into-1-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-insert-into-all-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-insert-into-all-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" + +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-copy-1-to-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-1-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-copy-1-to-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-all-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-copy-all-to-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-1-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-copy-all-to-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-all-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" + +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-copy-all-to-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-insert-into-all-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-copy-1-to-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-insert-into-1-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" + +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-copy-all-to-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-copy-all-to-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" + +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-copy-all-to-rep-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-select-from-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" + +permutation "s2-start-session-level-connection" "s2-begin-on-worker" "s2-insert-into-1-rep-table" "s1-begin" "s1-alter-table" "s2-commit-worker" "s1-commit" "s2-stop-connection" +permutation "s2-start-session-level-connection" "s2-begin-on-worker" "s2-select-from-rep-table" "s1-begin" "s1-alter-table" "s2-commit-worker" "s1-commit" "s2-stop-connection" + +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-insert-into-select" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-insert-into-all-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-insert-into-select" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-insert-into-1-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" + +permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-insert-into-select-from-single-rep" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-all-single-rep-table" "s2-update-all-rep-table" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" diff --git a/src/test/regress/spec/isolation_undistribute_table.spec b/src/test/regress/spec/isolation_undistribute_table.spec index cd08364a2..c3ae23ed5 100644 --- a/src/test/regress/spec/isolation_undistribute_table.spec +++ b/src/test/regress/spec/isolation_undistribute_table.spec @@ -1,5 +1,8 @@ setup { + SELECT citus_internal.replace_isolation_tester_func(); + SELECT citus_internal.refresh_isolation_tester_prepared_statement(); + CREATE TABLE dist_table(a INT, b INT); SELECT create_distributed_table('dist_table', 'a'); INSERT INTO dist_table VALUES (1, 2), (3, 4), (5, 6); @@ -8,6 +11,7 @@ setup teardown { DROP TABLE IF EXISTS dist_table; + SELECT citus_internal.restore_isolation_tester_func(); } session "s1" diff --git a/src/test/regress/sql/local_shard_execution_replicated.sql b/src/test/regress/sql/local_shard_execution_replicated.sql new file mode 100644 index 000000000..0d940a7c9 --- /dev/null +++ b/src/test/regress/sql/local_shard_execution_replicated.sql @@ -0,0 +1,1090 @@ +CREATE SCHEMA local_shard_execution_replicated; +SET search_path TO local_shard_execution_replicated; + +SET citus.shard_count TO 4; +SET citus.shard_replication_factor TO 2; +SET citus.next_shard_id TO 1500000; + +CREATE TABLE reference_table (key int PRIMARY KEY); +SELECT create_reference_table('reference_table'); + +CREATE TABLE distributed_table (key int PRIMARY KEY , value text, age bigint CHECK (age > 10)); +SELECT create_distributed_table('distributed_table','key'); + +CREATE TABLE second_distributed_table (key int PRIMARY KEY , value text); +SELECT create_distributed_table('second_distributed_table','key'); + +-- ingest some data to enable some tests with data +INSERT INTO distributed_table VALUES (1, '1', 20); + +-- This GUC prevents to acquire the remote lock for replicated +-- tables +BEGIN; + SET LOCAL citus.allow_modifications_from_workers_to_replicated_tables TO false; + + INSERT INTO second_distributed_table VALUES (1, '1'); + INSERT INTO reference_table VALUES (1); +COMMIT; + +-- a simple test for +CREATE TABLE collections_list ( + key bigserial, + ser bigserial, + ts timestamptz, + collection_id integer, + value numeric, + PRIMARY KEY(key, collection_id) +) PARTITION BY LIST (collection_id ); + +SELECT create_distributed_table('collections_list', 'key'); + +CREATE TABLE collections_list_0 + PARTITION OF collections_list (key, ser, ts, collection_id, value) + FOR VALUES IN ( 0 ); + +-- create a volatile function that returns the local node id +CREATE OR REPLACE FUNCTION get_local_node_id_volatile() +RETURNS INT AS $$ +DECLARE localGroupId int; +BEGIN + SELECT groupid INTO localGroupId FROM pg_dist_local_group; + RETURN localGroupId; +END; $$ language plpgsql VOLATILE; +SELECT create_distributed_function('get_local_node_id_volatile()'); + +-- test case for issue #3556 +CREATE TABLE accounts (id text PRIMARY KEY); +CREATE TABLE stats (account_id text PRIMARY KEY, spent int); + +SELECT create_distributed_table('accounts', 'id', colocate_with => 'none'); +SELECT create_distributed_table('stats', 'account_id', colocate_with => 'accounts'); + +INSERT INTO accounts (id) VALUES ('foo'); +INSERT INTO stats (account_id, spent) VALUES ('foo', 100); + +CREATE TABLE abcd(a int, b int, c int, d int); +SELECT create_distributed_table('abcd', 'b'); + +INSERT INTO abcd VALUES (1,2,3,4); +INSERT INTO abcd VALUES (2,3,4,5); +INSERT INTO abcd VALUES (3,4,5,6); + +ALTER TABLE abcd DROP COLUMN a; + +-- connection worker and get ready for the tests +\c - - - :worker_1_port +SET search_path TO local_shard_execution_replicated; + +-- test case for issue #3556 +SET citus.log_intermediate_results TO TRUE; +SET client_min_messages TO DEBUG1; + +SELECT * +FROM +( + WITH accounts_cte AS ( + SELECT id AS account_id + FROM accounts + ), + joined_stats_cte_1 AS ( + SELECT spent, account_id + FROM stats + INNER JOIN accounts_cte USING (account_id) + ), + joined_stats_cte_2 AS ( + SELECT spent, account_id + FROM joined_stats_cte_1 + INNER JOIN accounts_cte USING (account_id) + ) + SELECT SUM(spent) OVER (PARTITION BY coalesce(account_id, NULL)) + FROM accounts_cte + INNER JOIN joined_stats_cte_2 USING (account_id) +) inner_query; + +SET citus.log_intermediate_results TO DEFAULT; +SET client_min_messages TO DEFAULT; + +--- enable logging to see which tasks are executed locally +SET citus.log_local_commands TO ON; + +-- first, make sure that local execution works fine +-- with simple queries that are not in transcation blocks +SELECT count(*) FROM distributed_table WHERE key = 1; + +-- multiple tasks both of which are local should NOT use local execution +-- because local execution means executing the tasks locally, so the executor +-- favors parallel execution even if everyting is local to node +SELECT count(*) FROM distributed_table WHERE key IN (1,6); + +-- queries that hit any remote shards should NOT use local execution +SELECT count(*) FROM distributed_table WHERE key IN (1,11); +SELECT count(*) FROM distributed_table; + +-- modifications also follow the same rules +INSERT INTO reference_table VALUES (1) ON CONFLICT DO NOTHING; +INSERT INTO distributed_table VALUES (1, '1', 21) ON CONFLICT DO NOTHING; + +-- local query +DELETE FROM distributed_table WHERE key = 1 AND age = 21; + +-- hitting multiple shards, so should be a distributed execution +DELETE FROM distributed_table WHERE age = 21; + +-- although both shards are local, the executor choose the parallel execution +-- over the wire because as noted above local execution is sequential +DELETE FROM second_distributed_table WHERE key IN (1,6); + +-- similarly, any multi-shard query just follows distributed execution +DELETE FROM second_distributed_table; + +-- load some more data for the following tests +INSERT INTO second_distributed_table VALUES (1, '1'); + +-- INSERT .. SELECT hitting a single single (co-located) shard(s) should +-- be executed locally +INSERT INTO distributed_table +SELECT + distributed_table.* +FROM + distributed_table, second_distributed_table +WHERE + distributed_table.key = 1 and distributed_table.key=second_distributed_table.key +ON CONFLICT(key) DO UPDATE SET value = '22' +RETURNING *; + +-- INSERT .. SELECT hitting multi-shards should go thourgh distributed execution +INSERT INTO distributed_table +SELECT + distributed_table.* +FROM + distributed_table, second_distributed_table +WHERE + distributed_table.key != 1 and distributed_table.key=second_distributed_table.key +ON CONFLICT(key) DO UPDATE SET value = '22' +RETURNING *; + +-- INSERT..SELECT via coordinator consists of two steps, select + COPY +-- that's why it is disallowed to use local execution even if the SELECT +-- can be executed locally +INSERT INTO distributed_table SELECT * FROM distributed_table WHERE key = 1 OFFSET 0 ON CONFLICT DO NOTHING; +INSERT INTO distributed_table SELECT 1, '1',15 FROM distributed_table WHERE key = 2 LIMIT 1 ON CONFLICT DO NOTHING; + +-- sanity check: multi-shard INSERT..SELECT pushdown goes through distributed execution +INSERT INTO distributed_table SELECT * FROM distributed_table ON CONFLICT DO NOTHING; + +-- Ensure tuple data in explain analyze output is the same on all PG versions +SET citus.enable_binary_protocol = TRUE; + +-- EXPLAIN for local execution just works fine +-- though going through distributed execution +EXPLAIN (COSTS OFF) SELECT * FROM distributed_table WHERE key = 1 AND age = 20; + +EXPLAIN (ANALYZE, COSTS OFF, SUMMARY OFF, TIMING OFF) SELECT * FROM distributed_table WHERE key = 1 AND age = 20; + +EXPLAIN (ANALYZE ON, COSTS OFF, SUMMARY OFF, TIMING OFF) +WITH r AS ( SELECT GREATEST(random(), 2) z,* FROM distributed_table) +SELECT 1 FROM r WHERE z < 3; + +EXPLAIN (COSTS OFF) DELETE FROM distributed_table WHERE key = 1 AND age = 20; + +EXPLAIN (ANALYZE, COSTS OFF, SUMMARY OFF, TIMING OFF) DELETE FROM distributed_table WHERE key = 1 AND age = 20; +-- show that EXPLAIN ANALYZE deleted the row +SELECT * FROM distributed_table WHERE key = 1 AND age = 20 ORDER BY 1,2,3; +SELECT * FROM second_distributed_table WHERE key = 1 ORDER BY 1,2; +-- Put row back for other tests +INSERT INTO distributed_table VALUES (1, '22', 20); + +CREATE VIEW abcd_view AS SELECT * FROM abcd; + +SELECT * FROM abcd first join abcd second on first.b = second.b ORDER BY 1,2,3,4; + +BEGIN; +SELECT * FROM abcd first join abcd second on first.b = second.b ORDER BY 1,2,3,4; +END; + +BEGIN; +SELECT * FROM abcd_view first join abcd_view second on first.b = second.b ORDER BY 1,2,3,4; +END; + +BEGIN; +SELECT * FROM abcd first full join abcd second on first.b = second.b ORDER BY 1,2,3,4; +END; + +BEGIN; +SELECT * FROM abcd first join abcd second USING(b) ORDER BY 1,2,3,4; +END; + +BEGIN; +SELECT * FROM abcd first join abcd second USING(b) join abcd third on first.b=third.b ORDER BY 1,2,3,4; +END; + +-- copy always happens via distributed execution irrespective of the +-- shards that are accessed +COPY reference_table FROM STDIN; +6 +11 +\. + +COPY distributed_table FROM STDIN WITH CSV; +6,'6',25 +11,'11',121 +\. + +COPY second_distributed_table FROM STDIN WITH CSV; +6,'6' +\. + +-- the behaviour in transaction blocks is the following: + -- (a) Unless the first query is a local query, always use distributed execution. + -- (b) If the executor has used local execution, it has to use local execution + -- for the remaining of the transaction block. If that's not possible, the + -- executor has to error out + +-- rollback should be able to rollback local execution +BEGIN; + INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '29' RETURNING *; + SELECT * FROM distributed_table WHERE key = 1 ORDER BY 1,2,3; +ROLLBACK; + +-- make sure that the value is rollbacked +SELECT * FROM distributed_table WHERE key = 1 ORDER BY 1,2,3; + + +-- rollback should be able to rollback both the local and distributed executions +BEGIN; + INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '29' RETURNING *; + DELETE FROM distributed_table; + + SELECT count(*) FROM second_distributed_table; +ROLLBACK; + +-- make sure that everything is rollbacked +SELECT * FROM distributed_table WHERE key = 1 ORDER BY 1,2,3; +SELECT count(*) FROM second_distributed_table; +SELECT * FROM second_distributed_table; + +-- very simple examples, an SELECTs should see the modifications +-- that has done before +BEGIN; + -- INSERT is executed locally + INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '23' RETURNING *; + + -- since the INSERT is executed locally, the SELECT should also be + -- executed locally and see the changes + SELECT * FROM distributed_table WHERE key = 1 ORDER BY 1,2,3; + + -- multi-shard SELECTs are now forced to use local execution on + -- the shards that reside on this node + SELECT * FROM distributed_table WHERE value = '23' ORDER BY 1,2,3; + + -- similarly, multi-shard modifications should use local exection + -- on the shards that reside on this node + DELETE FROM distributed_table WHERE value = '23'; + + -- make sure that the value is deleted + SELECT * FROM distributed_table WHERE value = '23' ORDER BY 1,2,3; +COMMIT; + +-- make sure that we've committed everything +SELECT * FROM distributed_table WHERE key = 1 ORDER BY 1,2,3; + +-- if we start with a distributed execution, we should keep +-- using that and never switch back to local execution +BEGIN; + DELETE FROM distributed_table WHERE value = '11'; + + -- although this command could have been executed + -- locally, it is not going to be executed locally + SELECT * FROM distributed_table WHERE key = 1 ORDER BY 1,2,3; + + -- but we can still execute parallel queries, even if + -- they are utility commands + TRUNCATE distributed_table CASCADE; + + -- TRUNCATE didn't cascade into second_distributed_table + SELECT count(*) FROM second_distributed_table; +ROLLBACK; + +-- load some data +INSERT INTO reference_table SELECT i FROM generate_series(500, 600) i; + +-- show that complex tx blocks work fine +BEGIN; + INSERT INTO reference_table VALUES (701); + INSERT INTO distributed_table VALUES (701, '701', 701); + INSERT INTO second_distributed_table VALUES (701, '701'); + + DELETE FROM reference_table WHERE key = 701; + + SELECT count(*) FROM distributed_table WHERE key = 701; + SELECT count(*) FROM second_distributed_table WHERE key = 701; + + -- multi-shard commands should also see the changes + SELECT count(*) FROM distributed_table WHERE key > 700; + + -- we can still do multi-shard commands + DELETE FROM distributed_table; +ROLLBACK; + +-- multiple queries hitting different shards can be executed locally +BEGIN; + SELECT count(*) FROM distributed_table WHERE key = 1; + SELECT count(*) FROM distributed_table WHERE key = 6; + SELECT count(*) FROM distributed_table WHERE key = 500; +ROLLBACK; + +-- a local query followed by TRUNCATE command can be executed locally +BEGIN; + SELECT count(*) FROM distributed_table WHERE key = 1; + TRUNCATE distributed_table CASCADE; +ROLLBACK; + +-- a local query is followed by an INSERT..SELECT via the coordinator +BEGIN; + SELECT count(*) FROM distributed_table WHERE key = 1; + + INSERT INTO distributed_table (key) SELECT i FROM generate_series(1,1) i; +ROLLBACK; + +BEGIN; +SET citus.enable_repartition_joins TO ON; +SELECT count(*) FROM distributed_table; +SELECT count(*) FROM distributed_table d1 join distributed_table d2 using(age); +ROLLBACK; + +-- a local query is followed by an INSERT..SELECT with re-partitioning +BEGIN; + SELECT count(*) FROM distributed_table WHERE key = 6; + INSERT INTO reference_table (key) SELECT -key FROM distributed_table; + INSERT INTO distributed_table (key) SELECT -key FROM distributed_table; + SELECT count(*) FROM distributed_table WHERE key = -6; +ROLLBACK; + +INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '29' RETURNING *; + +BEGIN; + DELETE FROM distributed_table WHERE key = 1; + EXPLAIN ANALYZE DELETE FROM distributed_table WHERE key = 1; +ROLLBACK; + +BEGIN; + INSERT INTO distributed_table VALUES (11, '111',29) ON CONFLICT(key) DO UPDATE SET value = '29' RETURNING *; + + -- this is already disallowed on the nodes, adding it in case we + -- support DDLs from the worker nodes in the future + ALTER TABLE distributed_table ADD COLUMN x INT; +ROLLBACK; + +BEGIN; + INSERT INTO distributed_table VALUES (11, '111',29) ON CONFLICT(key) DO UPDATE SET value = '29' RETURNING *; + + -- this is already disallowed because VACUUM cannot be executed in tx block + -- adding in case this is supported some day + VACUUM second_distributed_table; +ROLLBACK; + +-- make sure that functions can use local execution +CREATE OR REPLACE PROCEDURE only_local_execution() AS $$ + DECLARE cnt INT; + BEGIN + INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '29'; + SELECT count(*) INTO cnt FROM distributed_table WHERE key = 1; + DELETE FROM distributed_table WHERE key = 1; + END; +$$ LANGUAGE plpgsql; + +CALL only_local_execution(); + +-- insert a row that we need in the next tests +INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '29'; + +-- make sure that functions can use local execution +CREATE OR REPLACE PROCEDURE only_local_execution_with_function_evaluation() AS $$ + DECLARE nodeId INT; + BEGIN + -- fast path router + SELECT get_local_node_id_volatile() INTO nodeId FROM distributed_table WHERE key = 1; + IF nodeId <= 0 THEN + RAISE NOTICE 'unexpected node id'; + END IF; + + -- regular router + SELECT get_local_node_id_volatile() INTO nodeId FROM distributed_table d1 JOIN distributed_table d2 USING (key) WHERE d1.key = 1; + IF nodeId <= 0 THEN + RAISE NOTICE 'unexpected node id'; + END IF; + END; +$$ LANGUAGE plpgsql; + +CALL only_local_execution_with_function_evaluation(); + +CREATE OR REPLACE PROCEDURE only_local_execution_with_params(int) AS $$ + DECLARE cnt INT; + BEGIN + INSERT INTO distributed_table VALUES ($1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '29'; + SELECT count(*) INTO cnt FROM distributed_table WHERE key = $1; + DELETE FROM distributed_table WHERE key = $1; + END; +$$ LANGUAGE plpgsql; + +CALL only_local_execution_with_params(1); + +CREATE OR REPLACE PROCEDURE only_local_execution_with_function_evaluation_param(int) AS $$ + DECLARE nodeId INT; + BEGIN + -- fast path router + SELECT get_local_node_id_volatile() INTO nodeId FROM distributed_table WHERE key = $1; + IF nodeId <= 0 THEN + RAISE NOTICE 'unexpected node id'; + END IF; + + -- regular router + SELECT get_local_node_id_volatile() INTO nodeId FROM distributed_table d1 JOIN distributed_table d2 USING (key) WHERE d1.key = $1; + IF nodeId <= 0 THEN + RAISE NOTICE 'unexpected node id'; + END IF; + END; +$$ LANGUAGE plpgsql; + +CALL only_local_execution_with_function_evaluation_param(1); + +CREATE OR REPLACE PROCEDURE local_execution_followed_by_dist() AS $$ + DECLARE cnt INT; + BEGIN + INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '29'; + SELECT count(*) INTO cnt FROM distributed_table WHERE key = 1; + DELETE FROM distributed_table; + SELECT count(*) INTO cnt FROM distributed_table; + END; +$$ LANGUAGE plpgsql; + +CALL local_execution_followed_by_dist(); + +-- test CTEs, including modifying CTEs +WITH local_insert AS (INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '29' RETURNING *), +distributed_local_mixed AS (SELECT * FROM reference_table WHERE key IN (SELECT key FROM local_insert)) +SELECT * FROM local_insert, distributed_local_mixed; + +-- since we start with parallel execution, we do not switch back to local execution in the +-- latter CTEs +WITH distributed_local_mixed AS (SELECT * FROM distributed_table), +local_insert AS (INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '29' RETURNING *) +SELECT * FROM local_insert, distributed_local_mixed ORDER BY 1,2,3,4,5; + +-- router CTE pushdown +WITH all_data AS (SELECT * FROM distributed_table WHERE key = 1) +SELECT + count(*) +FROM + distributed_table, all_data +WHERE + distributed_table.key = all_data.key AND distributed_table.key = 1; + +INSERT INTO reference_table VALUES (2); +INSERT INTO distributed_table VALUES (2, '29', 29); +INSERT INTO second_distributed_table VALUES (2, '29'); + +-- single shard that is not a local query followed by a local query +WITH all_data AS (SELECT * FROM second_distributed_table WHERE key = 2) +SELECT + distributed_table.key +FROM + distributed_table, all_data +WHERE + distributed_table.value = all_data.value AND distributed_table.key = 1 +ORDER BY + 1 DESC; + +-- multi-shard CTE is followed by a query which could be executed locally, but +-- since the query started with a parallel query, it doesn't use local execution +-- note that if we allow Postgres to inline the CTE (e.g., not have the EXISTS +-- subquery), then it'd pushdown the filters and the query becomes single-shard, +-- locally executable query +WITH all_data AS (SELECT * FROM distributed_table) +SELECT + count(*) +FROM + distributed_table, all_data +WHERE + distributed_table.key = all_data.key AND distributed_table.key = 1 + AND EXISTS (SELECT * FROM all_data); + +-- in pg12, the following CTE can be inlined, still the query becomes +-- a subquery that needs to be recursively planned and a parallel +-- query, so do not use local execution +WITH all_data AS (SELECT age FROM distributed_table) +SELECT + count(*) +FROM + distributed_table, all_data +WHERE + distributed_table.key = all_data.age AND distributed_table.key = 1; + +-- get ready for the next commands +TRUNCATE reference_table, distributed_table, second_distributed_table; + +-- local execution of returning of reference tables +INSERT INTO reference_table VALUES (1),(2),(3),(4),(5),(6) RETURNING *; + +-- local execution of multi-row INSERTs +INSERT INTO distributed_table VALUES (1, '11',21), (5,'55',22) ON CONFLICT(key) DO UPDATE SET value = (EXCLUDED.value::int + 1)::text RETURNING *; + + +-- distributed execution of multi-rows INSERTs, where executor +-- is smart enough to execute local tasks via local execution +INSERT INTO distributed_table VALUES (1, '11',21), (2,'22',22), (3,'33',33), (4,'44',44),(5,'55',55) ON CONFLICT(key) DO UPDATE SET value = (EXCLUDED.value::int + 1)::text RETURNING *; + + +PREPARE local_prepare_no_param AS SELECT count(*) FROM distributed_table WHERE key = 1; +PREPARE local_prepare_no_param_subquery AS +SELECT DISTINCT trim(value) FROM ( + SELECT value FROM distributed_table + WHERE + key IN (1, 6, 500, 701) + AND (select 2) > random() + order by 1 + limit 2 + ) t; +PREPARE local_prepare_param (int) AS SELECT count(*) FROM distributed_table WHERE key = $1; +PREPARE remote_prepare_param (int) AS SELECT count(*) FROM distributed_table WHERE key != $1; +BEGIN; + -- 8 local execution without params + EXECUTE local_prepare_no_param; + EXECUTE local_prepare_no_param; + EXECUTE local_prepare_no_param; + EXECUTE local_prepare_no_param; + EXECUTE local_prepare_no_param; + EXECUTE local_prepare_no_param; + EXECUTE local_prepare_no_param; + EXECUTE local_prepare_no_param; + + -- 8 local execution without params and some subqueries + EXECUTE local_prepare_no_param_subquery; + EXECUTE local_prepare_no_param_subquery; + EXECUTE local_prepare_no_param_subquery; + EXECUTE local_prepare_no_param_subquery; + EXECUTE local_prepare_no_param_subquery; + EXECUTE local_prepare_no_param_subquery; + EXECUTE local_prepare_no_param_subquery; + EXECUTE local_prepare_no_param_subquery; + + -- 8 local executions with params + EXECUTE local_prepare_param(1); + EXECUTE local_prepare_param(5); + EXECUTE local_prepare_param(6); + EXECUTE local_prepare_param(1); + EXECUTE local_prepare_param(5); + EXECUTE local_prepare_param(6); + EXECUTE local_prepare_param(6); + EXECUTE local_prepare_param(6); + + -- followed by a non-local execution + EXECUTE remote_prepare_param(1); +COMMIT; + +PREPARE local_insert_prepare_no_param AS INSERT INTO distributed_table VALUES (1+0*random(), '11',21::int) ON CONFLICT(key) DO UPDATE SET value = '29' || '28' RETURNING *, key + 1, value || '30', age * 15; +PREPARE local_insert_prepare_param (int) AS INSERT INTO distributed_table VALUES ($1+0*random(), '11',21::int) ON CONFLICT(key) DO UPDATE SET value = '29' || '28' RETURNING *, key + 1, value || '30', age * 15; +BEGIN; + -- 8 local execution without params + EXECUTE local_insert_prepare_no_param; + EXECUTE local_insert_prepare_no_param; + EXECUTE local_insert_prepare_no_param; + EXECUTE local_insert_prepare_no_param; + EXECUTE local_insert_prepare_no_param; + EXECUTE local_insert_prepare_no_param; + EXECUTE local_insert_prepare_no_param; + EXECUTE local_insert_prepare_no_param; + + -- 8 local executions with params + EXECUTE local_insert_prepare_param(1); + EXECUTE local_insert_prepare_param(5); + EXECUTE local_insert_prepare_param(6); + EXECUTE local_insert_prepare_param(1); + EXECUTE local_insert_prepare_param(5); + EXECUTE local_insert_prepare_param(6); + EXECUTE local_insert_prepare_param(6); + EXECUTE local_insert_prepare_param(6); + + -- followed by a non-local execution + EXECUTE remote_prepare_param(2); +COMMIT; + +PREPARE local_multi_row_insert_prepare_no_param AS + INSERT INTO distributed_table VALUES (1,'55', 21), (5,'15',33) ON CONFLICT (key) WHERE key > 3 and key < 4 DO UPDATE SET value = '88' || EXCLUDED.value; + +PREPARE local_multi_row_insert_prepare_no_param_multi_shard AS + INSERT INTO distributed_table VALUES (6,'55', 21), (5,'15',33) ON CONFLICT (key) WHERE key > 3 AND key < 4 DO UPDATE SET value = '88' || EXCLUDED.value;; + +PREPARE local_multi_row_insert_prepare_params(int,int) AS + INSERT INTO distributed_table VALUES ($1,'55', 21), ($2,'15',33) ON CONFLICT (key) WHERE key > 3 and key < 4 DO UPDATE SET value = '88' || EXCLUDED.value;; +INSERT INTO reference_table VALUES (11); +BEGIN; + EXECUTE local_multi_row_insert_prepare_no_param; + EXECUTE local_multi_row_insert_prepare_no_param; + EXECUTE local_multi_row_insert_prepare_no_param; + EXECUTE local_multi_row_insert_prepare_no_param; + EXECUTE local_multi_row_insert_prepare_no_param; + EXECUTE local_multi_row_insert_prepare_no_param; + EXECUTE local_multi_row_insert_prepare_no_param; + EXECUTE local_multi_row_insert_prepare_no_param; + + EXECUTE local_multi_row_insert_prepare_no_param_multi_shard; + EXECUTE local_multi_row_insert_prepare_no_param_multi_shard; + EXECUTE local_multi_row_insert_prepare_no_param_multi_shard; + EXECUTE local_multi_row_insert_prepare_no_param_multi_shard; + EXECUTE local_multi_row_insert_prepare_no_param_multi_shard; + EXECUTE local_multi_row_insert_prepare_no_param_multi_shard; + EXECUTE local_multi_row_insert_prepare_no_param_multi_shard; + EXECUTE local_multi_row_insert_prepare_no_param_multi_shard; + + EXECUTE local_multi_row_insert_prepare_params(1,6); + EXECUTE local_multi_row_insert_prepare_params(1,5); + EXECUTE local_multi_row_insert_prepare_params(6,5); + EXECUTE local_multi_row_insert_prepare_params(5,1); + EXECUTE local_multi_row_insert_prepare_params(5,6); + EXECUTE local_multi_row_insert_prepare_params(5,1); + EXECUTE local_multi_row_insert_prepare_params(1,6); + EXECUTE local_multi_row_insert_prepare_params(1,5); + + -- one task is remote + EXECUTE local_multi_row_insert_prepare_params(5,11); +ROLLBACK; + + + +-- failures of local execution should rollback both the +-- local execution and remote executions + +-- fail on a local execution +BEGIN; + INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '100' RETURNING *; + + UPDATE distributed_table SET value = '200'; + + INSERT INTO distributed_table VALUES (1, '100',21) ON CONFLICT(key) DO UPDATE SET value = (1 / (100.0 - EXCLUDED.value::int))::text RETURNING *; +ROLLBACK; + +-- we've rollbacked everything +SELECT count(*) FROM distributed_table WHERE value = '200'; + +-- RETURNING should just work fine for reference tables +INSERT INTO reference_table VALUES (500) RETURNING *; +DELETE FROM reference_table WHERE key = 500 RETURNING *; + +-- should be able to skip local execution even if in a sequential mode of execution +BEGIN; + SET LOCAL citus.multi_shard_modify_mode TO sequential ; + + DELETE FROM distributed_table; + INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '100' RETURNING *; +ROLLBACK; + +-- sequential execution should just work fine after a local execution +BEGIN; + SET citus.multi_shard_modify_mode TO sequential ; + + INSERT INTO distributed_table VALUES (1, '11',21) ON CONFLICT(key) DO UPDATE SET value = '100' RETURNING *; + DELETE FROM distributed_table; +ROLLBACK; + + + +-- load some data so that foreign keys won't complain with the next tests +TRUNCATE reference_table CASCADE; +INSERT INTO reference_table SELECT i FROM generate_series(500, 600) i; +INSERT INTO distributed_table SELECT i, i::text, i % 10 + 25 FROM generate_series(500, 600) i; + +-- show that both local, and mixed local-distributed executions +-- calculate rows processed correctly +BEGIN; + DELETE FROM distributed_table WHERE key = 500; + + DELETE FROM distributed_table WHERE value != '123123213123213'; +ROLLBACK; + +BEGIN; + + DELETE FROM reference_table WHERE key = 500 RETURNING *; + + DELETE FROM reference_table; +ROLLBACK; + + +BEGIN; + DELETE FROM distributed_table WHERE key = 500; + + SELECT count(*) FROM distributed_table; +ROLLBACK; + +BEGIN; + SET LOCAL client_min_messages TO INFO; + SELECT count(*) FROM distributed_table; + SET LOCAL client_min_messages TO LOG; + + DELETE FROM distributed_table WHERE key = 500; +ROLLBACK; + +-- probably not a realistic case since views are not very +-- well supported with MX +CREATE VIEW v_local_query_execution AS +SELECT * FROM distributed_table WHERE key = 500; + +SELECT * FROM v_local_query_execution; + +-- similar test, but this time the view itself is a non-local +-- query, but the query on the view is local +CREATE VIEW v_local_query_execution_2 AS +SELECT * FROM distributed_table; + +SELECT * FROM v_local_query_execution_2 WHERE key = 500; + +-- even if we switch from remote execution -> local execution, +-- we are able to use remote execution after rollback +BEGIN; + SAVEPOINT my_savepoint; + + SELECT count(*) FROM distributed_table; + + DELETE FROM distributed_table WHERE key = 500; + + ROLLBACK TO SAVEPOINT my_savepoint; + + DELETE FROM distributed_table WHERE key = 500; + +COMMIT; + +-- even if we switch from local execution -> remote execution, +-- we are able to use local execution after rollback +BEGIN; + + SAVEPOINT my_savepoint; + + DELETE FROM distributed_table WHERE key = 500; + + SELECT count(*) FROM distributed_table; + + ROLLBACK TO SAVEPOINT my_savepoint; + + DELETE FROM distributed_table WHERE key = 500; + +COMMIT; + +-- sanity check: local execution on partitions +INSERT INTO collections_list (collection_id) VALUES (0) RETURNING *; + +BEGIN; + INSERT INTO collections_list (key, collection_id) VALUES (1,0); + SELECT count(*) FROM collections_list_0 WHERE key = 1; + SELECT count(*) FROM collections_list; + SELECT * FROM collections_list ORDER BY 1,2,3,4; +COMMIT; + + +TRUNCATE collections_list; + +-- make sure that even if local execution is used, the sequence values +-- are generated locally +ALTER SEQUENCE collections_list_key_seq NO MINVALUE NO MAXVALUE; + +PREPARE serial_prepared_local AS INSERT INTO collections_list (collection_id) VALUES (0) RETURNING key, ser; + +SELECT setval('collections_list_key_seq', 4); +EXECUTE serial_prepared_local; +SELECT setval('collections_list_key_seq', 5); +EXECUTE serial_prepared_local; +SELECT setval('collections_list_key_seq', 499); +EXECUTE serial_prepared_local; +SELECT setval('collections_list_key_seq', 700); +EXECUTE serial_prepared_local; +SELECT setval('collections_list_key_seq', 708); +EXECUTE serial_prepared_local; +SELECT setval('collections_list_key_seq', 709); +EXECUTE serial_prepared_local; + +-- get ready for the next executions +DELETE FROM collections_list WHERE key IN (5,6); +SELECT setval('collections_list_key_seq', 4); +EXECUTE serial_prepared_local; +SELECT setval('collections_list_key_seq', 5); +EXECUTE serial_prepared_local; + +-- and, one remote test +SELECT setval('collections_list_key_seq', 10); +EXECUTE serial_prepared_local; + +-- the final queries for the following CTEs are going to happen on the intermediate results only +-- one of them will be executed remotely, and the other is locally +-- Citus currently doesn't allow using task_assignment_policy for intermediate results +WITH distributed_local_mixed AS (INSERT INTO reference_table VALUES (1000) RETURNING *) SELECT * FROM distributed_local_mixed; + +-- clean the table for the next tests +SET search_path TO local_shard_execution_replicated; +TRUNCATE distributed_table CASCADE; + +-- load some data on a remote shard +INSERT INTO reference_table (key) VALUES (1), (2); +INSERT INTO distributed_table (key) VALUES (2); +BEGIN; + -- local execution followed by a distributed query + INSERT INTO distributed_table (key) VALUES (1); + DELETE FROM distributed_table RETURNING key; +COMMIT; + +-- a similar test with a reference table +TRUNCATE reference_table CASCADE; + +-- load some data on a remote shard +INSERT INTO reference_table (key) VALUES (2); +BEGIN; + -- local execution followed by a distributed query + INSERT INTO reference_table (key) VALUES (1); + DELETE FROM reference_table RETURNING key; +COMMIT; + +-- however complex the query, local execution can handle +SET client_min_messages TO LOG; +SET citus.log_local_commands TO ON; +WITH cte_1 AS + (SELECT * + FROM + (WITH cte_1 AS + (SELECT * + FROM distributed_table + WHERE key = 1) SELECT * + FROM cte_1) AS foo) +SELECT count(*) +FROM cte_1 +JOIN distributed_table USING (key) +WHERE distributed_table.key = 1 + AND distributed_table.key IN + (SELECT key + FROM distributed_table + WHERE key = 1); + +RESET client_min_messages; +RESET citus.log_local_commands; + +\c - - - :master_port +SET citus.next_shard_id TO 1501000; +-- test both local and remote execution with custom type +SET citus.shard_replication_factor TO 2; +SET search_path TO local_shard_execution_replicated; +CREATE TYPE invite_resp AS ENUM ('yes', 'no', 'maybe'); + +CREATE TABLE event_responses ( + event_id int, + user_id int, + response invite_resp, + primary key (event_id, user_id) +); + +SELECT create_distributed_table('event_responses', 'event_id'); + +INSERT INTO event_responses VALUES (1, 1, 'yes'), (2, 2, 'yes'), (3, 3, 'no'), (4, 4, 'no'); + +CREATE OR REPLACE FUNCTION regular_func(p invite_resp) +RETURNS int AS $$ +DECLARE + q1Result INT; + q2Result INT; + q3Result INT; +BEGIN +SELECT count(*) INTO q1Result FROM event_responses WHERE response = $1; +SELECT count(*) INTO q2Result FROM event_responses e1 LEFT JOIN event_responses e2 USING (event_id) WHERE e2.response = $1; +SELECT count(*) INTO q3Result FROM (SELECT * FROM event_responses WHERE response = $1 LIMIT 5) as foo; +RETURN q3Result+q2Result+q1Result; +END; +$$ LANGUAGE plpgsql; + +SELECT regular_func('yes'); +SELECT regular_func('yes'); +SELECT regular_func('yes'); +SELECT regular_func('yes'); +SELECT regular_func('yes'); +SELECT regular_func('yes'); +SELECT regular_func('yes'); +SELECT regular_func('yes'); + +CREATE OR REPLACE PROCEDURE regular_procedure(p invite_resp) +AS $$ +BEGIN +PERFORM * FROM event_responses WHERE response = $1 ORDER BY 1 DESC, 2 DESC, 3 DESC; +PERFORM * FROM event_responses e1 LEFT JOIN event_responses e2 USING (event_id) WHERE e2.response = $1 ORDER BY 1 DESC, 2 DESC, 3 DESC, 4 DESC; +PERFORM * FROM (SELECT * FROM event_responses WHERE response = $1 LIMIT 5) as foo ORDER BY 1 DESC, 2 DESC, 3 DESC; +END; +$$ LANGUAGE plpgsql; + +CALL regular_procedure('no'); +CALL regular_procedure('no'); +CALL regular_procedure('no'); +CALL regular_procedure('no'); +CALL regular_procedure('no'); +CALL regular_procedure('no'); +CALL regular_procedure('no'); +CALL regular_procedure('no'); + +PREPARE multi_shard_no_dist_key(invite_resp) AS select * from event_responses where response = $1::invite_resp ORDER BY 1 DESC, 2 DESC, 3 DESC LIMIT 1; +EXECUTE multi_shard_no_dist_key('yes'); +EXECUTE multi_shard_no_dist_key('yes'); +EXECUTE multi_shard_no_dist_key('yes'); +EXECUTE multi_shard_no_dist_key('yes'); +EXECUTE multi_shard_no_dist_key('yes'); +EXECUTE multi_shard_no_dist_key('yes'); +EXECUTE multi_shard_no_dist_key('yes'); +EXECUTE multi_shard_no_dist_key('yes'); + +PREPARE multi_shard_with_dist_key(int, invite_resp) AS select * from event_responses where event_id > $1 AND response = $2::invite_resp ORDER BY 1 DESC, 2 DESC, 3 DESC LIMIT 1; +EXECUTE multi_shard_with_dist_key(1, 'yes'); +EXECUTE multi_shard_with_dist_key(1, 'yes'); +EXECUTE multi_shard_with_dist_key(1, 'yes'); +EXECUTE multi_shard_with_dist_key(1, 'yes'); +EXECUTE multi_shard_with_dist_key(1, 'yes'); +EXECUTE multi_shard_with_dist_key(1, 'yes'); +EXECUTE multi_shard_with_dist_key(1, 'yes'); +EXECUTE multi_shard_with_dist_key(1, 'yes'); + +PREPARE query_pushdown_no_dist_key(invite_resp) AS select * from event_responses e1 LEFT JOIN event_responses e2 USING(event_id) where e1.response = $1::invite_resp ORDER BY 1 DESC, 2 DESC, 3 DESC, 4 DESC LIMIT 1; +EXECUTE query_pushdown_no_dist_key('yes'); +EXECUTE query_pushdown_no_dist_key('yes'); +EXECUTE query_pushdown_no_dist_key('yes'); +EXECUTE query_pushdown_no_dist_key('yes'); +EXECUTE query_pushdown_no_dist_key('yes'); +EXECUTE query_pushdown_no_dist_key('yes'); +EXECUTE query_pushdown_no_dist_key('yes'); +EXECUTE query_pushdown_no_dist_key('yes'); + +PREPARE insert_select_via_coord(invite_resp) AS INSERT INTO event_responses SELECT * FROM event_responses where response = $1::invite_resp LIMIT 1 ON CONFLICT (event_id, user_id) DO NOTHING ; +EXECUTE insert_select_via_coord('yes'); +EXECUTE insert_select_via_coord('yes'); +EXECUTE insert_select_via_coord('yes'); +EXECUTE insert_select_via_coord('yes'); +EXECUTE insert_select_via_coord('yes'); +EXECUTE insert_select_via_coord('yes'); +EXECUTE insert_select_via_coord('yes'); +EXECUTE insert_select_via_coord('yes'); + +PREPARE insert_select_pushdown(invite_resp) AS INSERT INTO event_responses SELECT * FROM event_responses where response = $1::invite_resp ON CONFLICT (event_id, user_id) DO NOTHING; +EXECUTE insert_select_pushdown('yes'); +EXECUTE insert_select_pushdown('yes'); +EXECUTE insert_select_pushdown('yes'); +EXECUTE insert_select_pushdown('yes'); +EXECUTE insert_select_pushdown('yes'); +EXECUTE insert_select_pushdown('yes'); +EXECUTE insert_select_pushdown('yes'); +EXECUTE insert_select_pushdown('yes'); + +PREPARE router_select_with_no_dist_key_filter(invite_resp) AS select * from event_responses where event_id = 1 AND response = $1::invite_resp ORDER BY 1 DESC, 2 DESC, 3 DESC LIMIT 1; +EXECUTE router_select_with_no_dist_key_filter('yes'); +EXECUTE router_select_with_no_dist_key_filter('yes'); +EXECUTE router_select_with_no_dist_key_filter('yes'); +EXECUTE router_select_with_no_dist_key_filter('yes'); +EXECUTE router_select_with_no_dist_key_filter('yes'); +EXECUTE router_select_with_no_dist_key_filter('yes'); +EXECUTE router_select_with_no_dist_key_filter('yes'); +EXECUTE router_select_with_no_dist_key_filter('yes'); + +-- rest of the tests assume the table is empty +TRUNCATE event_responses; + +CREATE OR REPLACE PROCEDURE register_for_event(p_event_id int, p_user_id int, p_choice invite_resp) +LANGUAGE plpgsql AS $fn$ +BEGIN + INSERT INTO local_shard_execution_replicated.event_responses VALUES (p_event_id, p_user_id, p_choice) + ON CONFLICT (event_id, user_id) + DO UPDATE SET response = EXCLUDED.response; + + PERFORM count(*) FROM local_shard_execution_replicated.event_responses WHERE event_id = p_event_id; + + PERFORM count(*) FROM local_shard_execution_replicated.event_responses WHERE event_id = p_event_id AND false; + + UPDATE local_shard_execution_replicated.event_responses SET response = p_choice WHERE event_id = p_event_id; + +END; +$fn$; + +SELECT create_distributed_function('register_for_event(int,int,invite_resp)'); + +-- call 8 times to make sure it works after the 5th time(postgres binds values after the 5th time and Citus 2nd time) +-- after 6th, the local execution caches the local plans and uses it +-- execute it both locally and remotely +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); + +\c - - - :worker_2_port +SET search_path TO local_shard_execution_replicated; + +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); +CALL register_for_event(16, 1, 'yes'); + +-- values 16, 17 and 19 hits the same +-- shard, so we're re-using the same cached +-- plans per statement across different distribution +-- key values +CALL register_for_event(17, 1, 'yes'); +CALL register_for_event(19, 1, 'yes'); +CALL register_for_event(17, 1, 'yes'); +CALL register_for_event(19, 1, 'yes'); + +-- should work fine if the logs are enabled +\set VERBOSITY terse +SET citus.log_local_commands TO ON; +SET client_min_messages TO DEBUG2; +CALL register_for_event(19, 1, 'yes'); + +-- should be fine even if no parameters exists in the query +SELECT count(*) FROM event_responses WHERE event_id = 16; +SELECT count(*) FROM event_responses WHERE event_id = 16; +UPDATE event_responses SET response = 'no' WHERE event_id = 16; +INSERT INTO event_responses VALUES (16, 666, 'maybe') +ON CONFLICT (event_id, user_id) +DO UPDATE SET response = EXCLUDED.response RETURNING *; + +-- multi row INSERTs hitting the same shard +INSERT INTO event_responses VALUES (16, 666, 'maybe'), (17, 777, 'no') +ON CONFLICT (event_id, user_id) +DO UPDATE SET response = EXCLUDED.response RETURNING *; + +-- now, similar tests with some settings changed +SET citus.enable_local_execution TO false; +SET citus.enable_fast_path_router_planner TO false; +CALL register_for_event(19, 1, 'yes'); + +-- should be fine even if no parameters exists in the query +SELECT count(*) FROM event_responses WHERE event_id = 16; +SELECT count(*) FROM event_responses WHERE event_id = 16; +UPDATE event_responses SET response = 'no' WHERE event_id = 16; +INSERT INTO event_responses VALUES (16, 666, 'maybe') +ON CONFLICT (event_id, user_id) +DO UPDATE SET response = EXCLUDED.response RETURNING *; + +-- multi row INSERTs hitting the same shard +INSERT INTO event_responses VALUES (16, 666, 'maybe'), (17, 777, 'no') +ON CONFLICT (event_id, user_id) +DO UPDATE SET response = EXCLUDED.response RETURNING *; + +-- not allow commands over the workers when user disables +SET citus.allow_modifications_from_workers_to_replicated_tables TO false; +INSERT INTO event_responses VALUES (16, 666, 'maybe'), (17, 777, 'no') +ON CONFLICT (event_id, user_id) +DO UPDATE SET response = EXCLUDED.response RETURNING *; + +\c - - - :master_port + +SET client_min_messages TO ERROR; +SET search_path TO public; +DROP SCHEMA local_shard_execution_replicated CASCADE; + diff --git a/src/test/regress/sql/master_copy_shard_placement.sql b/src/test/regress/sql/master_copy_shard_placement.sql index 87ba7fd0c..c5dd7beeb 100644 --- a/src/test/regress/sql/master_copy_shard_placement.sql +++ b/src/test/regress/sql/master_copy_shard_placement.sql @@ -89,15 +89,10 @@ WHERE shardid = get_shard_id_for_distribution_column('history', 'key-1') AND nod SELECT count(*) FROM data; SELECT count(*) FROM history; --- test we can not replicate MX tables +-- test we can replicate MX tables SET citus.shard_replication_factor TO 1; --- metadata sync will fail as we have a statement replicated table -SELECT start_metadata_sync_to_node('localhost', :worker_1_port); - --- use streaming replication to enable metadata syncing -UPDATE pg_dist_partition SET repmodel='s' WHERE logicalrelid IN - ('history'::regclass); +-- metadata sync will succeed even if we have rep > 1 tables SELECT start_metadata_sync_to_node('localhost', :worker_1_port); CREATE TABLE mx_table(a int); diff --git a/src/test/regress/sql/metadata_sync_helpers.sql b/src/test/regress/sql/metadata_sync_helpers.sql index e4f174b16..dba29cfe9 100644 --- a/src/test/regress/sql/metadata_sync_helpers.sql +++ b/src/test/regress/sql/metadata_sync_helpers.sql @@ -225,14 +225,6 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; SELECT citus_internal_add_partition_metadata ('test_ref'::regclass, 'n', NULL, 0, 'c'); ROLLBACK; --- not-matching replication model for hash table -BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; - SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02'); - SET application_name to 'citus'; - SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 0, 't'); -ROLLBACK; - - -- add entry for super user table \c - postgres - :worker_1_port SET search_path TO metadata_sync_helpers; diff --git a/src/test/regress/sql/multi_colocation_utils.sql b/src/test/regress/sql/multi_colocation_utils.sql index b7604804d..ae19d5bb4 100644 --- a/src/test/regress/sql/multi_colocation_utils.sql +++ b/src/test/regress/sql/multi_colocation_utils.sql @@ -553,5 +553,4 @@ DROP TABLE range_table; DROP TABLE none; DROP TABLE ref; DROP TABLE local_table; - - +DROP FOREIGN TABLE table3_groupD CASCADE; diff --git a/src/test/regress/sql/multi_distribution_metadata.sql b/src/test/regress/sql/multi_distribution_metadata.sql index c92de854a..0e6363743 100644 --- a/src/test/regress/sql/multi_distribution_metadata.sql +++ b/src/test/regress/sql/multi_distribution_metadata.sql @@ -1,7 +1,8 @@ -- =================================================================== -- create test functions -- =================================================================== - +CREATE SCHEMA metadata_test; +SET search_path TO metadata_test; ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 540000; @@ -196,10 +197,13 @@ SELECT get_shard_id_for_distribution_column('get_shardid_test_table1', 3); -- verify result of the get_shard_id_for_distribution_column \c - - - :worker_1_port +SET search_path TO metadata_test; + SELECT * FROM get_shardid_test_table1_540006; SELECT * FROM get_shardid_test_table1_540009; SELECT * FROM get_shardid_test_table1_540007; \c - - - :master_port +SET search_path TO metadata_test; -- test non-existing value SELECT get_shard_id_for_distribution_column('get_shardid_test_table1', 4); @@ -217,9 +221,12 @@ SELECT get_shard_id_for_distribution_column('get_shardid_test_table2', '{d, e, f -- verify result of the get_shard_id_for_distribution_column \c - - - :worker_1_port +SET search_path TO metadata_test; + SELECT * FROM get_shardid_test_table2_540013; SELECT * FROM get_shardid_test_table2_540011; \c - - - :master_port +SET search_path TO metadata_test; -- test mismatching data type SELECT get_shard_id_for_distribution_column('get_shardid_test_table2', 'a'); @@ -355,4 +362,5 @@ ORDER BY types;$$); -- clear unnecessary tables; -DROP TABLE get_shardid_test_table1, get_shardid_test_table2, get_shardid_test_table3, get_shardid_test_table4, get_shardid_test_table5, events_table_count; +SET client_min_messages TO ERROR; +DROP SCHEMA metadata_test CASCADE; diff --git a/src/test/regress/sql/multi_mx_truncate_from_worker.sql b/src/test/regress/sql/multi_mx_truncate_from_worker.sql index f76f238fe..380c4b9dd 100644 --- a/src/test/regress/sql/multi_mx_truncate_from_worker.sql +++ b/src/test/regress/sql/multi_mx_truncate_from_worker.sql @@ -18,12 +18,25 @@ ALTER TABLE on_update_fkey_table ADD CONSTRAINT fkey FOREIGN KEY(value_1) REFERE INSERT INTO "refer'ence_table" SELECT i FROM generate_series(0, 100) i; INSERT INTO on_update_fkey_table SELECT i, i % 100 FROM generate_series(0, 1000) i; +-- also have one replicated table +SET citus.shard_replication_factor TO 2; +CREATE TABLE replicated_table(id int PRIMARY KEY, value_1 int); +SELECT create_distributed_table('replicated_table', 'id'); +INSERT INTO replicated_table SELECT i, i % 100 FROM generate_series(0, 1000) i; + -- first, make sure that truncate from the coordinator workers as expected TRUNCATE on_update_fkey_table; SELECT count(*) FROM on_update_fkey_table; --- fill the table again +TRUNCATE replicated_table; +SELECT count(*) FROM replicated_table; +SET citus.task_assignment_policy TO "round-robin"; +SELECT count(*) FROM replicated_table; +RESET citus.task_assignment_policy; + +-- fill the tables again INSERT INTO on_update_fkey_table SELECT i, i % 100 FROM generate_series(0, 1000) i; +INSERT INTO replicated_table SELECT i, i % 100 FROM generate_series(0, 1000) i; -- now, show that TRUNCATE CASCADE works expected from the coordinator TRUNCATE "refer'ence_table" CASCADE; @@ -41,6 +54,12 @@ BEGIN; SELECT count(*) FROM on_update_fkey_table; ROLLBACK; +BEGIN; + ALTER TABLE replicated_table ADD COLUMN x INT; + TRUNCATE replicated_table; + SELECT count(*) FROM replicated_table; +ROLLBACK; + \c - - - :worker_1_port SET search_path TO 'truncate_from_workers'; @@ -49,8 +68,13 @@ SET search_path TO 'truncate_from_workers'; TRUNCATE on_update_fkey_table; SELECT count(*) FROM on_update_fkey_table; +-- make sure that TRUNCATE workes expected from the worker node +TRUNCATE replicated_table; +SELECT count(*) FROM replicated_table; + -- load some data INSERT INTO on_update_fkey_table SELECT i, i % 100 FROM generate_series(0, 1000) i; +INSERT INTO replicated_table SELECT i, i % 100 FROM generate_series(0, 1000) i; -- now, show that TRUNCATE CASCADE works expected from the worker TRUNCATE "refer'ence_table" CASCADE; @@ -62,6 +86,11 @@ BEGIN; TRUNCATE on_update_fkey_table; ROLLBACK; +-- test within transaction blocks +BEGIN; + TRUNCATE replicated_table; +ROLLBACK; + -- test within transaction blocks BEGIN; TRUNCATE "refer'ence_table" CASCADE; @@ -74,6 +103,13 @@ BEGIN; TRUNCATE "refer'ence_table" CASCADE; ROLLBACK; + +-- test with sequential mode and CASCADE +BEGIN; + SET LOCAL citus.multi_shard_modify_mode TO sequential; + TRUNCATE replicated_table CASCADE; +ROLLBACK; + -- fill some data for the next test \c - - - :master_port SET search_path TO 'truncate_from_workers'; @@ -92,6 +128,17 @@ BEGIN; SELECT count(*) FROM on_update_fkey_table; ROLLBACK; +-- make sure that DMLs-SELECTs works along with TRUNCATE worker fine +TRUNCATE replicated_table; +BEGIN; + -- we can enable local execution when truncate can be executed locally. + SET citus.enable_local_execution = 'off'; + INSERT INTO replicated_table SELECT i, i % 100 FROM generate_series(0, 1000) i; + SELECT count(*) FROM replicated_table; + TRUNCATE replicated_table; + SELECT count(*) FROM replicated_table; +ROLLBACK; + RESET client_min_messages; \c - - - :master_port @@ -113,6 +160,13 @@ BEGIN; SELECT lock_relation_if_exists('on_update_fkey_table', 'ACCESS SHARE'); ROLLBACK; + +BEGIN; + -- should work since the schema is in the search path + SET search_path TO 'truncate_from_workers'; + SELECT lock_relation_if_exists('replicated_table', 'ACCESS SHARE'); +ROLLBACK; + BEGIN; -- should return false since there is no such table SELECT lock_relation_if_exists('truncate_from_workers.on_update_fkey_tableXXX', 'ACCESS SHARE'); diff --git a/src/test/regress/sql/multi_repair_shards.sql b/src/test/regress/sql/multi_repair_shards.sql index a36767875..98085b3c9 100644 --- a/src/test/regress/sql/multi_repair_shards.sql +++ b/src/test/regress/sql/multi_repair_shards.sql @@ -103,3 +103,6 @@ UPDATE pg_dist_placement SET shardstate = 3 WHERE shardid = :remotenewshardid AN -- oops! we don't support repairing shards backed by foreign tables SELECT master_copy_shard_placement(:remotenewshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port); + +-- clean-up +DROP FOREIGN TABLE remote_engagements CASCADE; diff --git a/src/test/regress/sql/mx_coordinator_shouldhaveshards.sql b/src/test/regress/sql/mx_coordinator_shouldhaveshards.sql index 44d1ed48c..2b2d9b118 100644 --- a/src/test/regress/sql/mx_coordinator_shouldhaveshards.sql +++ b/src/test/regress/sql/mx_coordinator_shouldhaveshards.sql @@ -19,6 +19,18 @@ SELECT create_distributed_table('table_2', 'key', colocate_with := 'none'); INSERT INTO table_1 VALUES (1, '1'), (2, '2'), (3, '3'), (4, '4'); INSERT INTO table_2 VALUES (1, '1'), (2, '2'), (3, '3'), (4, '4'), (5, '5'), (6, '6'); +SET citus.shard_replication_factor to 2; + +CREATE TABLE table_1_rep (key int, value text); +SELECT create_distributed_table('table_1_rep', 'key', colocate_with := 'none'); + +CREATE TABLE table_2_rep (key int, value text); +SELECT create_distributed_table('table_2_rep', 'key', colocate_with := 'none'); + +INSERT INTO table_1_rep VALUES (1, '1'), (2, '2'), (3, '3'), (4, '4'); +INSERT INTO table_2_rep VALUES (1, '1'), (2, '2'), (3, '3'), (4, '4'), (5, '5'), (6, '6'); + + set citus.log_intermediate_results TO ON; set client_min_messages to debug1; @@ -50,6 +62,34 @@ inserts AS ( RETURNING * ) SELECT count(*) FROM inserts; +WITH a AS (SELECT * FROM table_1_rep ORDER BY 1,2 DESC LIMIT 1) +SELECT count(*), +key +FROM a JOIN table_2_rep USING (key) +GROUP BY key +HAVING (max(table_2_rep.value) >= (SELECT value FROM a)); + +WITH a AS (SELECT * FROM table_1_rep ORDER BY 1,2 DESC LIMIT 1) +INSERT INTO table_1_rep SELECT count(*), +key +FROM a JOIN table_2_rep USING (key) +GROUP BY key +HAVING (max(table_2_rep.value) >= (SELECT value FROM a)); + +WITH stats AS ( + SELECT count(key) m FROM table_1_rep +), +inserts AS ( + INSERT INTO table_2_rep + SELECT key, count(*) + FROM table_1_rep + WHERE key >= (SELECT m FROM stats) + GROUP BY key + HAVING count(*) <= (SELECT m FROM stats) + LIMIT 1 + RETURNING * +) SELECT count(*) FROM inserts; + \c - - - :worker_1_port SET search_path TO mx_coordinator_shouldhaveshards; @@ -84,6 +124,34 @@ inserts AS ( RETURNING * ) SELECT count(*) FROM inserts; +WITH a AS (SELECT * FROM table_1_rep ORDER BY 1,2 DESC LIMIT 1) +SELECT count(*), +key +FROM a JOIN table_2_rep USING (key) +GROUP BY key +HAVING (max(table_2_rep.value) >= (SELECT value FROM a)); + +WITH a AS (SELECT * FROM table_1_rep ORDER BY 1,2 DESC LIMIT 1) +INSERT INTO table_1_rep SELECT count(*), +key +FROM a JOIN table_2_rep USING (key) +GROUP BY key +HAVING (max(table_2_rep.value) >= (SELECT value FROM a)); + +WITH stats AS ( + SELECT count(key) m FROM table_1_rep +), +inserts AS ( + INSERT INTO table_2_rep + SELECT key, count(*) + FROM table_1_rep + WHERE key >= (SELECT m FROM stats) + GROUP BY key + HAVING count(*) <= (SELECT m FROM stats) + LIMIT 1 + RETURNING * +) SELECT count(*) FROM inserts; + \c - - - :master_port SELECT 1 FROM master_set_node_property('localhost', :master_port, 'shouldhaveshards', false); diff --git a/src/test/regress/sql/start_stop_metadata_sync.sql b/src/test/regress/sql/start_stop_metadata_sync.sql index dc01cf87c..83113f89e 100644 --- a/src/test/regress/sql/start_stop_metadata_sync.sql +++ b/src/test/regress/sql/start_stop_metadata_sync.sql @@ -68,6 +68,35 @@ SELECT alter_table_set_access_method('events_2021_jan', 'columnar'); VACUUM (FREEZE, ANALYZE) events_2021_jan; +-- add some replicated tables +SET citus.shard_replication_factor TO 2; + +-- test for hybrid partitioned table (columnar+heap) +CREATE TABLE events_replicated(ts timestamptz, i int, n numeric, s text) + PARTITION BY RANGE (ts); + +CREATE TABLE events_replicated_2021_jan PARTITION OF events_replicated + FOR VALUES FROM ('2021-01-01') TO ('2021-02-01'); + +CREATE TABLE events_replicated_2021_feb PARTITION OF events_replicated + FOR VALUES FROM ('2021-02-01') TO ('2021-03-01'); + +INSERT INTO events_replicated SELECT + '2021-01-01'::timestamptz + '0.45 seconds'::interval * g, + g, + g*pi(), + 'number: ' || g::text + FROM generate_series(1,1000) g; + +VACUUM (FREEZE, ANALYZE) events_2021_feb; + +SELECT create_distributed_table('events_replicated', 'ts'); +SELECT alter_table_set_access_method('events_replicated_2021_jan', 'columnar'); + +CREATE TABLE distributed_table_replicated_1(col int unique, b tt2); +SELECT create_distributed_table('distributed_table_replicated_1', 'col'); +CREATE INDEX indrep1 ON distributed_table_replicated_1(b); + -- sync metadata SELECT start_metadata_sync_to_node('localhost', :worker_1_port); @@ -230,6 +259,15 @@ BEGIN; SELECT create_distributed_table('test_table', 'a'); ROLLBACK; +-- this is safe because start_metadata_sync_to_node already switches to +-- sequential execution +BEGIN; + SELECT start_metadata_sync_to_node('localhost', :worker_1_port); + SET LOCAL citus.shard_replication_factor TO 2; + CREATE TABLE test_table_rep(a int); + SELECT create_distributed_table('test_table_rep', 'a'); +ROLLBACK; + -- multi-shard commands are allowed with start_metadata_sync -- as long as the start_metadata_sync_to_node executed -- when it is OK to switch to sequential execution @@ -248,6 +286,26 @@ BEGIN; SELECT start_metadata_sync_to_node('localhost', :worker_1_port); ROLLBACK; +-- multi-shard commands are allowed with start_metadata_sync +-- as long as the start_metadata_sync_to_node executed +-- when it is OK to switch to sequential execution +BEGIN; + -- sync at the start of the tx + SELECT start_metadata_sync_to_node('localhost', :worker_1_port); + SET citus.multi_shard_modify_mode TO sequential; + SET LOCAL citus.shard_replication_factor TO 2; + CREATE TABLE test_table(a int); + SELECT create_distributed_table('test_table', 'a'); + ALTER TABLE test_table ADD COLUMN B INT; + INSERT INTO test_table SELECT i,i From generate_series(0,100)i; + SELECT count(*) FROM test_table; + ALTER TABLE distributed_table_3 ADD COLUMN new_col INT DEFAULT 15; + SELECT count(*) FROM distributed_table_3; + -- sync at the end of the tx + SELECT start_metadata_sync_to_node('localhost', :worker_1_port); +ROLLBACK; + + -- cleanup \c - - - :master_port SET search_path TO "start_stop_metadata_sync";