mirror of https://github.com/citusdata/citus.git
Merge pull request #3637 from citusdata/defer_reference_table_replication_copy
Defer reference table replicationpull/3738/head^2
commit
cd877f3fdd
|
@ -346,6 +346,12 @@ CreateDistributedTable(Oid relationId, Var *distributionColumn, char distributio
|
||||||
EnsureRelationCanBeDistributed(relationId, distributionColumn, distributionMethod,
|
EnsureRelationCanBeDistributed(relationId, distributionColumn, distributionMethod,
|
||||||
colocationId, replicationModel, viaDeprecatedAPI);
|
colocationId, replicationModel, viaDeprecatedAPI);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Make sure that existing reference tables have been replicated to all the nodes
|
||||||
|
* such that we can create foreign keys and joins work immediately after creation.
|
||||||
|
*/
|
||||||
|
EnsureReferenceTablesExistOnAllNodes();
|
||||||
|
|
||||||
/* we need to calculate these variables before creating distributed metadata */
|
/* we need to calculate these variables before creating distributed metadata */
|
||||||
bool localTableEmpty = LocalTableEmpty(relationId);
|
bool localTableEmpty = LocalTableEmpty(relationId);
|
||||||
Oid colocatedTableId = ColocatedTableId(colocationId);
|
Oid colocatedTableId = ColocatedTableId(colocationId);
|
||||||
|
|
|
@ -26,6 +26,8 @@
|
||||||
#include "distributed/metadata/distobject.h"
|
#include "distributed/metadata/distobject.h"
|
||||||
#include "distributed/metadata_cache.h"
|
#include "distributed/metadata_cache.h"
|
||||||
#include <distributed/metadata_sync.h>
|
#include <distributed/metadata_sync.h>
|
||||||
|
#include "distributed/reference_table_utils.h"
|
||||||
|
#include "distributed/resource_lock.h"
|
||||||
#include <distributed/remote_commands.h>
|
#include <distributed/remote_commands.h>
|
||||||
#include <distributed/remote_commands.h>
|
#include <distributed/remote_commands.h>
|
||||||
#include "nodes/parsenodes.h"
|
#include "nodes/parsenodes.h"
|
||||||
|
@ -90,6 +92,13 @@ PreprocessDropSchemaStmt(Node *node, const char *queryString)
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (IsReferenceTable(relationId))
|
||||||
|
{
|
||||||
|
/* prevent concurrent EnsureReferenceTablesExistOnAllNodes */
|
||||||
|
int colocationId = CreateReferenceTableColocationId();
|
||||||
|
LockColocationId(colocationId, ExclusiveLock);
|
||||||
|
}
|
||||||
|
|
||||||
/* invalidate foreign key cache if the table involved in any foreign key */
|
/* invalidate foreign key cache if the table involved in any foreign key */
|
||||||
if (TableReferenced(relationId) || TableReferencing(relationId))
|
if (TableReferenced(relationId) || TableReferencing(relationId))
|
||||||
{
|
{
|
||||||
|
|
|
@ -29,6 +29,7 @@
|
||||||
#include "distributed/metadata_sync.h"
|
#include "distributed/metadata_sync.h"
|
||||||
#include "distributed/multi_executor.h"
|
#include "distributed/multi_executor.h"
|
||||||
#include "distributed/multi_partitioning_utils.h"
|
#include "distributed/multi_partitioning_utils.h"
|
||||||
|
#include "distributed/reference_table_utils.h"
|
||||||
#include "distributed/relation_access_tracking.h"
|
#include "distributed/relation_access_tracking.h"
|
||||||
#include "distributed/resource_lock.h"
|
#include "distributed/resource_lock.h"
|
||||||
#include "distributed/version_compat.h"
|
#include "distributed/version_compat.h"
|
||||||
|
@ -85,6 +86,13 @@ PreprocessDropTableStmt(Node *node, const char *queryString)
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (IsReferenceTable(relationId))
|
||||||
|
{
|
||||||
|
/* prevent concurrent EnsureReferenceTablesExistOnAllNodes */
|
||||||
|
int colocationId = CreateReferenceTableColocationId();
|
||||||
|
LockColocationId(colocationId, ExclusiveLock);
|
||||||
|
}
|
||||||
|
|
||||||
/* invalidate foreign key cache if the table involved in any foreign key */
|
/* invalidate foreign key cache if the table involved in any foreign key */
|
||||||
if ((TableReferenced(relationId) || TableReferencing(relationId)))
|
if ((TableReferenced(relationId) || TableReferencing(relationId)))
|
||||||
{
|
{
|
||||||
|
|
|
@ -86,6 +86,8 @@ master_create_worker_shards(PG_FUNCTION_ARGS)
|
||||||
ObjectAddressSet(tableAddress, RelationRelationId, distributedTableId);
|
ObjectAddressSet(tableAddress, RelationRelationId, distributedTableId);
|
||||||
EnsureDependenciesExistOnAllNodes(&tableAddress);
|
EnsureDependenciesExistOnAllNodes(&tableAddress);
|
||||||
|
|
||||||
|
EnsureReferenceTablesExistOnAllNodes();
|
||||||
|
|
||||||
CreateShardsWithRoundRobinPolicy(distributedTableId, shardCount, replicationFactor,
|
CreateShardsWithRoundRobinPolicy(distributedTableId, shardCount, replicationFactor,
|
||||||
useExclusiveConnections);
|
useExclusiveConnections);
|
||||||
|
|
||||||
|
|
|
@ -28,6 +28,7 @@
|
||||||
#include "distributed/metadata_sync.h"
|
#include "distributed/metadata_sync.h"
|
||||||
#include "distributed/multi_join_order.h"
|
#include "distributed/multi_join_order.h"
|
||||||
#include "distributed/multi_partitioning_utils.h"
|
#include "distributed/multi_partitioning_utils.h"
|
||||||
|
#include "distributed/reference_table_utils.h"
|
||||||
#include "distributed/resource_lock.h"
|
#include "distributed/resource_lock.h"
|
||||||
#include "distributed/worker_manager.h"
|
#include "distributed/worker_manager.h"
|
||||||
#include "distributed/worker_protocol.h"
|
#include "distributed/worker_protocol.h"
|
||||||
|
@ -42,12 +43,6 @@
|
||||||
#include "utils/lsyscache.h"
|
#include "utils/lsyscache.h"
|
||||||
#include "utils/palloc.h"
|
#include "utils/palloc.h"
|
||||||
|
|
||||||
|
|
||||||
#define TRANSFER_MODE_AUTOMATIC 'a'
|
|
||||||
#define TRANSFER_MODE_FORCE_LOGICAL 'l'
|
|
||||||
#define TRANSFER_MODE_BLOCK_WRITES 'b'
|
|
||||||
|
|
||||||
|
|
||||||
/* local function forward declarations */
|
/* local function forward declarations */
|
||||||
static char LookupShardTransferMode(Oid shardReplicationModeOid);
|
static char LookupShardTransferMode(Oid shardReplicationModeOid);
|
||||||
static void ErrorIfTableCannotBeReplicated(Oid relationId);
|
static void ErrorIfTableCannotBeReplicated(Oid relationId);
|
||||||
|
@ -449,6 +444,19 @@ ReplicateColocatedShardPlacement(int64 shardId, char *sourceNodeName,
|
||||||
targetNodeName, targetNodePort);
|
targetNodeName, targetNodePort);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (!IsReferenceTable(distributedTableId))
|
||||||
|
{
|
||||||
|
/*
|
||||||
|
* When copying a shard to a new node, we should first ensure that reference
|
||||||
|
* tables are present such that joins work immediately after copying the shard.
|
||||||
|
* When copying a reference table, we are probably trying to achieve just that.
|
||||||
|
*
|
||||||
|
* Since this a long-running operation we do this after the error checks, but
|
||||||
|
* before taking metadata locks.
|
||||||
|
*/
|
||||||
|
EnsureReferenceTablesExistOnAllNodes();
|
||||||
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* CopyColocatedShardPlacement function copies given shard with its co-located
|
* CopyColocatedShardPlacement function copies given shard with its co-located
|
||||||
* shards.
|
* shards.
|
||||||
|
|
|
@ -3591,6 +3591,55 @@ DistTableOidList(void)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* ReferenceTableOidList function scans pg_dist_partition to create a list of all
|
||||||
|
* reference tables. To create the list, it performs sequential scan. Since it is not
|
||||||
|
* expected that this function will be called frequently, it is OK not to use index scan.
|
||||||
|
* If this function becomes performance bottleneck, it is possible to modify this function
|
||||||
|
* to perform index scan.
|
||||||
|
*/
|
||||||
|
List *
|
||||||
|
ReferenceTableOidList()
|
||||||
|
{
|
||||||
|
ScanKeyData scanKey[1];
|
||||||
|
int scanKeyCount = 0;
|
||||||
|
List *referenceTableOidList = NIL;
|
||||||
|
|
||||||
|
Relation pgDistPartition = heap_open(DistPartitionRelationId(), AccessShareLock);
|
||||||
|
|
||||||
|
SysScanDesc scanDescriptor = systable_beginscan(pgDistPartition,
|
||||||
|
InvalidOid, false,
|
||||||
|
NULL, scanKeyCount, scanKey);
|
||||||
|
|
||||||
|
TupleDesc tupleDescriptor = RelationGetDescr(pgDistPartition);
|
||||||
|
|
||||||
|
HeapTuple heapTuple = systable_getnext(scanDescriptor);
|
||||||
|
while (HeapTupleIsValid(heapTuple))
|
||||||
|
{
|
||||||
|
bool isNull = false;
|
||||||
|
Datum relationIdDatum = heap_getattr(heapTuple,
|
||||||
|
Anum_pg_dist_partition_logicalrelid,
|
||||||
|
tupleDescriptor, &isNull);
|
||||||
|
Oid relationId = DatumGetObjectId(relationIdDatum);
|
||||||
|
char partitionMethod = heap_getattr(heapTuple,
|
||||||
|
Anum_pg_dist_partition_partmethod,
|
||||||
|
tupleDescriptor, &isNull);
|
||||||
|
|
||||||
|
if (partitionMethod == DISTRIBUTE_BY_NONE)
|
||||||
|
{
|
||||||
|
referenceTableOidList = lappend_oid(referenceTableOidList, relationId);
|
||||||
|
}
|
||||||
|
|
||||||
|
heapTuple = systable_getnext(scanDescriptor);
|
||||||
|
}
|
||||||
|
|
||||||
|
systable_endscan(scanDescriptor);
|
||||||
|
heap_close(pgDistPartition, AccessShareLock);
|
||||||
|
|
||||||
|
return referenceTableOidList;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* InvalidateNodeRelationCacheCallback destroys the WorkerNodeHash when
|
* InvalidateNodeRelationCacheCallback destroys the WorkerNodeHash when
|
||||||
* any change happens on pg_dist_node table. It also set WorkerNodeHash to
|
* any change happens on pg_dist_node table. It also set WorkerNodeHash to
|
||||||
|
|
|
@ -60,6 +60,15 @@ int GroupSize = 1;
|
||||||
/* config variable managed via guc.c */
|
/* config variable managed via guc.c */
|
||||||
char *CurrentCluster = "default";
|
char *CurrentCluster = "default";
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Config variable to control whether we should replicate reference tables on
|
||||||
|
* node activation or we should defer it to shard creation.
|
||||||
|
*/
|
||||||
|
bool ReplicateReferenceTablesOnActivate = true;
|
||||||
|
|
||||||
|
/* did current transaction modify pg_dist_node? */
|
||||||
|
bool TransactionModifiedNodeMetadata = false;
|
||||||
|
|
||||||
typedef struct NodeMetadata
|
typedef struct NodeMetadata
|
||||||
{
|
{
|
||||||
int32 groupId;
|
int32 groupId;
|
||||||
|
@ -158,6 +167,7 @@ master_add_node(PG_FUNCTION_ARGS)
|
||||||
|
|
||||||
int nodeId = AddNodeMetadata(nodeNameString, nodePort, &nodeMetadata,
|
int nodeId = AddNodeMetadata(nodeNameString, nodePort, &nodeMetadata,
|
||||||
&nodeAlreadyExists);
|
&nodeAlreadyExists);
|
||||||
|
TransactionModifiedNodeMetadata = true;
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* After adding new node, if the node did not already exist, we will activate
|
* After adding new node, if the node did not already exist, we will activate
|
||||||
|
@ -196,6 +206,7 @@ master_add_inactive_node(PG_FUNCTION_ARGS)
|
||||||
|
|
||||||
int nodeId = AddNodeMetadata(nodeNameString, nodePort, &nodeMetadata,
|
int nodeId = AddNodeMetadata(nodeNameString, nodePort, &nodeMetadata,
|
||||||
&nodeAlreadyExists);
|
&nodeAlreadyExists);
|
||||||
|
TransactionModifiedNodeMetadata = true;
|
||||||
|
|
||||||
PG_RETURN_INT32(nodeId);
|
PG_RETURN_INT32(nodeId);
|
||||||
}
|
}
|
||||||
|
@ -229,6 +240,7 @@ master_add_secondary_node(PG_FUNCTION_ARGS)
|
||||||
|
|
||||||
int nodeId = AddNodeMetadata(nodeNameString, nodePort, &nodeMetadata,
|
int nodeId = AddNodeMetadata(nodeNameString, nodePort, &nodeMetadata,
|
||||||
&nodeAlreadyExists);
|
&nodeAlreadyExists);
|
||||||
|
TransactionModifiedNodeMetadata = true;
|
||||||
|
|
||||||
PG_RETURN_INT32(nodeId);
|
PG_RETURN_INT32(nodeId);
|
||||||
}
|
}
|
||||||
|
@ -252,6 +264,7 @@ master_remove_node(PG_FUNCTION_ARGS)
|
||||||
CheckCitusVersion(ERROR);
|
CheckCitusVersion(ERROR);
|
||||||
|
|
||||||
RemoveNodeFromCluster(text_to_cstring(nodeNameText), nodePort);
|
RemoveNodeFromCluster(text_to_cstring(nodeNameText), nodePort);
|
||||||
|
TransactionModifiedNodeMetadata = true;
|
||||||
|
|
||||||
PG_RETURN_VOID();
|
PG_RETURN_VOID();
|
||||||
}
|
}
|
||||||
|
@ -305,6 +318,7 @@ master_disable_node(PG_FUNCTION_ARGS)
|
||||||
}
|
}
|
||||||
|
|
||||||
SetNodeState(nodeName, nodePort, isActive);
|
SetNodeState(nodeName, nodePort, isActive);
|
||||||
|
TransactionModifiedNodeMetadata = true;
|
||||||
}
|
}
|
||||||
PG_CATCH();
|
PG_CATCH();
|
||||||
{
|
{
|
||||||
|
@ -351,6 +365,7 @@ master_set_node_property(PG_FUNCTION_ARGS)
|
||||||
)));
|
)));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
TransactionModifiedNodeMetadata = true;
|
||||||
|
|
||||||
PG_RETURN_VOID();
|
PG_RETURN_VOID();
|
||||||
}
|
}
|
||||||
|
@ -371,8 +386,12 @@ SetUpDistributedTableDependencies(WorkerNode *newWorkerNode)
|
||||||
EnsureNoModificationsHaveBeenDone();
|
EnsureNoModificationsHaveBeenDone();
|
||||||
ReplicateAllDependenciesToNode(newWorkerNode->workerName,
|
ReplicateAllDependenciesToNode(newWorkerNode->workerName,
|
||||||
newWorkerNode->workerPort);
|
newWorkerNode->workerPort);
|
||||||
|
|
||||||
|
if (ReplicateReferenceTablesOnActivate)
|
||||||
|
{
|
||||||
ReplicateAllReferenceTablesToNode(newWorkerNode->workerName,
|
ReplicateAllReferenceTablesToNode(newWorkerNode->workerName,
|
||||||
newWorkerNode->workerPort);
|
newWorkerNode->workerPort);
|
||||||
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Let the maintenance daemon do the hard work of syncing the metadata.
|
* Let the maintenance daemon do the hard work of syncing the metadata.
|
||||||
|
@ -452,6 +471,8 @@ master_activate_node(PG_FUNCTION_ARGS)
|
||||||
nodePort);
|
nodePort);
|
||||||
ActivateNode(workerNode->workerName, workerNode->workerPort);
|
ActivateNode(workerNode->workerName, workerNode->workerPort);
|
||||||
|
|
||||||
|
TransactionModifiedNodeMetadata = true;
|
||||||
|
|
||||||
PG_RETURN_INT32(workerNode->nodeId);
|
PG_RETURN_INT32(workerNode->nodeId);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -723,6 +744,8 @@ master_update_node(PG_FUNCTION_ARGS)
|
||||||
TerminateBackgroundWorker(handle);
|
TerminateBackgroundWorker(handle);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
TransactionModifiedNodeMetadata = true;
|
||||||
|
|
||||||
PG_RETURN_VOID();
|
PG_RETURN_VOID();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -53,6 +53,7 @@
|
||||||
#include "distributed/multi_server_executor.h"
|
#include "distributed/multi_server_executor.h"
|
||||||
#include "distributed/pg_dist_partition.h"
|
#include "distributed/pg_dist_partition.h"
|
||||||
#include "distributed/placement_connection.h"
|
#include "distributed/placement_connection.h"
|
||||||
|
#include "distributed/reference_table_utils.h"
|
||||||
#include "distributed/relation_access_tracking.h"
|
#include "distributed/relation_access_tracking.h"
|
||||||
#include "distributed/run_from_same_connection.h"
|
#include "distributed/run_from_same_connection.h"
|
||||||
#include "distributed/query_pushdown_planning.h"
|
#include "distributed/query_pushdown_planning.h"
|
||||||
|
@ -1370,6 +1371,16 @@ RegisterCitusConfigVariables(void)
|
||||||
GUC_NO_SHOW_ALL,
|
GUC_NO_SHOW_ALL,
|
||||||
NULL, NULL, NULL);
|
NULL, NULL, NULL);
|
||||||
|
|
||||||
|
DefineCustomBoolVariable(
|
||||||
|
"citus.replicate_reference_tables_on_activate",
|
||||||
|
NULL,
|
||||||
|
NULL,
|
||||||
|
&ReplicateReferenceTablesOnActivate,
|
||||||
|
true,
|
||||||
|
PGC_USERSET,
|
||||||
|
GUC_NO_SHOW_ALL,
|
||||||
|
NULL, NULL, NULL);
|
||||||
|
|
||||||
/* warn about config items in the citus namespace that are not registered above */
|
/* warn about config items in the citus namespace that are not registered above */
|
||||||
EmitWarningsOnPlaceholders("citus");
|
EmitWarningsOnPlaceholders("citus");
|
||||||
}
|
}
|
||||||
|
|
|
@ -4,3 +4,4 @@
|
||||||
|
|
||||||
#include "udfs/citus_extradata_container/9.3-2.sql"
|
#include "udfs/citus_extradata_container/9.3-2.sql"
|
||||||
#include "udfs/update_distributed_table_colocation/9.3-2.sql"
|
#include "udfs/update_distributed_table_colocation/9.3-2.sql"
|
||||||
|
#include "udfs/replicate_reference_tables/9.3-2.sql"
|
||||||
|
|
|
@ -0,0 +1,7 @@
|
||||||
|
CREATE FUNCTION pg_catalog.replicate_reference_tables()
|
||||||
|
RETURNS VOID
|
||||||
|
LANGUAGE C STRICT
|
||||||
|
AS 'MODULE_PATHNAME', $$replicate_reference_tables$$;
|
||||||
|
COMMENT ON FUNCTION pg_catalog.replicate_reference_tables()
|
||||||
|
IS 'replicate reference tables to all nodes';
|
||||||
|
REVOKE ALL ON FUNCTION pg_catalog.replicate_reference_tables() FROM PUBLIC;
|
|
@ -0,0 +1,7 @@
|
||||||
|
CREATE FUNCTION pg_catalog.replicate_reference_tables()
|
||||||
|
RETURNS VOID
|
||||||
|
LANGUAGE C STRICT
|
||||||
|
AS 'MODULE_PATHNAME', $$replicate_reference_tables$$;
|
||||||
|
COMMENT ON FUNCTION pg_catalog.replicate_reference_tables()
|
||||||
|
IS 'replicate reference tables to all nodes';
|
||||||
|
REVOKE ALL ON FUNCTION pg_catalog.replicate_reference_tables() FROM PUBLIC;
|
|
@ -449,6 +449,7 @@ ResetGlobalVariables()
|
||||||
dlist_init(&InProgressTransactions);
|
dlist_init(&InProgressTransactions);
|
||||||
activeSetStmts = NULL;
|
activeSetStmts = NULL;
|
||||||
CoordinatedTransactionUses2PC = false;
|
CoordinatedTransactionUses2PC = false;
|
||||||
|
TransactionModifiedNodeMetadata = false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -22,28 +22,313 @@
|
||||||
#include "distributed/master_metadata_utility.h"
|
#include "distributed/master_metadata_utility.h"
|
||||||
#include "distributed/metadata_cache.h"
|
#include "distributed/metadata_cache.h"
|
||||||
#include "distributed/metadata_sync.h"
|
#include "distributed/metadata_sync.h"
|
||||||
|
#include "distributed/multi_executor.h"
|
||||||
#include "distributed/multi_logical_planner.h"
|
#include "distributed/multi_logical_planner.h"
|
||||||
#include "distributed/reference_table_utils.h"
|
#include "distributed/reference_table_utils.h"
|
||||||
|
#include "distributed/relation_access_tracking.h"
|
||||||
|
#include "distributed/remote_commands.h"
|
||||||
#include "distributed/resource_lock.h"
|
#include "distributed/resource_lock.h"
|
||||||
#include "distributed/shardinterval_utils.h"
|
#include "distributed/shardinterval_utils.h"
|
||||||
#include "distributed/transaction_management.h"
|
#include "distributed/transaction_management.h"
|
||||||
#include "distributed/worker_manager.h"
|
#include "distributed/worker_manager.h"
|
||||||
#include "distributed/worker_transaction.h"
|
#include "distributed/worker_transaction.h"
|
||||||
|
#include "postmaster/postmaster.h"
|
||||||
#include "storage/lmgr.h"
|
#include "storage/lmgr.h"
|
||||||
|
#include "utils/builtins.h"
|
||||||
#include "utils/fmgroids.h"
|
#include "utils/fmgroids.h"
|
||||||
#include "utils/lsyscache.h"
|
#include "utils/lsyscache.h"
|
||||||
#include "utils/rel.h"
|
#include "utils/rel.h"
|
||||||
|
|
||||||
|
|
||||||
/* local function forward declarations */
|
/* local function forward declarations */
|
||||||
|
static List * WorkersWithoutReferenceTablePlacement(uint64 shardId, LOCKMODE lockMode);
|
||||||
|
static StringInfo CopyShardPlacementToWorkerNodeQuery(
|
||||||
|
ShardPlacement *sourceShardPlacement,
|
||||||
|
WorkerNode *workerNode,
|
||||||
|
char transferMode);
|
||||||
static void ReplicateSingleShardTableToAllNodes(Oid relationId);
|
static void ReplicateSingleShardTableToAllNodes(Oid relationId);
|
||||||
static void ReplicateShardToAllNodes(ShardInterval *shardInterval);
|
static void ReplicateShardToAllNodes(ShardInterval *shardInterval);
|
||||||
static void ReplicateShardToNode(ShardInterval *shardInterval, char *nodeName,
|
static void ReplicateShardToNode(ShardInterval *shardInterval, char *nodeName,
|
||||||
int nodePort);
|
int nodePort);
|
||||||
static void ConvertToReferenceTableMetadata(Oid relationId, uint64 shardId);
|
static void ConvertToReferenceTableMetadata(Oid relationId, uint64 shardId);
|
||||||
|
static bool AnyRelationsModifiedInTransaction(List *relationIdList);
|
||||||
|
|
||||||
/* exports for SQL callable functions */
|
/* exports for SQL callable functions */
|
||||||
PG_FUNCTION_INFO_V1(upgrade_to_reference_table);
|
PG_FUNCTION_INFO_V1(upgrade_to_reference_table);
|
||||||
|
PG_FUNCTION_INFO_V1(replicate_reference_tables);
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* IsReferenceTable returns whether the given relation ID identifies a reference
|
||||||
|
* table.
|
||||||
|
*/
|
||||||
|
bool
|
||||||
|
IsReferenceTable(Oid relationId)
|
||||||
|
{
|
||||||
|
CitusTableCacheEntry *tableEntry = GetCitusTableCacheEntry(relationId);
|
||||||
|
|
||||||
|
if (!tableEntry->isCitusTable)
|
||||||
|
{
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (tableEntry->partitionMethod != DISTRIBUTE_BY_NONE)
|
||||||
|
{
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* replicate_reference_tables is a UDF to ensure that allreference tables are
|
||||||
|
* replicated to all nodes.
|
||||||
|
*/
|
||||||
|
Datum
|
||||||
|
replicate_reference_tables(PG_FUNCTION_ARGS)
|
||||||
|
{
|
||||||
|
EnsureReferenceTablesExistOnAllNodes();
|
||||||
|
|
||||||
|
PG_RETURN_VOID();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* EnsureReferenceTablesExistOnAllNodes ensures that a shard placement for every
|
||||||
|
* reference table exists on all nodes. If a node does not have a set of shard
|
||||||
|
* placements, then master_copy_shard_placement is called in a subtransaction
|
||||||
|
* to pull the data to the new node.
|
||||||
|
*/
|
||||||
|
void
|
||||||
|
EnsureReferenceTablesExistOnAllNodes(void)
|
||||||
|
{
|
||||||
|
/*
|
||||||
|
* TODO: remove this. This is here so we don't have to update regression test
|
||||||
|
* outputs because of reference table colocation id being changed. Specially
|
||||||
|
* multi_colocation_utils which requires the test itself being updated.
|
||||||
|
*/
|
||||||
|
if (ReferenceTableOidList() == NIL)
|
||||||
|
{
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Prevent this function from running concurrently with itself.
|
||||||
|
*
|
||||||
|
* It also prevents concurrent DROP TABLE or DROP SCHEMA. We need this
|
||||||
|
* because through-out this function we assume values in referenceTableIdList
|
||||||
|
* are still valid.
|
||||||
|
*
|
||||||
|
* We don't need to handle other kinds of reference table DML/DDL here, since
|
||||||
|
* master_copy_shard_placement gets enough locks for that.
|
||||||
|
*
|
||||||
|
* We also don't need special handling for concurrent create_refernece_table.
|
||||||
|
* Since that will trigger a call to this function from another backend,
|
||||||
|
* which will block until our call is finished.
|
||||||
|
*/
|
||||||
|
int colocationId = CreateReferenceTableColocationId();
|
||||||
|
LockColocationId(colocationId, ExclusiveLock);
|
||||||
|
|
||||||
|
List *referenceTableIdList = ReferenceTableOidList();
|
||||||
|
if (referenceTableIdList == NIL)
|
||||||
|
{
|
||||||
|
/* no reference tables exist */
|
||||||
|
UnlockColocationId(colocationId, ExclusiveLock);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
Oid referenceTableId = linitial_oid(referenceTableIdList);
|
||||||
|
const char *referenceTableName = get_rel_name(referenceTableId);
|
||||||
|
List *shardIntervalList = LoadShardIntervalList(referenceTableId);
|
||||||
|
if (list_length(shardIntervalList) == 0)
|
||||||
|
{
|
||||||
|
/* check for corrupt metadata */
|
||||||
|
ereport(ERROR, (errmsg("reference table \"%s\" does not have a shard",
|
||||||
|
referenceTableName)));
|
||||||
|
}
|
||||||
|
|
||||||
|
ShardInterval *shardInterval = (ShardInterval *) linitial(shardIntervalList);
|
||||||
|
uint64 shardId = shardInterval->shardId;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* We only take an access share lock, otherwise we'll hold up master_add_node.
|
||||||
|
* In case of create_reference_table() and upgrade_to_reference_table(), where
|
||||||
|
* we don't want concurrent writes to pg_dist_node, we have already acquired
|
||||||
|
* ShareLock on pg_dist_node.
|
||||||
|
*/
|
||||||
|
List *newWorkersList = WorkersWithoutReferenceTablePlacement(shardId,
|
||||||
|
AccessShareLock);
|
||||||
|
if (list_length(newWorkersList) == 0)
|
||||||
|
{
|
||||||
|
/* nothing to do, no need for lock */
|
||||||
|
UnlockColocationId(colocationId, ExclusiveLock);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* master_copy_shard_placement triggers metadata sync-up, which tries to
|
||||||
|
* acquire a ShareLock on pg_dist_node. We do master_copy_shad_placement
|
||||||
|
* in a separate connection. If we have modified pg_dist_node in the
|
||||||
|
* current backend, this will cause a deadlock.
|
||||||
|
*/
|
||||||
|
if (TransactionModifiedNodeMetadata)
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
||||||
|
errmsg("cannot replicate reference tables in a transaction "
|
||||||
|
"that modified node metadata")));
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Modifications to reference tables in current transaction are not visible
|
||||||
|
* to master_copy_shard_placement, since it is done in a separate backend.
|
||||||
|
*/
|
||||||
|
if (AnyRelationsModifiedInTransaction(referenceTableIdList))
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
||||||
|
errmsg("cannot replicate reference tables in a transaction "
|
||||||
|
"that modified a reference table")));
|
||||||
|
}
|
||||||
|
|
||||||
|
bool missingOk = false;
|
||||||
|
ShardPlacement *sourceShardPlacement = ActiveShardPlacement(shardId, missingOk);
|
||||||
|
if (sourceShardPlacement == NULL)
|
||||||
|
{
|
||||||
|
/* check for corrupt metadata */
|
||||||
|
ereport(ERROR, (errmsg("reference table shard "
|
||||||
|
UINT64_FORMAT
|
||||||
|
" does not have an active shard placement",
|
||||||
|
shardId)));
|
||||||
|
}
|
||||||
|
|
||||||
|
WorkerNode *newWorkerNode = NULL;
|
||||||
|
foreach_ptr(newWorkerNode, newWorkersList)
|
||||||
|
{
|
||||||
|
ereport(NOTICE, (errmsg("replicating reference table '%s' to %s:%d ...",
|
||||||
|
referenceTableName, newWorkerNode->workerName,
|
||||||
|
newWorkerNode->workerPort)));
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Call master_copy_shard_placement using citus extension owner. Current
|
||||||
|
* user might not have permissions to do the copy.
|
||||||
|
*/
|
||||||
|
const char *userName = CitusExtensionOwnerName();
|
||||||
|
int connectionFlags = OUTSIDE_TRANSACTION;
|
||||||
|
|
||||||
|
MultiConnection *connection = GetNodeUserDatabaseConnection(
|
||||||
|
connectionFlags, "localhost", PostPortNumber,
|
||||||
|
userName, NULL);
|
||||||
|
|
||||||
|
if (PQstatus(connection->pgConn) == CONNECTION_OK)
|
||||||
|
{
|
||||||
|
StringInfo placementCopyCommand =
|
||||||
|
CopyShardPlacementToWorkerNodeQuery(sourceShardPlacement,
|
||||||
|
newWorkerNode,
|
||||||
|
TRANSFER_MODE_AUTOMATIC);
|
||||||
|
ExecuteCriticalRemoteCommand(connection, placementCopyCommand->data);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errmsg("could not open a connection to localhost "
|
||||||
|
"when replicating reference tables"),
|
||||||
|
errdetail(
|
||||||
|
"citus.replicate_reference_tables_on_activate = false "
|
||||||
|
"requires localhost connectivity.")));
|
||||||
|
}
|
||||||
|
|
||||||
|
CloseConnection(connection);
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Unblock other backends, they will probably observe that there are no
|
||||||
|
* more worker nodes without placements, unless nodes were added concurrently
|
||||||
|
*/
|
||||||
|
UnlockColocationId(colocationId, ExclusiveLock);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* AnyRelationsModifiedInTransaction returns true if any of the given relations
|
||||||
|
* were modified in the current transaction.
|
||||||
|
*/
|
||||||
|
static bool
|
||||||
|
AnyRelationsModifiedInTransaction(List *relationIdList)
|
||||||
|
{
|
||||||
|
Oid relationId = InvalidOid;
|
||||||
|
|
||||||
|
foreach_oid(relationId, relationIdList)
|
||||||
|
{
|
||||||
|
if (GetRelationDDLAccessMode(relationId) != RELATION_NOT_ACCESSED ||
|
||||||
|
GetRelationDMLAccessMode(relationId) != RELATION_NOT_ACCESSED)
|
||||||
|
{
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* WorkersWithoutReferenceTablePlacement returns a list of workers (WorkerNode) that
|
||||||
|
* do not yet have a placement for the given reference table shard ID, but are
|
||||||
|
* supposed to.
|
||||||
|
*/
|
||||||
|
static List *
|
||||||
|
WorkersWithoutReferenceTablePlacement(uint64 shardId, LOCKMODE lockMode)
|
||||||
|
{
|
||||||
|
List *workersWithoutPlacements = NIL;
|
||||||
|
|
||||||
|
List *shardPlacementList = ActiveShardPlacementList(shardId);
|
||||||
|
|
||||||
|
List *workerNodeList = ReferenceTablePlacementNodeList(lockMode);
|
||||||
|
workerNodeList = SortList(workerNodeList, CompareWorkerNodes);
|
||||||
|
|
||||||
|
WorkerNode *workerNode = NULL;
|
||||||
|
foreach_ptr(workerNode, workerNodeList)
|
||||||
|
{
|
||||||
|
char *nodeName = workerNode->workerName;
|
||||||
|
uint32 nodePort = workerNode->workerPort;
|
||||||
|
ShardPlacement *targetPlacement = SearchShardPlacementInList(shardPlacementList,
|
||||||
|
nodeName, nodePort);
|
||||||
|
if (targetPlacement == NULL)
|
||||||
|
{
|
||||||
|
workersWithoutPlacements = lappend(workersWithoutPlacements, workerNode);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return workersWithoutPlacements;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* CopyShardPlacementToWorkerNodeQuery returns the master_copy_shard_placement
|
||||||
|
* command to copy the given shard placement to given node.
|
||||||
|
*/
|
||||||
|
static StringInfo
|
||||||
|
CopyShardPlacementToWorkerNodeQuery(ShardPlacement *sourceShardPlacement,
|
||||||
|
WorkerNode *workerNode,
|
||||||
|
char transferMode)
|
||||||
|
{
|
||||||
|
StringInfo queryString = makeStringInfo();
|
||||||
|
|
||||||
|
const char *transferModeString =
|
||||||
|
transferMode == TRANSFER_MODE_BLOCK_WRITES ? "block_writes" :
|
||||||
|
transferMode == TRANSFER_MODE_FORCE_LOGICAL ? "force_logical" :
|
||||||
|
"auto";
|
||||||
|
|
||||||
|
appendStringInfo(queryString,
|
||||||
|
"SELECT master_copy_shard_placement("
|
||||||
|
UINT64_FORMAT ", %s, %d, %s, %d, do_repair := false, "
|
||||||
|
"transfer_mode := %s)",
|
||||||
|
sourceShardPlacement->shardId,
|
||||||
|
quote_literal_cstr(sourceShardPlacement->nodeName),
|
||||||
|
sourceShardPlacement->nodePort,
|
||||||
|
quote_literal_cstr(workerNode->workerName),
|
||||||
|
workerNode->workerPort,
|
||||||
|
quote_literal_cstr(transferModeString));
|
||||||
|
|
||||||
|
return queryString;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
|
@ -104,72 +389,13 @@ upgrade_to_reference_table(PG_FUNCTION_ARGS)
|
||||||
"reference tables.", relationName)));
|
"reference tables.", relationName)));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
EnsureReferenceTablesExistOnAllNodes();
|
||||||
ReplicateSingleShardTableToAllNodes(relationId);
|
ReplicateSingleShardTableToAllNodes(relationId);
|
||||||
|
|
||||||
PG_RETURN_VOID();
|
PG_RETURN_VOID();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
|
||||||
* ReplicateAllReferenceTablesToNode function finds all reference tables and
|
|
||||||
* replicates them to the given worker node. It also modifies pg_dist_colocation
|
|
||||||
* table to update the replication factor column when necessary. This function
|
|
||||||
* skips reference tables if that node already has healthy placement of that
|
|
||||||
* reference table to prevent unnecessary data transfer.
|
|
||||||
*/
|
|
||||||
void
|
|
||||||
ReplicateAllReferenceTablesToNode(char *nodeName, int nodePort)
|
|
||||||
{
|
|
||||||
List *referenceTableList = ReferenceTableOidList();
|
|
||||||
|
|
||||||
/* if there is no reference table, we do not need to replicate anything */
|
|
||||||
if (list_length(referenceTableList) > 0)
|
|
||||||
{
|
|
||||||
List *referenceShardIntervalList = NIL;
|
|
||||||
|
|
||||||
/*
|
|
||||||
* We sort the reference table list to prevent deadlocks in concurrent
|
|
||||||
* ReplicateAllReferenceTablesToAllNodes calls.
|
|
||||||
*/
|
|
||||||
referenceTableList = SortList(referenceTableList, CompareOids);
|
|
||||||
Oid referenceTableId = InvalidOid;
|
|
||||||
foreach_oid(referenceTableId, referenceTableList)
|
|
||||||
{
|
|
||||||
List *shardIntervalList = LoadShardIntervalList(referenceTableId);
|
|
||||||
ShardInterval *shardInterval = (ShardInterval *) linitial(shardIntervalList);
|
|
||||||
|
|
||||||
referenceShardIntervalList = lappend(referenceShardIntervalList,
|
|
||||||
shardInterval);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (ClusterHasKnownMetadataWorkers())
|
|
||||||
{
|
|
||||||
BlockWritesToShardList(referenceShardIntervalList);
|
|
||||||
}
|
|
||||||
|
|
||||||
ShardInterval *shardInterval = NULL;
|
|
||||||
foreach_ptr(shardInterval, referenceShardIntervalList)
|
|
||||||
{
|
|
||||||
uint64 shardId = shardInterval->shardId;
|
|
||||||
|
|
||||||
LockShardDistributionMetadata(shardId, ExclusiveLock);
|
|
||||||
|
|
||||||
ReplicateShardToNode(shardInterval, nodeName, nodePort);
|
|
||||||
}
|
|
||||||
|
|
||||||
/* create foreign constraints between reference tables */
|
|
||||||
foreach_ptr(shardInterval, referenceShardIntervalList)
|
|
||||||
{
|
|
||||||
char *tableOwner = TableOwner(shardInterval->relationId);
|
|
||||||
List *commandList = CopyShardForeignConstraintCommandList(shardInterval);
|
|
||||||
|
|
||||||
SendCommandListToWorkerInSingleTransaction(nodeName, nodePort, tableOwner,
|
|
||||||
commandList);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* ReplicateSingleShardTableToAllNodes accepts a broadcast table and replicates
|
* ReplicateSingleShardTableToAllNodes accepts a broadcast table and replicates
|
||||||
* it to all worker nodes, and the coordinator if it has been added by the user
|
* it to all worker nodes, and the coordinator if it has been added by the user
|
||||||
|
@ -447,34 +673,6 @@ DeleteAllReferenceTablePlacementsFromNodeGroup(int32 groupId)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
|
||||||
* ReferenceTableOidList function scans pg_dist_partition to create a list of all
|
|
||||||
* reference tables. To create the list, it performs sequential scan. Since it is not
|
|
||||||
* expected that this function will be called frequently, it is OK not to use index scan.
|
|
||||||
* If this function becomes performance bottleneck, it is possible to modify this function
|
|
||||||
* to perform index scan.
|
|
||||||
*/
|
|
||||||
List *
|
|
||||||
ReferenceTableOidList()
|
|
||||||
{
|
|
||||||
List *referenceTableList = NIL;
|
|
||||||
|
|
||||||
List *distTableOidList = DistTableOidList();
|
|
||||||
Oid relationId = InvalidOid;
|
|
||||||
foreach_oid(relationId, distTableOidList)
|
|
||||||
{
|
|
||||||
CitusTableCacheEntry *cacheEntry = GetCitusTableCacheEntry(relationId);
|
|
||||||
|
|
||||||
if (cacheEntry->partitionMethod == DISTRIBUTE_BY_NONE)
|
|
||||||
{
|
|
||||||
referenceTableList = lappend_oid(referenceTableList, relationId);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return referenceTableList;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/* CompareOids is a comparison function for sort shard oids */
|
/* CompareOids is a comparison function for sort shard oids */
|
||||||
int
|
int
|
||||||
CompareOids(const void *leftElement, const void *rightElement)
|
CompareOids(const void *leftElement, const void *rightElement)
|
||||||
|
@ -508,3 +706,63 @@ ReferenceTableReplicationFactor(void)
|
||||||
int replicationFactor = list_length(nodeList);
|
int replicationFactor = list_length(nodeList);
|
||||||
return replicationFactor;
|
return replicationFactor;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* ReplicateAllReferenceTablesToNode function finds all reference tables and
|
||||||
|
* replicates them to the given worker node. It also modifies pg_dist_colocation
|
||||||
|
* table to update the replication factor column when necessary. This function
|
||||||
|
* skips reference tables if that node already has healthy placement of that
|
||||||
|
* reference table to prevent unnecessary data transfer.
|
||||||
|
*/
|
||||||
|
void
|
||||||
|
ReplicateAllReferenceTablesToNode(char *nodeName, int nodePort)
|
||||||
|
{
|
||||||
|
List *referenceTableList = ReferenceTableOidList();
|
||||||
|
|
||||||
|
/* if there is no reference table, we do not need to replicate anything */
|
||||||
|
if (list_length(referenceTableList) > 0)
|
||||||
|
{
|
||||||
|
List *referenceShardIntervalList = NIL;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* We sort the reference table list to prevent deadlocks in concurrent
|
||||||
|
* ReplicateAllReferenceTablesToAllNodes calls.
|
||||||
|
*/
|
||||||
|
referenceTableList = SortList(referenceTableList, CompareOids);
|
||||||
|
Oid referenceTableId = InvalidOid;
|
||||||
|
foreach_oid(referenceTableId, referenceTableList)
|
||||||
|
{
|
||||||
|
List *shardIntervalList = LoadShardIntervalList(referenceTableId);
|
||||||
|
ShardInterval *shardInterval = (ShardInterval *) linitial(shardIntervalList);
|
||||||
|
|
||||||
|
referenceShardIntervalList = lappend(referenceShardIntervalList,
|
||||||
|
shardInterval);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (ClusterHasKnownMetadataWorkers())
|
||||||
|
{
|
||||||
|
BlockWritesToShardList(referenceShardIntervalList);
|
||||||
|
}
|
||||||
|
|
||||||
|
ShardInterval *shardInterval = NULL;
|
||||||
|
foreach_ptr(shardInterval, referenceShardIntervalList)
|
||||||
|
{
|
||||||
|
uint64 shardId = shardInterval->shardId;
|
||||||
|
|
||||||
|
LockShardDistributionMetadata(shardId, ExclusiveLock);
|
||||||
|
|
||||||
|
ReplicateShardToNode(shardInterval, nodeName, nodePort);
|
||||||
|
}
|
||||||
|
|
||||||
|
/* create foreign constraints between reference tables */
|
||||||
|
foreach_ptr(shardInterval, referenceShardIntervalList)
|
||||||
|
{
|
||||||
|
char *tableOwner = TableOwner(shardInterval->relationId);
|
||||||
|
List *commandList = CopyShardForeignConstraintCommandList(shardInterval);
|
||||||
|
|
||||||
|
SendCommandListToWorkerInSingleTransaction(nodeName, nodePort, tableOwner,
|
||||||
|
commandList);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
|
@ -319,6 +319,36 @@ IntToLockMode(int mode)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* LockColocationId returns after acquiring a co-location ID lock, typically used
|
||||||
|
* for rebalancing and replication.
|
||||||
|
*/
|
||||||
|
void
|
||||||
|
LockColocationId(int colocationId, LOCKMODE lockMode)
|
||||||
|
{
|
||||||
|
LOCKTAG tag;
|
||||||
|
const bool sessionLock = false;
|
||||||
|
const bool dontWait = false;
|
||||||
|
|
||||||
|
SET_LOCKTAG_REBALANCE_COLOCATION(tag, (int64) colocationId);
|
||||||
|
(void) LockAcquire(&tag, lockMode, sessionLock, dontWait);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* UnlockColocationId releases a co-location ID lock.
|
||||||
|
*/
|
||||||
|
void
|
||||||
|
UnlockColocationId(int colocationId, LOCKMODE lockMode)
|
||||||
|
{
|
||||||
|
LOCKTAG tag;
|
||||||
|
const bool sessionLock = false;
|
||||||
|
|
||||||
|
SET_LOCKTAG_REBALANCE_COLOCATION(tag, (int64) colocationId);
|
||||||
|
LockRelease(&tag, lockMode, sessionLock);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* LockShardDistributionMetadata returns after grabbing a lock for distribution
|
* LockShardDistributionMetadata returns after grabbing a lock for distribution
|
||||||
* metadata related to the specified shard, blocking if required. Any locks
|
* metadata related to the specified shard, blocking if required. Any locks
|
||||||
|
|
|
@ -48,6 +48,11 @@
|
||||||
#define CANDIDATE_NODE_FIELDS 2
|
#define CANDIDATE_NODE_FIELDS 2
|
||||||
#define WORKER_NODE_FIELDS 2
|
#define WORKER_NODE_FIELDS 2
|
||||||
|
|
||||||
|
/* transfer mode for master_copy_shard_placement */
|
||||||
|
#define TRANSFER_MODE_AUTOMATIC 'a'
|
||||||
|
#define TRANSFER_MODE_FORCE_LOGICAL 'l'
|
||||||
|
#define TRANSFER_MODE_BLOCK_WRITES 'b'
|
||||||
|
|
||||||
/* Name of columnar foreign data wrapper */
|
/* Name of columnar foreign data wrapper */
|
||||||
#define CSTORE_FDW_NAME "cstore_fdw"
|
#define CSTORE_FDW_NAME "cstore_fdw"
|
||||||
|
|
||||||
|
|
|
@ -130,6 +130,7 @@ extern DistObjectCacheEntry * LookupDistObjectCacheEntry(Oid classid, Oid objid,
|
||||||
objsubid);
|
objsubid);
|
||||||
extern int32 GetLocalGroupId(void);
|
extern int32 GetLocalGroupId(void);
|
||||||
extern List * DistTableOidList(void);
|
extern List * DistTableOidList(void);
|
||||||
|
extern List * ReferenceTableOidList(void);
|
||||||
extern Oid LookupShardRelation(int64 shardId, bool missing_ok);
|
extern Oid LookupShardRelation(int64 shardId, bool missing_ok);
|
||||||
extern List * ShardPlacementList(uint64 shardId);
|
extern List * ShardPlacementList(uint64 shardId);
|
||||||
extern void CitusInvalidateRelcacheByRelid(Oid relationId);
|
extern void CitusInvalidateRelcacheByRelid(Oid relationId);
|
||||||
|
|
|
@ -16,12 +16,12 @@
|
||||||
|
|
||||||
#include "listutils.h"
|
#include "listutils.h"
|
||||||
|
|
||||||
|
extern bool IsReferenceTable(Oid relationId);
|
||||||
|
extern void EnsureReferenceTablesExistOnAllNodes(void);
|
||||||
extern uint32 CreateReferenceTableColocationId(void);
|
extern uint32 CreateReferenceTableColocationId(void);
|
||||||
extern void ReplicateAllReferenceTablesToNode(char *nodeName, int nodePort);
|
|
||||||
extern void DeleteAllReferenceTablePlacementsFromNodeGroup(int32 groupId);
|
extern void DeleteAllReferenceTablePlacementsFromNodeGroup(int32 groupId);
|
||||||
extern List * ReferenceTableOidList(void);
|
|
||||||
extern int CompareOids(const void *leftElement, const void *rightElement);
|
extern int CompareOids(const void *leftElement, const void *rightElement);
|
||||||
extern int ReferenceTableReplicationFactor(void);
|
extern int ReferenceTableReplicationFactor(void);
|
||||||
|
extern void ReplicateAllReferenceTablesToNode(char *nodeName, int nodePort);
|
||||||
|
|
||||||
#endif /* REFERENCE_TABLE_UTILS_H_ */
|
#endif /* REFERENCE_TABLE_UTILS_H_ */
|
||||||
|
|
|
@ -102,6 +102,10 @@ extern void UnlockShardResource(uint64 shardId, LOCKMODE lockmode);
|
||||||
extern void LockJobResource(uint64 jobId, LOCKMODE lockmode);
|
extern void LockJobResource(uint64 jobId, LOCKMODE lockmode);
|
||||||
extern void UnlockJobResource(uint64 jobId, LOCKMODE lockmode);
|
extern void UnlockJobResource(uint64 jobId, LOCKMODE lockmode);
|
||||||
|
|
||||||
|
/* Lock a co-location group */
|
||||||
|
extern void LockColocationId(int colocationId, LOCKMODE lockMode);
|
||||||
|
extern void UnlockColocationId(int colocationId, LOCKMODE lockMode);
|
||||||
|
|
||||||
/* Lock multiple shards for safe modification */
|
/* Lock multiple shards for safe modification */
|
||||||
extern void LockShardListMetadata(List *shardIntervalList, LOCKMODE lockMode);
|
extern void LockShardListMetadata(List *shardIntervalList, LOCKMODE lockMode);
|
||||||
extern void LockShardsInPlacementListMetadata(List *shardPlacementList,
|
extern void LockShardsInPlacementListMetadata(List *shardPlacementList,
|
||||||
|
|
|
@ -103,6 +103,9 @@ extern int DoBlockLevel;
|
||||||
/* SET LOCAL statements active in the current (sub-)transaction. */
|
/* SET LOCAL statements active in the current (sub-)transaction. */
|
||||||
extern StringInfo activeSetStmts;
|
extern StringInfo activeSetStmts;
|
||||||
|
|
||||||
|
/* did current transaction modify pg_dist_node? */
|
||||||
|
extern bool TransactionModifiedNodeMetadata;
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Coordinated transaction management.
|
* Coordinated transaction management.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -61,6 +61,7 @@ typedef struct WorkerNode
|
||||||
extern int MaxWorkerNodesTracked;
|
extern int MaxWorkerNodesTracked;
|
||||||
extern char *WorkerListFileName;
|
extern char *WorkerListFileName;
|
||||||
extern char *CurrentCluster;
|
extern char *CurrentCluster;
|
||||||
|
extern bool ReplicateReferenceTablesOnActivate;
|
||||||
|
|
||||||
|
|
||||||
/* Function declarations for finding worker nodes to place shards on */
|
/* Function declarations for finding worker nodes to place shards on */
|
||||||
|
|
|
@ -111,3 +111,6 @@ s/worker_hash_partition_table \([0-9]+/worker_hash_partition_table \(xxxxxxx/g
|
||||||
# ignore first parameter for citus_extradata_container due to differences between pg11 and pg12
|
# ignore first parameter for citus_extradata_container due to differences between pg11 and pg12
|
||||||
# can be removed when we remove PG_VERSION_NUM >= 120000
|
# can be removed when we remove PG_VERSION_NUM >= 120000
|
||||||
s/pg_catalog.citus_extradata_container\([0-9]+/pg_catalog.citus_extradata_container\(XXX/g
|
s/pg_catalog.citus_extradata_container\([0-9]+/pg_catalog.citus_extradata_container\(XXX/g
|
||||||
|
|
||||||
|
# ignore referene table replication messages
|
||||||
|
/replicating reference table.*$/d
|
||||||
|
|
|
@ -11,6 +11,7 @@ SELECT citus.mitmproxy('conn.allow()');
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
SET citus.next_shard_id TO 200000;
|
SET citus.next_shard_id TO 200000;
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
-- verify we have all worker nodes present
|
-- verify we have all worker nodes present
|
||||||
SELECT * FROM master_get_active_worker_nodes()
|
SELECT * FROM master_get_active_worker_nodes()
|
||||||
ORDER BY 1, 2;
|
ORDER BY 1, 2;
|
||||||
|
@ -75,25 +76,6 @@ ORDER BY placementid;
|
||||||
200000 | 1
|
200000 | 1
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
-- fail activate node by failing reference table creation
|
|
||||||
SELECT citus.mitmproxy('conn.onQuery(query="CREATE TABLE").kill()');
|
|
||||||
mitmproxy
|
|
||||||
---------------------------------------------------------------------
|
|
||||||
|
|
||||||
(1 row)
|
|
||||||
|
|
||||||
SELECT master_activate_node('localhost', :worker_2_proxy_port);
|
|
||||||
NOTICE: Replicating reference table "user_table" to the node localhost:xxxxx
|
|
||||||
ERROR: server closed the connection unexpectedly
|
|
||||||
This probably means the server terminated abnormally
|
|
||||||
before or while processing the request.
|
|
||||||
CONTEXT: while executing command on localhost:xxxxx
|
|
||||||
SELECT citus.mitmproxy('conn.allow()');
|
|
||||||
mitmproxy
|
|
||||||
---------------------------------------------------------------------
|
|
||||||
|
|
||||||
(1 row)
|
|
||||||
|
|
||||||
-- verify node is not activated
|
-- verify node is not activated
|
||||||
SELECT * FROM master_get_active_worker_nodes()
|
SELECT * FROM master_get_active_worker_nodes()
|
||||||
ORDER BY 1, 2;
|
ORDER BY 1, 2;
|
||||||
|
@ -140,39 +122,6 @@ ORDER BY placementid;
|
||||||
200000 | 1
|
200000 | 1
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
-- fail activate node by failing reference table creation
|
|
||||||
SELECT citus.mitmproxy('conn.onQuery(query="CREATE TABLE").cancel(' || pg_backend_pid() || ')');
|
|
||||||
mitmproxy
|
|
||||||
---------------------------------------------------------------------
|
|
||||||
|
|
||||||
(1 row)
|
|
||||||
|
|
||||||
SELECT master_activate_node('localhost', :worker_2_proxy_port);
|
|
||||||
NOTICE: Replicating reference table "user_table" to the node localhost:xxxxx
|
|
||||||
ERROR: canceling statement due to user request
|
|
||||||
-- verify node is not activated
|
|
||||||
SELECT * FROM master_get_active_worker_nodes()
|
|
||||||
ORDER BY 1, 2;
|
|
||||||
node_name | node_port
|
|
||||||
---------------------------------------------------------------------
|
|
||||||
localhost | 57637
|
|
||||||
(1 row)
|
|
||||||
|
|
||||||
SELECT shardid, shardstate
|
|
||||||
FROM pg_dist_placement p JOIN pg_dist_shard s USING (shardid)
|
|
||||||
WHERE s.logicalrelid = 'user_table'::regclass
|
|
||||||
ORDER BY placementid;
|
|
||||||
shardid | shardstate
|
|
||||||
---------------------------------------------------------------------
|
|
||||||
200000 | 1
|
|
||||||
(1 row)
|
|
||||||
|
|
||||||
SELECT citus.mitmproxy('conn.allow()');
|
|
||||||
mitmproxy
|
|
||||||
---------------------------------------------------------------------
|
|
||||||
|
|
||||||
(1 row)
|
|
||||||
|
|
||||||
-- master_remove_node fails when there are shards on that worker
|
-- master_remove_node fails when there are shards on that worker
|
||||||
SELECT master_remove_node('localhost', :worker_2_proxy_port);
|
SELECT master_remove_node('localhost', :worker_2_proxy_port);
|
||||||
ERROR: you cannot remove the primary node of a node group which has shard placements
|
ERROR: you cannot remove the primary node of a node group which has shard placements
|
||||||
|
@ -225,63 +174,6 @@ ORDER BY placementid;
|
||||||
200000 | 1
|
200000 | 1
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
-- test master_add_node replicated a reference table
|
|
||||||
-- to newly added node.
|
|
||||||
SELECT citus.mitmproxy('conn.onQuery(query="CREATE TABLE").kill()');
|
|
||||||
mitmproxy
|
|
||||||
---------------------------------------------------------------------
|
|
||||||
|
|
||||||
(1 row)
|
|
||||||
|
|
||||||
SELECT master_add_node('localhost', :worker_2_proxy_port);
|
|
||||||
NOTICE: Replicating reference table "user_table" to the node localhost:xxxxx
|
|
||||||
ERROR: server closed the connection unexpectedly
|
|
||||||
This probably means the server terminated abnormally
|
|
||||||
before or while processing the request.
|
|
||||||
CONTEXT: while executing command on localhost:xxxxx
|
|
||||||
-- verify node is not added
|
|
||||||
SELECT * FROM master_get_active_worker_nodes()
|
|
||||||
ORDER BY 1, 2;
|
|
||||||
node_name | node_port
|
|
||||||
---------------------------------------------------------------------
|
|
||||||
localhost | 57637
|
|
||||||
(1 row)
|
|
||||||
|
|
||||||
SELECT shardid, shardstate
|
|
||||||
FROM pg_dist_placement p JOIN pg_dist_shard s USING (shardid)
|
|
||||||
WHERE s.logicalrelid = 'user_table'::regclass
|
|
||||||
ORDER BY placementid;
|
|
||||||
shardid | shardstate
|
|
||||||
---------------------------------------------------------------------
|
|
||||||
200000 | 1
|
|
||||||
(1 row)
|
|
||||||
|
|
||||||
SELECT citus.mitmproxy('conn.onQuery(query="CREATE TABLE").cancel(' || pg_backend_pid() || ')');
|
|
||||||
mitmproxy
|
|
||||||
---------------------------------------------------------------------
|
|
||||||
|
|
||||||
(1 row)
|
|
||||||
|
|
||||||
SELECT master_add_node('localhost', :worker_2_proxy_port);
|
|
||||||
NOTICE: Replicating reference table "user_table" to the node localhost:xxxxx
|
|
||||||
ERROR: canceling statement due to user request
|
|
||||||
-- verify node is not added
|
|
||||||
SELECT * FROM master_get_active_worker_nodes()
|
|
||||||
ORDER BY 1, 2;
|
|
||||||
node_name | node_port
|
|
||||||
---------------------------------------------------------------------
|
|
||||||
localhost | 57637
|
|
||||||
(1 row)
|
|
||||||
|
|
||||||
SELECT shardid, shardstate
|
|
||||||
FROM pg_dist_placement p JOIN pg_dist_shard s USING (shardid)
|
|
||||||
WHERE s.logicalrelid = 'user_table'::regclass
|
|
||||||
ORDER BY placementid;
|
|
||||||
shardid | shardstate
|
|
||||||
---------------------------------------------------------------------
|
|
||||||
200000 | 1
|
|
||||||
(1 row)
|
|
||||||
|
|
||||||
-- reset cluster to original state
|
-- reset cluster to original state
|
||||||
SELECT citus.mitmproxy('conn.allow()');
|
SELECT citus.mitmproxy('conn.allow()');
|
||||||
mitmproxy
|
mitmproxy
|
||||||
|
@ -290,10 +182,9 @@ SELECT citus.mitmproxy('conn.allow()');
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
SELECT master_add_node('localhost', :worker_2_proxy_port);
|
SELECT master_add_node('localhost', :worker_2_proxy_port);
|
||||||
NOTICE: Replicating reference table "user_table" to the node localhost:xxxxx
|
|
||||||
master_add_node
|
master_add_node
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
6
|
4
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
-- verify node is added
|
-- verify node is added
|
||||||
|
@ -312,32 +203,6 @@ ORDER BY placementid;
|
||||||
shardid | shardstate
|
shardid | shardstate
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
200000 | 1
|
200000 | 1
|
||||||
200000 | 1
|
|
||||||
(2 rows)
|
|
||||||
|
|
||||||
-- fail master_add_node by failing copy out operation
|
|
||||||
SELECT master_remove_node('localhost', :worker_1_port);
|
|
||||||
master_remove_node
|
|
||||||
---------------------------------------------------------------------
|
|
||||||
|
|
||||||
(1 row)
|
|
||||||
|
|
||||||
SELECT citus.mitmproxy('conn.onQuery(query="COPY").kill()');
|
|
||||||
mitmproxy
|
|
||||||
---------------------------------------------------------------------
|
|
||||||
|
|
||||||
(1 row)
|
|
||||||
|
|
||||||
SELECT master_add_node('localhost', :worker_1_port);
|
|
||||||
NOTICE: Replicating reference table "user_table" to the node localhost:xxxxx
|
|
||||||
ERROR: could not copy table "user_table_200000" from "localhost:xxxxx"
|
|
||||||
CONTEXT: while executing command on localhost:xxxxx
|
|
||||||
-- verify node is not added
|
|
||||||
SELECT * FROM master_get_active_worker_nodes()
|
|
||||||
ORDER BY 1, 2;
|
|
||||||
node_name | node_port
|
|
||||||
---------------------------------------------------------------------
|
|
||||||
localhost | 9060
|
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
SELECT citus.mitmproxy('conn.allow()');
|
SELECT citus.mitmproxy('conn.allow()');
|
||||||
|
@ -347,10 +212,9 @@ SELECT citus.mitmproxy('conn.allow()');
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
SELECT master_add_node('localhost', :worker_1_port);
|
SELECT master_add_node('localhost', :worker_1_port);
|
||||||
NOTICE: Replicating reference table "user_table" to the node localhost:xxxxx
|
|
||||||
master_add_node
|
master_add_node
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
8
|
1
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
-- verify node is added
|
-- verify node is added
|
||||||
|
@ -369,8 +233,7 @@ ORDER BY placementid;
|
||||||
shardid | shardstate
|
shardid | shardstate
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
200000 | 1
|
200000 | 1
|
||||||
200000 | 1
|
(1 row)
|
||||||
(2 rows)
|
|
||||||
|
|
||||||
RESET SEARCH_PATH;
|
RESET SEARCH_PATH;
|
||||||
DROP SCHEMA add_remove_node CASCADE;
|
DROP SCHEMA add_remove_node CASCADE;
|
||||||
|
|
|
@ -1,9 +1,12 @@
|
||||||
Parsed test spec with 2 sessions
|
Parsed test spec with 2 sessions
|
||||||
|
|
||||||
starting permutation: s2-load-metadata-cache s1-begin s1-add-second-worker s2-copy-to-reference-table s1-commit s2-print-content
|
starting permutation: s1-do-not-replicate-on-activate s2-load-metadata-cache s1-begin s1-add-second-worker s2-copy-to-reference-table s1-commit s2-print-content
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
|
|
||||||
|
|
||||||
|
step s1-do-not-replicate-on-activate:
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
step s2-load-metadata-cache:
|
step s2-load-metadata-cache:
|
||||||
COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5';
|
COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5';
|
||||||
|
|
||||||
|
@ -18,11 +21,10 @@ step s1-add-second-worker:
|
||||||
1
|
1
|
||||||
step s2-copy-to-reference-table:
|
step s2-copy-to-reference-table:
|
||||||
COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5';
|
COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5';
|
||||||
<waiting ...>
|
|
||||||
step s1-commit:
|
step s1-commit:
|
||||||
COMMIT;
|
COMMIT;
|
||||||
|
|
||||||
step s2-copy-to-reference-table: <... completed>
|
|
||||||
step s2-print-content:
|
step s2-print-content:
|
||||||
SELECT
|
SELECT
|
||||||
nodeport, success, result
|
nodeport, success, result
|
||||||
|
@ -34,16 +36,18 @@ step s2-print-content:
|
||||||
nodeport success result
|
nodeport success result
|
||||||
|
|
||||||
57637 t 10
|
57637 t 10
|
||||||
57638 t 10
|
|
||||||
master_remove_node
|
master_remove_node
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
starting permutation: s2-load-metadata-cache s2-begin s2-copy-to-reference-table s1-add-second-worker s2-commit s2-print-content
|
starting permutation: s1-do-not-replicate-on-activate s2-load-metadata-cache s2-begin s2-copy-to-reference-table s1-add-second-worker s2-commit s2-print-content
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
|
|
||||||
|
|
||||||
|
step s1-do-not-replicate-on-activate:
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
step s2-load-metadata-cache:
|
step s2-load-metadata-cache:
|
||||||
COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5';
|
COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5';
|
||||||
|
|
||||||
|
@ -55,14 +59,13 @@ step s2-copy-to-reference-table:
|
||||||
|
|
||||||
step s1-add-second-worker:
|
step s1-add-second-worker:
|
||||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||||
<waiting ...>
|
|
||||||
step s2-commit:
|
|
||||||
COMMIT;
|
|
||||||
|
|
||||||
step s1-add-second-worker: <... completed>
|
|
||||||
?column?
|
?column?
|
||||||
|
|
||||||
1
|
1
|
||||||
|
step s2-commit:
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
step s2-print-content:
|
step s2-print-content:
|
||||||
SELECT
|
SELECT
|
||||||
nodeport, success, result
|
nodeport, success, result
|
||||||
|
@ -74,16 +77,18 @@ step s2-print-content:
|
||||||
nodeport success result
|
nodeport success result
|
||||||
|
|
||||||
57637 t 10
|
57637 t 10
|
||||||
57638 t 10
|
|
||||||
master_remove_node
|
master_remove_node
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
starting permutation: s2-load-metadata-cache s1-begin s1-add-second-worker s2-insert-to-reference-table s1-commit s2-print-content
|
starting permutation: s1-do-not-replicate-on-activate s2-load-metadata-cache s1-begin s1-add-second-worker s2-insert-to-reference-table s1-commit s2-print-content
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
|
|
||||||
|
|
||||||
|
step s1-do-not-replicate-on-activate:
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
step s2-load-metadata-cache:
|
step s2-load-metadata-cache:
|
||||||
COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5';
|
COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5';
|
||||||
|
|
||||||
|
@ -98,11 +103,10 @@ step s1-add-second-worker:
|
||||||
1
|
1
|
||||||
step s2-insert-to-reference-table:
|
step s2-insert-to-reference-table:
|
||||||
INSERT INTO test_reference_table VALUES (6);
|
INSERT INTO test_reference_table VALUES (6);
|
||||||
<waiting ...>
|
|
||||||
step s1-commit:
|
step s1-commit:
|
||||||
COMMIT;
|
COMMIT;
|
||||||
|
|
||||||
step s2-insert-to-reference-table: <... completed>
|
|
||||||
step s2-print-content:
|
step s2-print-content:
|
||||||
SELECT
|
SELECT
|
||||||
nodeport, success, result
|
nodeport, success, result
|
||||||
|
@ -114,16 +118,18 @@ step s2-print-content:
|
||||||
nodeport success result
|
nodeport success result
|
||||||
|
|
||||||
57637 t 6
|
57637 t 6
|
||||||
57638 t 6
|
|
||||||
master_remove_node
|
master_remove_node
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
starting permutation: s2-load-metadata-cache s2-begin s2-insert-to-reference-table s1-add-second-worker s2-commit s2-print-content
|
starting permutation: s1-do-not-replicate-on-activate s2-load-metadata-cache s2-begin s2-insert-to-reference-table s1-add-second-worker s2-commit s2-print-content
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
|
|
||||||
|
|
||||||
|
step s1-do-not-replicate-on-activate:
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
step s2-load-metadata-cache:
|
step s2-load-metadata-cache:
|
||||||
COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5';
|
COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5';
|
||||||
|
|
||||||
|
@ -135,14 +141,13 @@ step s2-insert-to-reference-table:
|
||||||
|
|
||||||
step s1-add-second-worker:
|
step s1-add-second-worker:
|
||||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||||
<waiting ...>
|
|
||||||
step s2-commit:
|
|
||||||
COMMIT;
|
|
||||||
|
|
||||||
step s1-add-second-worker: <... completed>
|
|
||||||
?column?
|
?column?
|
||||||
|
|
||||||
1
|
1
|
||||||
|
step s2-commit:
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
step s2-print-content:
|
step s2-print-content:
|
||||||
SELECT
|
SELECT
|
||||||
nodeport, success, result
|
nodeport, success, result
|
||||||
|
@ -154,16 +159,18 @@ step s2-print-content:
|
||||||
nodeport success result
|
nodeport success result
|
||||||
|
|
||||||
57637 t 6
|
57637 t 6
|
||||||
57638 t 6
|
|
||||||
master_remove_node
|
master_remove_node
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
starting permutation: s2-load-metadata-cache s1-begin s1-add-second-worker s2-ddl-on-reference-table s1-commit s2-print-index-count
|
starting permutation: s1-do-not-replicate-on-activate s2-load-metadata-cache s1-begin s1-add-second-worker s2-ddl-on-reference-table s1-commit s2-print-index-count
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
|
|
||||||
|
|
||||||
|
step s1-do-not-replicate-on-activate:
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
step s2-load-metadata-cache:
|
step s2-load-metadata-cache:
|
||||||
COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5';
|
COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5';
|
||||||
|
|
||||||
|
@ -194,16 +201,18 @@ step s2-print-index-count:
|
||||||
nodeport success result
|
nodeport success result
|
||||||
|
|
||||||
57637 t 1
|
57637 t 1
|
||||||
57638 t 1
|
|
||||||
master_remove_node
|
master_remove_node
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
starting permutation: s2-load-metadata-cache s2-begin s2-ddl-on-reference-table s1-add-second-worker s2-commit s2-print-index-count
|
starting permutation: s1-do-not-replicate-on-activate s2-load-metadata-cache s2-begin s2-ddl-on-reference-table s1-add-second-worker s2-commit s2-print-index-count
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
|
|
||||||
|
|
||||||
|
step s1-do-not-replicate-on-activate:
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
step s2-load-metadata-cache:
|
step s2-load-metadata-cache:
|
||||||
COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5';
|
COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5';
|
||||||
|
|
||||||
|
@ -234,16 +243,18 @@ step s2-print-index-count:
|
||||||
nodeport success result
|
nodeport success result
|
||||||
|
|
||||||
57637 t 1
|
57637 t 1
|
||||||
57638 t 1
|
|
||||||
master_remove_node
|
master_remove_node
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
starting permutation: s2-load-metadata-cache s1-begin s1-add-second-worker s2-create-reference-table-2 s1-commit s2-print-content-2
|
starting permutation: s1-do-not-replicate-on-activate s2-load-metadata-cache s1-begin s1-add-second-worker s2-create-reference-table-2 s1-commit s2-print-content-2
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
|
|
||||||
|
|
||||||
|
step s1-do-not-replicate-on-activate:
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
step s2-load-metadata-cache:
|
step s2-load-metadata-cache:
|
||||||
COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5';
|
COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5';
|
||||||
|
|
||||||
|
@ -283,10 +294,13 @@ master_remove_node
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
starting permutation: s2-load-metadata-cache s2-begin s2-create-reference-table-2 s1-add-second-worker s2-commit s2-print-content-2
|
starting permutation: s1-do-not-replicate-on-activate s2-load-metadata-cache s2-begin s2-create-reference-table-2 s1-add-second-worker s2-commit s2-print-content-2
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
|
|
||||||
|
|
||||||
|
step s1-do-not-replicate-on-activate:
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
step s2-load-metadata-cache:
|
step s2-load-metadata-cache:
|
||||||
COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5';
|
COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5';
|
||||||
|
|
||||||
|
@ -320,16 +334,18 @@ step s2-print-content-2:
|
||||||
nodeport success result
|
nodeport success result
|
||||||
|
|
||||||
57637 t 1
|
57637 t 1
|
||||||
57638 t 1
|
|
||||||
master_remove_node
|
master_remove_node
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
starting permutation: s1-begin s1-add-second-worker s2-copy-to-reference-table s1-commit s2-print-content
|
starting permutation: s1-do-not-replicate-on-activate s1-begin s1-add-second-worker s2-copy-to-reference-table s1-commit s2-print-content
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
|
|
||||||
|
|
||||||
|
step s1-do-not-replicate-on-activate:
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
step s1-begin:
|
step s1-begin:
|
||||||
BEGIN;
|
BEGIN;
|
||||||
|
|
||||||
|
@ -341,11 +357,10 @@ step s1-add-second-worker:
|
||||||
1
|
1
|
||||||
step s2-copy-to-reference-table:
|
step s2-copy-to-reference-table:
|
||||||
COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5';
|
COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5';
|
||||||
<waiting ...>
|
|
||||||
step s1-commit:
|
step s1-commit:
|
||||||
COMMIT;
|
COMMIT;
|
||||||
|
|
||||||
step s2-copy-to-reference-table: <... completed>
|
|
||||||
step s2-print-content:
|
step s2-print-content:
|
||||||
SELECT
|
SELECT
|
||||||
nodeport, success, result
|
nodeport, success, result
|
||||||
|
@ -357,16 +372,18 @@ step s2-print-content:
|
||||||
nodeport success result
|
nodeport success result
|
||||||
|
|
||||||
57637 t 5
|
57637 t 5
|
||||||
57638 t 5
|
|
||||||
master_remove_node
|
master_remove_node
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
starting permutation: s2-begin s2-copy-to-reference-table s1-add-second-worker s2-commit s2-print-content
|
starting permutation: s1-do-not-replicate-on-activate s2-begin s2-copy-to-reference-table s1-add-second-worker s2-commit s2-print-content
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
|
|
||||||
|
|
||||||
|
step s1-do-not-replicate-on-activate:
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
step s2-begin:
|
step s2-begin:
|
||||||
BEGIN;
|
BEGIN;
|
||||||
|
|
||||||
|
@ -375,14 +392,13 @@ step s2-copy-to-reference-table:
|
||||||
|
|
||||||
step s1-add-second-worker:
|
step s1-add-second-worker:
|
||||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||||
<waiting ...>
|
|
||||||
step s2-commit:
|
|
||||||
COMMIT;
|
|
||||||
|
|
||||||
step s1-add-second-worker: <... completed>
|
|
||||||
?column?
|
?column?
|
||||||
|
|
||||||
1
|
1
|
||||||
|
step s2-commit:
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
step s2-print-content:
|
step s2-print-content:
|
||||||
SELECT
|
SELECT
|
||||||
nodeport, success, result
|
nodeport, success, result
|
||||||
|
@ -394,16 +410,18 @@ step s2-print-content:
|
||||||
nodeport success result
|
nodeport success result
|
||||||
|
|
||||||
57637 t 5
|
57637 t 5
|
||||||
57638 t 5
|
|
||||||
master_remove_node
|
master_remove_node
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
starting permutation: s1-begin s1-add-second-worker s2-insert-to-reference-table s1-commit s2-print-content
|
starting permutation: s1-do-not-replicate-on-activate s1-begin s1-add-second-worker s2-insert-to-reference-table s1-commit s2-print-content
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
|
|
||||||
|
|
||||||
|
step s1-do-not-replicate-on-activate:
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
step s1-begin:
|
step s1-begin:
|
||||||
BEGIN;
|
BEGIN;
|
||||||
|
|
||||||
|
@ -415,11 +433,10 @@ step s1-add-second-worker:
|
||||||
1
|
1
|
||||||
step s2-insert-to-reference-table:
|
step s2-insert-to-reference-table:
|
||||||
INSERT INTO test_reference_table VALUES (6);
|
INSERT INTO test_reference_table VALUES (6);
|
||||||
<waiting ...>
|
|
||||||
step s1-commit:
|
step s1-commit:
|
||||||
COMMIT;
|
COMMIT;
|
||||||
|
|
||||||
step s2-insert-to-reference-table: <... completed>
|
|
||||||
step s2-print-content:
|
step s2-print-content:
|
||||||
SELECT
|
SELECT
|
||||||
nodeport, success, result
|
nodeport, success, result
|
||||||
|
@ -431,16 +448,18 @@ step s2-print-content:
|
||||||
nodeport success result
|
nodeport success result
|
||||||
|
|
||||||
57637 t 1
|
57637 t 1
|
||||||
57638 t 1
|
|
||||||
master_remove_node
|
master_remove_node
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
starting permutation: s2-begin s2-insert-to-reference-table s1-add-second-worker s2-commit s2-print-content
|
starting permutation: s1-do-not-replicate-on-activate s2-begin s2-insert-to-reference-table s1-add-second-worker s2-commit s2-print-content
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
|
|
||||||
|
|
||||||
|
step s1-do-not-replicate-on-activate:
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
step s2-begin:
|
step s2-begin:
|
||||||
BEGIN;
|
BEGIN;
|
||||||
|
|
||||||
|
@ -449,14 +468,13 @@ step s2-insert-to-reference-table:
|
||||||
|
|
||||||
step s1-add-second-worker:
|
step s1-add-second-worker:
|
||||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||||
<waiting ...>
|
|
||||||
step s2-commit:
|
|
||||||
COMMIT;
|
|
||||||
|
|
||||||
step s1-add-second-worker: <... completed>
|
|
||||||
?column?
|
?column?
|
||||||
|
|
||||||
1
|
1
|
||||||
|
step s2-commit:
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
step s2-print-content:
|
step s2-print-content:
|
||||||
SELECT
|
SELECT
|
||||||
nodeport, success, result
|
nodeport, success, result
|
||||||
|
@ -468,16 +486,18 @@ step s2-print-content:
|
||||||
nodeport success result
|
nodeport success result
|
||||||
|
|
||||||
57637 t 1
|
57637 t 1
|
||||||
57638 t 1
|
|
||||||
master_remove_node
|
master_remove_node
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
starting permutation: s1-begin s1-add-second-worker s2-ddl-on-reference-table s1-commit s2-print-index-count
|
starting permutation: s1-do-not-replicate-on-activate s1-begin s1-add-second-worker s2-ddl-on-reference-table s1-commit s2-print-index-count
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
|
|
||||||
|
|
||||||
|
step s1-do-not-replicate-on-activate:
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
step s1-begin:
|
step s1-begin:
|
||||||
BEGIN;
|
BEGIN;
|
||||||
|
|
||||||
|
@ -505,16 +525,18 @@ step s2-print-index-count:
|
||||||
nodeport success result
|
nodeport success result
|
||||||
|
|
||||||
57637 t 1
|
57637 t 1
|
||||||
57638 t 1
|
|
||||||
master_remove_node
|
master_remove_node
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
starting permutation: s2-begin s2-ddl-on-reference-table s1-add-second-worker s2-commit s2-print-index-count
|
starting permutation: s1-do-not-replicate-on-activate s2-begin s2-ddl-on-reference-table s1-add-second-worker s2-commit s2-print-index-count
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
|
|
||||||
|
|
||||||
|
step s1-do-not-replicate-on-activate:
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
step s2-begin:
|
step s2-begin:
|
||||||
BEGIN;
|
BEGIN;
|
||||||
|
|
||||||
|
@ -542,16 +564,18 @@ step s2-print-index-count:
|
||||||
nodeport success result
|
nodeport success result
|
||||||
|
|
||||||
57637 t 1
|
57637 t 1
|
||||||
57638 t 1
|
|
||||||
master_remove_node
|
master_remove_node
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
starting permutation: s1-begin s1-add-second-worker s2-create-reference-table-2 s1-commit s2-print-content-2
|
starting permutation: s1-do-not-replicate-on-activate s1-begin s1-add-second-worker s2-create-reference-table-2 s1-commit s2-print-content-2
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
|
|
||||||
|
|
||||||
|
step s1-do-not-replicate-on-activate:
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
step s1-begin:
|
step s1-begin:
|
||||||
BEGIN;
|
BEGIN;
|
||||||
|
|
||||||
|
@ -588,10 +612,338 @@ master_remove_node
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
starting permutation: s2-begin s2-create-reference-table-2 s1-add-second-worker s2-commit s2-print-content-2
|
starting permutation: s1-do-not-replicate-on-activate s2-begin s2-create-reference-table-2 s1-add-second-worker s2-commit s2-print-content-2
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
|
|
||||||
|
|
||||||
|
step s1-do-not-replicate-on-activate:
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
|
step s2-begin:
|
||||||
|
BEGIN;
|
||||||
|
|
||||||
|
step s2-create-reference-table-2:
|
||||||
|
SELECT create_reference_table('test_reference_table_2');
|
||||||
|
|
||||||
|
create_reference_table
|
||||||
|
|
||||||
|
|
||||||
|
step s1-add-second-worker:
|
||||||
|
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||||
|
<waiting ...>
|
||||||
|
step s2-commit:
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
step s1-add-second-worker: <... completed>
|
||||||
|
?column?
|
||||||
|
|
||||||
|
1
|
||||||
|
step s2-print-content-2:
|
||||||
|
SELECT
|
||||||
|
nodeport, success, result
|
||||||
|
FROM
|
||||||
|
run_command_on_placements('test_reference_table_2', 'select count(*) from %s')
|
||||||
|
ORDER BY
|
||||||
|
nodeport;
|
||||||
|
|
||||||
|
nodeport success result
|
||||||
|
|
||||||
|
57637 t 1
|
||||||
|
master_remove_node
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
starting permutation: s1-replicate-on-activate s1-begin s1-add-second-worker s2-copy-to-reference-table s1-commit s2-print-content
|
||||||
|
create_distributed_table
|
||||||
|
|
||||||
|
|
||||||
|
step s1-replicate-on-activate:
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO on;
|
||||||
|
|
||||||
|
step s1-begin:
|
||||||
|
BEGIN;
|
||||||
|
|
||||||
|
step s1-add-second-worker:
|
||||||
|
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||||
|
|
||||||
|
?column?
|
||||||
|
|
||||||
|
1
|
||||||
|
step s2-copy-to-reference-table:
|
||||||
|
COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5';
|
||||||
|
<waiting ...>
|
||||||
|
step s1-commit:
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
step s2-copy-to-reference-table: <... completed>
|
||||||
|
step s2-print-content:
|
||||||
|
SELECT
|
||||||
|
nodeport, success, result
|
||||||
|
FROM
|
||||||
|
run_command_on_placements('test_reference_table', 'select count(*) from %s')
|
||||||
|
ORDER BY
|
||||||
|
nodeport;
|
||||||
|
|
||||||
|
nodeport success result
|
||||||
|
|
||||||
|
57637 t 5
|
||||||
|
57638 t 5
|
||||||
|
master_remove_node
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
starting permutation: s1-replicate-on-activate s2-begin s2-copy-to-reference-table s1-add-second-worker s2-commit s2-print-content
|
||||||
|
create_distributed_table
|
||||||
|
|
||||||
|
|
||||||
|
step s1-replicate-on-activate:
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO on;
|
||||||
|
|
||||||
|
step s2-begin:
|
||||||
|
BEGIN;
|
||||||
|
|
||||||
|
step s2-copy-to-reference-table:
|
||||||
|
COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5';
|
||||||
|
|
||||||
|
step s1-add-second-worker:
|
||||||
|
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||||
|
<waiting ...>
|
||||||
|
step s2-commit:
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
step s1-add-second-worker: <... completed>
|
||||||
|
?column?
|
||||||
|
|
||||||
|
1
|
||||||
|
step s2-print-content:
|
||||||
|
SELECT
|
||||||
|
nodeport, success, result
|
||||||
|
FROM
|
||||||
|
run_command_on_placements('test_reference_table', 'select count(*) from %s')
|
||||||
|
ORDER BY
|
||||||
|
nodeport;
|
||||||
|
|
||||||
|
nodeport success result
|
||||||
|
|
||||||
|
57637 t 5
|
||||||
|
57638 t 5
|
||||||
|
master_remove_node
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
starting permutation: s1-replicate-on-activate s1-begin s1-add-second-worker s2-insert-to-reference-table s1-commit s2-print-content
|
||||||
|
create_distributed_table
|
||||||
|
|
||||||
|
|
||||||
|
step s1-replicate-on-activate:
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO on;
|
||||||
|
|
||||||
|
step s1-begin:
|
||||||
|
BEGIN;
|
||||||
|
|
||||||
|
step s1-add-second-worker:
|
||||||
|
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||||
|
|
||||||
|
?column?
|
||||||
|
|
||||||
|
1
|
||||||
|
step s2-insert-to-reference-table:
|
||||||
|
INSERT INTO test_reference_table VALUES (6);
|
||||||
|
<waiting ...>
|
||||||
|
step s1-commit:
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
step s2-insert-to-reference-table: <... completed>
|
||||||
|
step s2-print-content:
|
||||||
|
SELECT
|
||||||
|
nodeport, success, result
|
||||||
|
FROM
|
||||||
|
run_command_on_placements('test_reference_table', 'select count(*) from %s')
|
||||||
|
ORDER BY
|
||||||
|
nodeport;
|
||||||
|
|
||||||
|
nodeport success result
|
||||||
|
|
||||||
|
57637 t 1
|
||||||
|
57638 t 1
|
||||||
|
master_remove_node
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
starting permutation: s1-replicate-on-activate s2-begin s2-insert-to-reference-table s1-add-second-worker s2-commit s2-print-content
|
||||||
|
create_distributed_table
|
||||||
|
|
||||||
|
|
||||||
|
step s1-replicate-on-activate:
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO on;
|
||||||
|
|
||||||
|
step s2-begin:
|
||||||
|
BEGIN;
|
||||||
|
|
||||||
|
step s2-insert-to-reference-table:
|
||||||
|
INSERT INTO test_reference_table VALUES (6);
|
||||||
|
|
||||||
|
step s1-add-second-worker:
|
||||||
|
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||||
|
<waiting ...>
|
||||||
|
step s2-commit:
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
step s1-add-second-worker: <... completed>
|
||||||
|
?column?
|
||||||
|
|
||||||
|
1
|
||||||
|
step s2-print-content:
|
||||||
|
SELECT
|
||||||
|
nodeport, success, result
|
||||||
|
FROM
|
||||||
|
run_command_on_placements('test_reference_table', 'select count(*) from %s')
|
||||||
|
ORDER BY
|
||||||
|
nodeport;
|
||||||
|
|
||||||
|
nodeport success result
|
||||||
|
|
||||||
|
57637 t 1
|
||||||
|
57638 t 1
|
||||||
|
master_remove_node
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
starting permutation: s1-replicate-on-activate s1-begin s1-add-second-worker s2-ddl-on-reference-table s1-commit s2-print-index-count
|
||||||
|
create_distributed_table
|
||||||
|
|
||||||
|
|
||||||
|
step s1-replicate-on-activate:
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO on;
|
||||||
|
|
||||||
|
step s1-begin:
|
||||||
|
BEGIN;
|
||||||
|
|
||||||
|
step s1-add-second-worker:
|
||||||
|
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||||
|
|
||||||
|
?column?
|
||||||
|
|
||||||
|
1
|
||||||
|
step s2-ddl-on-reference-table:
|
||||||
|
CREATE INDEX reference_index ON test_reference_table(test_id);
|
||||||
|
<waiting ...>
|
||||||
|
step s1-commit:
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
step s2-ddl-on-reference-table: <... completed>
|
||||||
|
step s2-print-index-count:
|
||||||
|
SELECT
|
||||||
|
nodeport, success, result
|
||||||
|
FROM
|
||||||
|
run_command_on_placements('test_reference_table', 'select count(*) from pg_indexes WHERE tablename = ''%s''')
|
||||||
|
ORDER BY
|
||||||
|
nodeport;
|
||||||
|
|
||||||
|
nodeport success result
|
||||||
|
|
||||||
|
57637 t 1
|
||||||
|
57638 t 1
|
||||||
|
master_remove_node
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
starting permutation: s1-replicate-on-activate s2-begin s2-ddl-on-reference-table s1-add-second-worker s2-commit s2-print-index-count
|
||||||
|
create_distributed_table
|
||||||
|
|
||||||
|
|
||||||
|
step s1-replicate-on-activate:
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO on;
|
||||||
|
|
||||||
|
step s2-begin:
|
||||||
|
BEGIN;
|
||||||
|
|
||||||
|
step s2-ddl-on-reference-table:
|
||||||
|
CREATE INDEX reference_index ON test_reference_table(test_id);
|
||||||
|
|
||||||
|
step s1-add-second-worker:
|
||||||
|
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||||
|
<waiting ...>
|
||||||
|
step s2-commit:
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
step s1-add-second-worker: <... completed>
|
||||||
|
?column?
|
||||||
|
|
||||||
|
1
|
||||||
|
step s2-print-index-count:
|
||||||
|
SELECT
|
||||||
|
nodeport, success, result
|
||||||
|
FROM
|
||||||
|
run_command_on_placements('test_reference_table', 'select count(*) from pg_indexes WHERE tablename = ''%s''')
|
||||||
|
ORDER BY
|
||||||
|
nodeport;
|
||||||
|
|
||||||
|
nodeport success result
|
||||||
|
|
||||||
|
57637 t 1
|
||||||
|
57638 t 1
|
||||||
|
master_remove_node
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
starting permutation: s1-replicate-on-activate s1-begin s1-add-second-worker s2-create-reference-table-2 s1-commit s2-print-content-2
|
||||||
|
create_distributed_table
|
||||||
|
|
||||||
|
|
||||||
|
step s1-replicate-on-activate:
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO on;
|
||||||
|
|
||||||
|
step s1-begin:
|
||||||
|
BEGIN;
|
||||||
|
|
||||||
|
step s1-add-second-worker:
|
||||||
|
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||||
|
|
||||||
|
?column?
|
||||||
|
|
||||||
|
1
|
||||||
|
step s2-create-reference-table-2:
|
||||||
|
SELECT create_reference_table('test_reference_table_2');
|
||||||
|
<waiting ...>
|
||||||
|
step s1-commit:
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
step s2-create-reference-table-2: <... completed>
|
||||||
|
create_reference_table
|
||||||
|
|
||||||
|
|
||||||
|
step s2-print-content-2:
|
||||||
|
SELECT
|
||||||
|
nodeport, success, result
|
||||||
|
FROM
|
||||||
|
run_command_on_placements('test_reference_table_2', 'select count(*) from %s')
|
||||||
|
ORDER BY
|
||||||
|
nodeport;
|
||||||
|
|
||||||
|
nodeport success result
|
||||||
|
|
||||||
|
57637 t 1
|
||||||
|
57638 t 1
|
||||||
|
master_remove_node
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
starting permutation: s1-replicate-on-activate s2-begin s2-create-reference-table-2 s1-add-second-worker s2-commit s2-print-content-2
|
||||||
|
create_distributed_table
|
||||||
|
|
||||||
|
|
||||||
|
step s1-replicate-on-activate:
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO on;
|
||||||
|
|
||||||
step s2-begin:
|
step s2-begin:
|
||||||
BEGIN;
|
BEGIN;
|
||||||
|
|
||||||
|
@ -627,3 +979,45 @@ master_remove_node
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
starting permutation: s1-do-not-replicate-on-activate s1-add-second-worker s2-begin s1-begin s1-drop-reference-table s2-replicate-reference-tables s1-commit s2-commit
|
||||||
|
create_distributed_table
|
||||||
|
|
||||||
|
|
||||||
|
step s1-do-not-replicate-on-activate:
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
|
step s1-add-second-worker:
|
||||||
|
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||||
|
|
||||||
|
?column?
|
||||||
|
|
||||||
|
1
|
||||||
|
step s2-begin:
|
||||||
|
BEGIN;
|
||||||
|
|
||||||
|
step s1-begin:
|
||||||
|
BEGIN;
|
||||||
|
|
||||||
|
step s1-drop-reference-table:
|
||||||
|
DROP TABLE test_reference_table;
|
||||||
|
|
||||||
|
step s2-replicate-reference-tables:
|
||||||
|
SET client_min_messages TO DEBUG2;
|
||||||
|
SELECT replicate_reference_tables();
|
||||||
|
RESET client_min_messages;
|
||||||
|
<waiting ...>
|
||||||
|
step s1-commit:
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
step s2-replicate-reference-tables: <... completed>
|
||||||
|
replicate_reference_tables
|
||||||
|
|
||||||
|
|
||||||
|
step s2-commit:
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
master_remove_node
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -480,9 +480,7 @@ step s2-commit:
|
||||||
COMMIT;
|
COMMIT;
|
||||||
|
|
||||||
step s1-remove-node-2: <... completed>
|
step s1-remove-node-2: <... completed>
|
||||||
master_remove_node
|
error in steps s2-commit s1-remove-node-2: ERROR: you cannot remove the primary node of a node group which has shard placements
|
||||||
|
|
||||||
|
|
||||||
step s2-select:
|
step s2-select:
|
||||||
SELECT * FROM dist_table;
|
SELECT * FROM dist_table;
|
||||||
|
|
||||||
|
@ -491,3 +489,4 @@ x y
|
||||||
master_remove_node
|
master_remove_node
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -28,11 +28,11 @@ step detector-dump-wait-edges:
|
||||||
|
|
||||||
waiting_transaction_numblocking_transaction_numblocking_transaction_waiting
|
waiting_transaction_numblocking_transaction_numblocking_transaction_waiting
|
||||||
|
|
||||||
357 356 f
|
392 391 f
|
||||||
transactionnumberwaitingtransactionnumbers
|
transactionnumberwaitingtransactionnumbers
|
||||||
|
|
||||||
356
|
391
|
||||||
357 356
|
392 391
|
||||||
step s1-abort:
|
step s1-abort:
|
||||||
ABORT;
|
ABORT;
|
||||||
|
|
||||||
|
@ -75,14 +75,14 @@ step detector-dump-wait-edges:
|
||||||
|
|
||||||
waiting_transaction_numblocking_transaction_numblocking_transaction_waiting
|
waiting_transaction_numblocking_transaction_numblocking_transaction_waiting
|
||||||
|
|
||||||
361 360 f
|
396 395 f
|
||||||
362 360 f
|
397 395 f
|
||||||
362 361 t
|
397 396 t
|
||||||
transactionnumberwaitingtransactionnumbers
|
transactionnumberwaitingtransactionnumbers
|
||||||
|
|
||||||
360
|
395
|
||||||
361 360
|
396 395
|
||||||
362 360,361
|
397 395,396
|
||||||
step s1-abort:
|
step s1-abort:
|
||||||
ABORT;
|
ABORT;
|
||||||
|
|
||||||
|
|
|
@ -1013,7 +1013,7 @@ master_remove_node
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
starting permutation: s1-print-distributed-objects s1-begin s2-begin s3-begin s1-add-worker s2-create-schema s3-create-schema2 s2-create-table s3-create-table s1-commit s2-commit s3-commit s2-print-distributed-objects
|
starting permutation: s1-print-distributed-objects s1-begin s2-begin s3-begin s1-add-worker s2-create-schema s3-create-schema2 s2-create-table s3-create-table s1-commit s3-commit s2-commit s2-print-distributed-objects
|
||||||
?column?
|
?column?
|
||||||
|
|
||||||
1
|
1
|
||||||
|
@ -1109,10 +1109,10 @@ step s3-create-table: <... completed>
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
|
|
||||||
|
|
||||||
step s2-commit:
|
step s3-commit:
|
||||||
COMMIT;
|
COMMIT;
|
||||||
|
|
||||||
step s3-commit:
|
step s2-commit:
|
||||||
COMMIT;
|
COMMIT;
|
||||||
|
|
||||||
step s2-print-distributed-objects:
|
step s2-print-distributed-objects:
|
||||||
|
|
|
@ -2,6 +2,7 @@ CREATE SCHEMA local_shard_copy;
|
||||||
SET search_path TO local_shard_copy;
|
SET search_path TO local_shard_copy;
|
||||||
SET client_min_messages TO DEBUG;
|
SET client_min_messages TO DEBUG;
|
||||||
SET citus.next_shard_id TO 1570000;
|
SET citus.next_shard_id TO 1570000;
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
SELECT * FROM master_add_node('localhost', :master_port, groupid := 0);
|
SELECT * FROM master_add_node('localhost', :master_port, groupid := 0);
|
||||||
DEBUG: schema "public" already exists, skipping
|
DEBUG: schema "public" already exists, skipping
|
||||||
DETAIL: NOTICE from localhost:xxxxx
|
DETAIL: NOTICE from localhost:xxxxx
|
||||||
|
@ -15,10 +16,6 @@ DEBUG: schema "citus_mx_test_schema_join_2" already exists, skipping
|
||||||
DETAIL: NOTICE from localhost:xxxxx
|
DETAIL: NOTICE from localhost:xxxxx
|
||||||
DEBUG: schema "citus_mx_schema_for_xacts" already exists, skipping
|
DEBUG: schema "citus_mx_schema_for_xacts" already exists, skipping
|
||||||
DETAIL: NOTICE from localhost:xxxxx
|
DETAIL: NOTICE from localhost:xxxxx
|
||||||
NOTICE: Replicating reference table "customer_mx" to the node localhost:xxxxx
|
|
||||||
NOTICE: Replicating reference table "nation_mx" to the node localhost:xxxxx
|
|
||||||
NOTICE: Replicating reference table "part_mx" to the node localhost:xxxxx
|
|
||||||
NOTICE: Replicating reference table "supplier_mx" to the node localhost:xxxxx
|
|
||||||
master_add_node
|
master_add_node
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
32
|
32
|
||||||
|
|
|
@ -4,6 +4,7 @@
|
||||||
-- Tests for metadata snapshot functions, metadata syncing functions and propagation of
|
-- Tests for metadata snapshot functions, metadata syncing functions and propagation of
|
||||||
-- metadata changes to MX tables.
|
-- metadata changes to MX tables.
|
||||||
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1310000;
|
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1310000;
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
SELECT nextval('pg_catalog.pg_dist_placement_placementid_seq') AS last_placement_id
|
SELECT nextval('pg_catalog.pg_dist_placement_placementid_seq') AS last_placement_id
|
||||||
\gset
|
\gset
|
||||||
ALTER SEQUENCE pg_catalog.pg_dist_placement_placementid_seq RESTART 100000;
|
ALTER SEQUENCE pg_catalog.pg_dist_placement_placementid_seq RESTART 100000;
|
||||||
|
|
|
@ -8,6 +8,7 @@ SELECT nextval('pg_catalog.pg_dist_shardid_seq') AS last_shard_id \gset
|
||||||
SET citus.replication_model TO streaming;
|
SET citus.replication_model TO streaming;
|
||||||
SET citus.shard_count TO 8;
|
SET citus.shard_count TO 8;
|
||||||
SET citus.shard_replication_factor TO 1;
|
SET citus.shard_replication_factor TO 1;
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
-- Simulates a readonly node by setting default_transaction_read_only.
|
-- Simulates a readonly node by setting default_transaction_read_only.
|
||||||
CREATE FUNCTION mark_node_readonly(hostname TEXT, port INTEGER, isreadonly BOOLEAN)
|
CREATE FUNCTION mark_node_readonly(hostname TEXT, port INTEGER, isreadonly BOOLEAN)
|
||||||
RETURNS TEXT
|
RETURNS TEXT
|
||||||
|
@ -173,7 +174,6 @@ SELECT nodeid, hasmetadata, metadatasynced FROM pg_dist_node;
|
||||||
-- Test updating a node when another node is in readonly-mode
|
-- Test updating a node when another node is in readonly-mode
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
SELECT master_add_node('localhost', :worker_2_port) AS nodeid_2 \gset
|
SELECT master_add_node('localhost', :worker_2_port) AS nodeid_2 \gset
|
||||||
NOTICE: Replicating reference table "ref_table" to the node localhost:xxxxx
|
|
||||||
SELECT 1 FROM start_metadata_sync_to_node('localhost', :worker_2_port);
|
SELECT 1 FROM start_metadata_sync_to_node('localhost', :worker_2_port);
|
||||||
?column?
|
?column?
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
|
@ -383,7 +383,6 @@ SELECT verify_metadata('localhost', :worker_1_port);
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
SELECT 1 FROM master_activate_node('localhost', :worker_2_port);
|
SELECT 1 FROM master_activate_node('localhost', :worker_2_port);
|
||||||
NOTICE: Replicating reference table "ref_table" to the node localhost:xxxxx
|
|
||||||
?column?
|
?column?
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
1
|
||||||
|
@ -449,7 +448,6 @@ SELECT wait_until_metadata_sync();
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
SELECT 1 FROM master_activate_node('localhost', :worker_2_port);
|
SELECT 1 FROM master_activate_node('localhost', :worker_2_port);
|
||||||
NOTICE: Replicating reference table "ref_table" to the node localhost:xxxxx
|
|
||||||
?column?
|
?column?
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
1
|
||||||
|
@ -511,7 +509,6 @@ SELECT wait_until_metadata_sync();
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
SELECT 1 FROM master_activate_node('localhost', :worker_2_port);
|
SELECT 1 FROM master_activate_node('localhost', :worker_2_port);
|
||||||
NOTICE: Replicating reference table "ref_table" to the node localhost:xxxxx
|
|
||||||
?column?
|
?column?
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
1
|
||||||
|
|
|
@ -3,6 +3,7 @@
|
||||||
--
|
--
|
||||||
-- Tests that check the metadata after master_remove_node.
|
-- Tests that check the metadata after master_remove_node.
|
||||||
SET citus.next_shard_id TO 1380000;
|
SET citus.next_shard_id TO 1380000;
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 1380000;
|
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 1380000;
|
||||||
ALTER SEQUENCE pg_catalog.pg_dist_groupid_seq RESTART 1380000;
|
ALTER SEQUENCE pg_catalog.pg_dist_groupid_seq RESTART 1380000;
|
||||||
ALTER SEQUENCE pg_catalog.pg_dist_node_nodeid_seq RESTART 1380000;
|
ALTER SEQUENCE pg_catalog.pg_dist_node_nodeid_seq RESTART 1380000;
|
||||||
|
@ -216,12 +217,12 @@ WHERE
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
-- remove same node twice
|
-- remove same node twice
|
||||||
SELECT master_remove_node('localhost', :worker_2_port);
|
SELECT master_remove_node('localhost', :worker_2_port);
|
||||||
ERROR: node at "localhost:xxxxx" does not exist
|
ERROR: node at "localhost:xxxxx" does not exist
|
||||||
-- re-add the node for next tests
|
-- re-add the node for next tests
|
||||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
NOTICE: Replicating reference table "remove_node_reference_table" to the node localhost:xxxxx
|
|
||||||
?column?
|
?column?
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
1
|
||||||
|
@ -242,7 +243,6 @@ SELECT master_remove_node('localhost', :worker_2_port);
|
||||||
|
|
||||||
-- re-add the node for the next test
|
-- re-add the node for the next test
|
||||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
NOTICE: Replicating reference table "remove_node_reference_table" to the node localhost:xxxxx
|
|
||||||
?column?
|
?column?
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
1
|
||||||
|
@ -264,8 +264,7 @@ WHERE
|
||||||
nodeport = :worker_2_port;
|
nodeport = :worker_2_port;
|
||||||
shardid | shardstate | shardlength | nodename | nodeport
|
shardid | shardstate | shardlength | nodename | nodeport
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1380000 | 1 | 0 | localhost | 57638
|
(0 rows)
|
||||||
(1 row)
|
|
||||||
|
|
||||||
SELECT *
|
SELECT *
|
||||||
FROM pg_dist_colocation
|
FROM pg_dist_colocation
|
||||||
|
@ -293,8 +292,7 @@ WHERE
|
||||||
nodeport = :worker_2_port;
|
nodeport = :worker_2_port;
|
||||||
shardid | shardstate | shardlength | nodename | nodeport
|
shardid | shardstate | shardlength | nodename | nodeport
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1380000 | 1 | 0 | localhost | 57638
|
(0 rows)
|
||||||
(1 row)
|
|
||||||
|
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
BEGIN;
|
BEGIN;
|
||||||
|
@ -320,8 +318,7 @@ WHERE
|
||||||
nodeport = :worker_2_port;
|
nodeport = :worker_2_port;
|
||||||
shardid | shardstate | shardlength | nodename | nodeport
|
shardid | shardstate | shardlength | nodename | nodeport
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1380000 | 1 | 0 | localhost | 57638
|
(0 rows)
|
||||||
(1 row)
|
|
||||||
|
|
||||||
SELECT *
|
SELECT *
|
||||||
FROM pg_dist_colocation
|
FROM pg_dist_colocation
|
||||||
|
@ -349,8 +346,7 @@ WHERE
|
||||||
nodeport = :worker_2_port;
|
nodeport = :worker_2_port;
|
||||||
shardid | shardstate | shardlength | nodename | nodeport
|
shardid | shardstate | shardlength | nodename | nodeport
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1380000 | 1 | 0 | localhost | 57638
|
(0 rows)
|
||||||
(1 row)
|
|
||||||
|
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
-- remove node in a transaction and COMMIT
|
-- remove node in a transaction and COMMIT
|
||||||
|
@ -369,8 +365,7 @@ WHERE
|
||||||
nodeport = :worker_2_port;
|
nodeport = :worker_2_port;
|
||||||
shardid | shardstate | shardlength | nodename | nodeport
|
shardid | shardstate | shardlength | nodename | nodeport
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1380000 | 1 | 0 | localhost | 57638
|
(0 rows)
|
||||||
(1 row)
|
|
||||||
|
|
||||||
SELECT *
|
SELECT *
|
||||||
FROM pg_dist_colocation
|
FROM pg_dist_colocation
|
||||||
|
@ -398,8 +393,7 @@ WHERE
|
||||||
nodeport = :worker_2_port;
|
nodeport = :worker_2_port;
|
||||||
shardid | shardstate | shardlength | nodename | nodeport
|
shardid | shardstate | shardlength | nodename | nodeport
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1380000 | 1 | 0 | localhost | 57638
|
(0 rows)
|
||||||
(1 row)
|
|
||||||
|
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
BEGIN;
|
BEGIN;
|
||||||
|
@ -456,9 +450,9 @@ WHERE
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
-- re-add the node for next tests
|
-- re-add the node for next tests
|
||||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
NOTICE: Replicating reference table "remove_node_reference_table" to the node localhost:xxxxx
|
|
||||||
?column?
|
?column?
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
1
|
||||||
|
@ -480,8 +474,7 @@ WHERE
|
||||||
nodeport = :worker_2_port;
|
nodeport = :worker_2_port;
|
||||||
shardid | shardstate | shardlength | nodename | nodeport
|
shardid | shardstate | shardlength | nodename | nodeport
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1380000 | 1 | 0 | localhost | 57638
|
(0 rows)
|
||||||
(1 row)
|
|
||||||
|
|
||||||
SELECT *
|
SELECT *
|
||||||
FROM pg_dist_colocation
|
FROM pg_dist_colocation
|
||||||
|
@ -509,8 +502,7 @@ WHERE
|
||||||
nodeport = :worker_2_port;
|
nodeport = :worker_2_port;
|
||||||
shardid | shardstate | shardlength | nodename | nodeport
|
shardid | shardstate | shardlength | nodename | nodeport
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1380000 | 1 | 0 | localhost | 57638
|
(0 rows)
|
||||||
(1 row)
|
|
||||||
|
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
BEGIN;
|
BEGIN;
|
||||||
|
@ -581,9 +573,9 @@ SELECT * FROM remove_node_reference_table;
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
-- re-add the node for next tests
|
-- re-add the node for next tests
|
||||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
NOTICE: Replicating reference table "remove_node_reference_table" to the node localhost:xxxxx
|
|
||||||
?column?
|
?column?
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
1
|
||||||
|
@ -605,8 +597,7 @@ WHERE
|
||||||
nodeport = :worker_2_port;
|
nodeport = :worker_2_port;
|
||||||
shardid | shardstate | shardlength | nodename | nodeport
|
shardid | shardstate | shardlength | nodename | nodeport
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1380000 | 1 | 0 | localhost | 57638
|
(0 rows)
|
||||||
(1 row)
|
|
||||||
|
|
||||||
SELECT *
|
SELECT *
|
||||||
FROM pg_dist_colocation
|
FROM pg_dist_colocation
|
||||||
|
@ -634,8 +625,7 @@ WHERE
|
||||||
nodeport = :worker_2_port;
|
nodeport = :worker_2_port;
|
||||||
shardid | shardstate | shardlength | nodename | nodeport
|
shardid | shardstate | shardlength | nodename | nodeport
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1380000 | 1 | 0 | localhost | 57638
|
(0 rows)
|
||||||
(1 row)
|
|
||||||
|
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
BEGIN;
|
BEGIN;
|
||||||
|
@ -693,6 +683,7 @@ WHERE
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
SET citus.next_shard_id TO 1380001;
|
SET citus.next_shard_id TO 1380001;
|
||||||
-- verify table structure is changed
|
-- verify table structure is changed
|
||||||
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.remove_node_reference_table'::regclass;
|
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.remove_node_reference_table'::regclass;
|
||||||
|
@ -704,7 +695,6 @@ SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.remove_
|
||||||
|
|
||||||
-- re-add the node for next tests
|
-- re-add the node for next tests
|
||||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
NOTICE: Replicating reference table "remove_node_reference_table" to the node localhost:xxxxx
|
|
||||||
?column?
|
?column?
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
1
|
||||||
|
@ -726,8 +716,7 @@ WHERE
|
||||||
nodeport = :worker_2_port;
|
nodeport = :worker_2_port;
|
||||||
shardid | shardstate | shardlength | nodename | nodeport
|
shardid | shardstate | shardlength | nodename | nodeport
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1380000 | 1 | 0 | localhost | 57638
|
(0 rows)
|
||||||
(1 row)
|
|
||||||
|
|
||||||
SELECT *
|
SELECT *
|
||||||
FROM pg_dist_colocation
|
FROM pg_dist_colocation
|
||||||
|
@ -901,10 +890,9 @@ WHERE
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
-- re-add the node for next tests
|
-- re-add the node for next tests
|
||||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
NOTICE: Replicating reference table "remove_node_reference_table" to the node localhost:xxxxx
|
|
||||||
NOTICE: Replicating reference table "table1" to the node localhost:xxxxx
|
|
||||||
?column?
|
?column?
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
1
|
||||||
|
@ -928,9 +916,7 @@ ORDER BY
|
||||||
shardid;
|
shardid;
|
||||||
shardid | shardstate | shardlength | nodename | nodeport
|
shardid | shardstate | shardlength | nodename | nodeport
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1380001 | 1 | 0 | localhost | 57638
|
(0 rows)
|
||||||
1380002 | 1 | 0 | localhost | 57638
|
|
||||||
(2 rows)
|
|
||||||
|
|
||||||
SELECT *
|
SELECT *
|
||||||
FROM pg_dist_colocation
|
FROM pg_dist_colocation
|
||||||
|
@ -959,9 +945,7 @@ WHERE
|
||||||
ORDER BY shardid ASC;
|
ORDER BY shardid ASC;
|
||||||
shardid | shardstate | shardlength | nodename | nodeport
|
shardid | shardstate | shardlength | nodename | nodeport
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1380001 | 1 | 0 | localhost | 57638
|
(0 rows)
|
||||||
1380002 | 1 | 0 | localhost | 57638
|
|
||||||
(2 rows)
|
|
||||||
|
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
SELECT master_disable_node('localhost', :worker_2_port);
|
SELECT master_disable_node('localhost', :worker_2_port);
|
||||||
|
@ -1016,10 +1000,9 @@ WHERE
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
-- re-add the node for next tests
|
-- re-add the node for next tests
|
||||||
SELECT 1 FROM master_activate_node('localhost', :worker_2_port);
|
SELECT 1 FROM master_activate_node('localhost', :worker_2_port);
|
||||||
NOTICE: Replicating reference table "remove_node_reference_table" to the node localhost:xxxxx
|
|
||||||
NOTICE: Replicating reference table "table1" to the node localhost:xxxxx
|
|
||||||
?column?
|
?column?
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
1
|
||||||
|
|
|
@ -2,10 +2,16 @@
|
||||||
-- MULTI_REPLICATE_REFERENCE_TABLE
|
-- MULTI_REPLICATE_REFERENCE_TABLE
|
||||||
--
|
--
|
||||||
-- Tests that check that reference tables are replicated when adding new nodes.
|
-- Tests that check that reference tables are replicated when adding new nodes.
|
||||||
|
CREATE SCHEMA replicate_reference_table;
|
||||||
|
SET search_path TO replicate_reference_table;
|
||||||
SET citus.next_shard_id TO 1370000;
|
SET citus.next_shard_id TO 1370000;
|
||||||
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 1370000;
|
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 1370000;
|
||||||
ALTER SEQUENCE pg_catalog.pg_dist_groupid_seq RESTART 1370000;
|
ALTER SEQUENCE pg_catalog.pg_dist_groupid_seq RESTART 1370000;
|
||||||
ALTER SEQUENCE pg_catalog.pg_dist_node_nodeid_seq RESTART 1370000;
|
ALTER SEQUENCE pg_catalog.pg_dist_node_nodeid_seq RESTART 1370000;
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
-- only query shards created in this test
|
||||||
|
CREATE VIEW pg_dist_shard_placement_view AS
|
||||||
|
SELECT * FROM pg_dist_shard_placement WHERE shardid BETWEEN 1370000 AND 1380000;
|
||||||
-- remove a node for testing purposes
|
-- remove a node for testing purposes
|
||||||
CREATE TABLE tmp_shard_placement AS SELECT * FROM pg_dist_shard_placement WHERE nodeport = :worker_2_port;
|
CREATE TABLE tmp_shard_placement AS SELECT * FROM pg_dist_shard_placement WHERE nodeport = :worker_2_port;
|
||||||
DELETE FROM pg_dist_shard_placement WHERE nodeport = :worker_2_port;
|
DELETE FROM pg_dist_shard_placement WHERE nodeport = :worker_2_port;
|
||||||
|
@ -40,7 +46,7 @@ SELECT COUNT(*) FROM pg_dist_node WHERE nodeport = :worker_2_port;
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port;
|
nodeport = :worker_2_port;
|
||||||
shardid | shardstate | shardlength | nodename | nodeport
|
shardid | shardstate | shardlength | nodename | nodeport
|
||||||
|
@ -70,19 +76,23 @@ SELECT create_reference_table('replicate_reference_table_unhealthy');
|
||||||
|
|
||||||
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE shardid = 1370000;
|
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE shardid = 1370000;
|
||||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
ERROR: could not find any healthy placement for shard xxxxx
|
?column?
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
1
|
||||||
|
(1 row)
|
||||||
|
|
||||||
-- verify node is not added
|
-- verify node is not added
|
||||||
SELECT COUNT(*) FROM pg_dist_node WHERE nodeport = :worker_2_port;
|
SELECT COUNT(*) FROM pg_dist_node WHERE nodeport = :worker_2_port;
|
||||||
count
|
count
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
0
|
1
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
-- verify nothing is replicated to the new node
|
-- verify nothing is replicated to the new node
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port;
|
nodeport = :worker_2_port;
|
||||||
shardid | shardstate | shardlength | nodename | nodeport
|
shardid | shardstate | shardlength | nodename | nodeport
|
||||||
|
@ -102,27 +112,26 @@ SELECT create_reference_table('replicate_reference_table_valid');
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
shardid | shardstate | shardlength | nodename | nodeport
|
shardid | shardstate | shardlength | nodename | nodeport
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
(0 rows)
|
1370001 | 1 | 0 | localhost | 57638
|
||||||
|
(1 row)
|
||||||
|
|
||||||
SELECT *
|
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation
|
||||||
FROM pg_dist_colocation
|
|
||||||
WHERE colocationid IN
|
WHERE colocationid IN
|
||||||
(SELECT colocationid
|
(SELECT colocationid
|
||||||
FROM pg_dist_partition
|
FROM pg_dist_partition
|
||||||
WHERE logicalrelid = 'replicate_reference_table_valid'::regclass);
|
WHERE logicalrelid = 'replicate_reference_table_valid'::regclass);
|
||||||
colocationid | shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation
|
shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
10004 | 1 | -1 | 0 | 0
|
1 | -1 | 0 | 0
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
NOTICE: Replicating reference table "replicate_reference_table_valid" to the node localhost:xxxxx
|
|
||||||
?column?
|
?column?
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
1
|
||||||
|
@ -132,7 +141,7 @@ NOTICE: Replicating reference table "replicate_reference_table_valid" to the no
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
|
@ -141,15 +150,14 @@ ORDER BY shardid, nodeport;
|
||||||
1370001 | 1 | 0 | localhost | 57638
|
1370001 | 1 | 0 | localhost | 57638
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
SELECT *
|
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation
|
||||||
FROM pg_dist_colocation
|
|
||||||
WHERE colocationid IN
|
WHERE colocationid IN
|
||||||
(SELECT colocationid
|
(SELECT colocationid
|
||||||
FROM pg_dist_partition
|
FROM pg_dist_partition
|
||||||
WHERE logicalrelid = 'replicate_reference_table_valid'::regclass);
|
WHERE logicalrelid = 'replicate_reference_table_valid'::regclass);
|
||||||
colocationid | shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation
|
shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
10004 | 1 | -1 | 0 | 0
|
1 | -1 | 0 | 0
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
-- test add same node twice
|
-- test add same node twice
|
||||||
|
@ -157,7 +165,7 @@ WHERE colocationid IN
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
|
@ -166,15 +174,14 @@ ORDER BY shardid, nodeport;
|
||||||
1370001 | 1 | 0 | localhost | 57638
|
1370001 | 1 | 0 | localhost | 57638
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
SELECT *
|
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation
|
||||||
FROM pg_dist_colocation
|
|
||||||
WHERE colocationid IN
|
WHERE colocationid IN
|
||||||
(SELECT colocationid
|
(SELECT colocationid
|
||||||
FROM pg_dist_partition
|
FROM pg_dist_partition
|
||||||
WHERE logicalrelid = 'replicate_reference_table_valid'::regclass);
|
WHERE logicalrelid = 'replicate_reference_table_valid'::regclass);
|
||||||
colocationid | shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation
|
shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
10004 | 1 | -1 | 0 | 0
|
1 | -1 | 0 | 0
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
|
@ -187,7 +194,7 @@ SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
|
@ -196,15 +203,14 @@ ORDER BY shardid, nodeport;
|
||||||
1370001 | 1 | 0 | localhost | 57638
|
1370001 | 1 | 0 | localhost | 57638
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
SELECT *
|
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation
|
||||||
FROM pg_dist_colocation
|
|
||||||
WHERE colocationid IN
|
WHERE colocationid IN
|
||||||
(SELECT colocationid
|
(SELECT colocationid
|
||||||
FROM pg_dist_partition
|
FROM pg_dist_partition
|
||||||
WHERE logicalrelid = 'replicate_reference_table_valid'::regclass);
|
WHERE logicalrelid = 'replicate_reference_table_valid'::regclass);
|
||||||
colocationid | shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation
|
shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
10004 | 1 | -1 | 0 | 0
|
1 | -1 | 0 | 0
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
DROP TABLE replicate_reference_table_valid;
|
DROP TABLE replicate_reference_table_valid;
|
||||||
|
@ -226,7 +232,7 @@ SELECT create_reference_table('replicate_reference_table_rollback');
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
|
@ -234,20 +240,18 @@ ORDER BY shardid, nodeport;
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
|
||||||
SELECT *
|
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation
|
||||||
FROM pg_dist_colocation
|
|
||||||
WHERE colocationid IN
|
WHERE colocationid IN
|
||||||
(SELECT colocationid
|
(SELECT colocationid
|
||||||
FROM pg_dist_partition
|
FROM pg_dist_partition
|
||||||
WHERE logicalrelid = 'replicate_reference_table_rollback'::regclass);
|
WHERE logicalrelid = 'replicate_reference_table_rollback'::regclass);
|
||||||
colocationid | shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation
|
shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
10004 | 1 | -1 | 0 | 0
|
1 | -1 | 0 | 0
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
BEGIN;
|
BEGIN;
|
||||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
NOTICE: Replicating reference table "replicate_reference_table_rollback" to the node localhost:xxxxx
|
|
||||||
?column?
|
?column?
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
1
|
||||||
|
@ -258,7 +262,7 @@ ROLLBACK;
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
|
@ -266,15 +270,14 @@ ORDER BY shardid, nodeport;
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
|
||||||
SELECT *
|
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation
|
||||||
FROM pg_dist_colocation
|
|
||||||
WHERE colocationid IN
|
WHERE colocationid IN
|
||||||
(SELECT colocationid
|
(SELECT colocationid
|
||||||
FROM pg_dist_partition
|
FROM pg_dist_partition
|
||||||
WHERE logicalrelid = 'replicate_reference_table_rollback'::regclass);
|
WHERE logicalrelid = 'replicate_reference_table_rollback'::regclass);
|
||||||
colocationid | shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation
|
shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
10004 | 1 | -1 | 0 | 0
|
1 | -1 | 0 | 0
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
DROP TABLE replicate_reference_table_rollback;
|
DROP TABLE replicate_reference_table_rollback;
|
||||||
|
@ -290,7 +293,7 @@ SELECT create_reference_table('replicate_reference_table_commit');
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
|
@ -298,20 +301,18 @@ ORDER BY shardid, nodeport;
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
|
||||||
SELECT *
|
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation
|
||||||
FROM pg_dist_colocation
|
|
||||||
WHERE colocationid IN
|
WHERE colocationid IN
|
||||||
(SELECT colocationid
|
(SELECT colocationid
|
||||||
FROM pg_dist_partition
|
FROM pg_dist_partition
|
||||||
WHERE logicalrelid = 'replicate_reference_table_commit'::regclass);
|
WHERE logicalrelid = 'replicate_reference_table_commit'::regclass);
|
||||||
colocationid | shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation
|
shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
10004 | 1 | -1 | 0 | 0
|
1 | -1 | 0 | 0
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
BEGIN;
|
BEGIN;
|
||||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
NOTICE: Replicating reference table "replicate_reference_table_commit" to the node localhost:xxxxx
|
|
||||||
?column?
|
?column?
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
1
|
||||||
|
@ -322,24 +323,22 @@ COMMIT;
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
shardid | shardstate | shardlength | nodename | nodeport
|
shardid | shardstate | shardlength | nodename | nodeport
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1370003 | 1 | 0 | localhost | 57638
|
(0 rows)
|
||||||
(1 row)
|
|
||||||
|
|
||||||
SELECT *
|
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation
|
||||||
FROM pg_dist_colocation
|
|
||||||
WHERE colocationid IN
|
WHERE colocationid IN
|
||||||
(SELECT colocationid
|
(SELECT colocationid
|
||||||
FROM pg_dist_partition
|
FROM pg_dist_partition
|
||||||
WHERE logicalrelid = 'replicate_reference_table_commit'::regclass);
|
WHERE logicalrelid = 'replicate_reference_table_commit'::regclass);
|
||||||
colocationid | shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation
|
shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
10004 | 1 | -1 | 0 | 0
|
1 | -1 | 0 | 0
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
DROP TABLE replicate_reference_table_commit;
|
DROP TABLE replicate_reference_table_commit;
|
||||||
|
@ -374,22 +373,21 @@ CREATE TABLE replicate_reference_table_reference_two(column1 int);
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port;
|
nodeport = :worker_2_port;
|
||||||
shardid | shardstate | shardlength | nodename | nodeport
|
shardid | shardstate | shardlength | nodename | nodeport
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
|
||||||
SELECT *
|
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation
|
||||||
FROM pg_dist_colocation
|
|
||||||
WHERE colocationid IN
|
WHERE colocationid IN
|
||||||
(SELECT colocationid
|
(SELECT colocationid
|
||||||
FROM pg_dist_partition
|
FROM pg_dist_partition
|
||||||
WHERE logicalrelid = 'replicate_reference_table_reference_one'::regclass);
|
WHERE logicalrelid = 'replicate_reference_table_reference_one'::regclass);
|
||||||
colocationid | shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation
|
shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
10004 | 1 | -1 | 0 | 0
|
1 | -1 | 0 | 0
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
SELECT colocationid AS reference_table_colocationid FROM pg_dist_colocation WHERE distributioncolumntype = 0 \gset
|
SELECT colocationid AS reference_table_colocationid FROM pg_dist_colocation WHERE distributioncolumntype = 0 \gset
|
||||||
|
@ -406,8 +404,7 @@ ORDER BY logicalrelid;
|
||||||
replicate_reference_table_hash | h | f | c
|
replicate_reference_table_hash | h | f | c
|
||||||
(2 rows)
|
(2 rows)
|
||||||
|
|
||||||
BEGIN;
|
SET client_min_messages TO WARNING;
|
||||||
SET LOCAL client_min_messages TO ERROR;
|
|
||||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
?column?
|
?column?
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
|
@ -426,12 +423,12 @@ SELECT create_reference_table('replicate_reference_table_reference_two');
|
||||||
|
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
COMMIT;
|
RESET client_min_messages;
|
||||||
-- status after master_add_node
|
-- status after master_add_node
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
|
@ -442,15 +439,14 @@ ORDER BY shardid, nodeport;
|
||||||
1370006 | 1 | 0 | localhost | 57638
|
1370006 | 1 | 0 | localhost | 57638
|
||||||
(3 rows)
|
(3 rows)
|
||||||
|
|
||||||
SELECT *
|
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation
|
||||||
FROM pg_dist_colocation
|
|
||||||
WHERE colocationid IN
|
WHERE colocationid IN
|
||||||
(SELECT colocationid
|
(SELECT colocationid
|
||||||
FROM pg_dist_partition
|
FROM pg_dist_partition
|
||||||
WHERE logicalrelid = 'replicate_reference_table_reference_one'::regclass);
|
WHERE logicalrelid = 'replicate_reference_table_reference_one'::regclass);
|
||||||
colocationid | shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation
|
shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
10004 | 1 | -1 | 0 | 0
|
1 | -1 | 0 | 0
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
SELECT
|
SELECT
|
||||||
|
@ -533,7 +529,7 @@ SELECT create_reference_table('replicate_reference_table_drop');
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
|
@ -541,20 +537,18 @@ ORDER BY shardid, nodeport;
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
|
||||||
SELECT *
|
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation
|
||||||
FROM pg_dist_colocation
|
|
||||||
WHERE colocationid IN
|
WHERE colocationid IN
|
||||||
(SELECT colocationid
|
(SELECT colocationid
|
||||||
FROM pg_dist_partition
|
FROM pg_dist_partition
|
||||||
WHERE logicalrelid = 'replicate_reference_table_drop'::regclass);
|
WHERE logicalrelid = 'replicate_reference_table_drop'::regclass);
|
||||||
colocationid | shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation
|
shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
10004 | 1 | -1 | 0 | 0
|
1 | -1 | 0 | 0
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
BEGIN;
|
BEGIN;
|
||||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
NOTICE: Replicating reference table "replicate_reference_table_drop" to the node localhost:xxxxx
|
|
||||||
?column?
|
?column?
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
1
|
||||||
|
@ -566,7 +560,7 @@ COMMIT;
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
|
@ -598,7 +592,7 @@ SELECT create_reference_table('replicate_reference_table_schema.table1');
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
|
@ -606,19 +600,17 @@ ORDER BY shardid, nodeport;
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
|
||||||
SELECT *
|
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation
|
||||||
FROM pg_dist_colocation
|
|
||||||
WHERE colocationid IN
|
WHERE colocationid IN
|
||||||
(SELECT colocationid
|
(SELECT colocationid
|
||||||
FROM pg_dist_partition
|
FROM pg_dist_partition
|
||||||
WHERE logicalrelid = 'replicate_reference_table_schema.table1'::regclass);
|
WHERE logicalrelid = 'replicate_reference_table_schema.table1'::regclass);
|
||||||
colocationid | shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation
|
shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
10004 | 1 | -1 | 0 | 0
|
1 | -1 | 0 | 0
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
NOTICE: Replicating reference table "table1" to the node localhost:xxxxx
|
|
||||||
?column?
|
?column?
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
1
|
||||||
|
@ -628,24 +620,22 @@ NOTICE: Replicating reference table "table1" to the node localhost:xxxxx
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
shardid | shardstate | shardlength | nodename | nodeport
|
shardid | shardstate | shardlength | nodename | nodeport
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1370011 | 1 | 0 | localhost | 57638
|
(0 rows)
|
||||||
(1 row)
|
|
||||||
|
|
||||||
SELECT *
|
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation
|
||||||
FROM pg_dist_colocation
|
|
||||||
WHERE colocationid IN
|
WHERE colocationid IN
|
||||||
(SELECT colocationid
|
(SELECT colocationid
|
||||||
FROM pg_dist_partition
|
FROM pg_dist_partition
|
||||||
WHERE logicalrelid = 'replicate_reference_table_schema.table1'::regclass);
|
WHERE logicalrelid = 'replicate_reference_table_schema.table1'::regclass);
|
||||||
colocationid | shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation
|
shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
10004 | 1 | -1 | 0 | 0
|
1 | -1 | 0 | 0
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
DROP TABLE replicate_reference_table_schema.table1;
|
DROP TABLE replicate_reference_table_schema.table1;
|
||||||
|
@ -672,7 +662,7 @@ SELECT create_reference_table('ref_table_1'),
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
|
@ -681,9 +671,6 @@ ORDER BY shardid, nodeport;
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
|
||||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
NOTICE: Replicating reference table "ref_table_1" to the node localhost:xxxxx
|
|
||||||
NOTICE: Replicating reference table "ref_table_2" to the node localhost:xxxxx
|
|
||||||
NOTICE: Replicating reference table "ref_table_3" to the node localhost:xxxxx
|
|
||||||
?column?
|
?column?
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
1
|
||||||
|
@ -693,23 +680,20 @@ NOTICE: Replicating reference table "ref_table_3" to the node localhost:xxxxx
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
shardid | shardstate | shardlength | nodename | nodeport
|
shardid | shardstate | shardlength | nodename | nodeport
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1370012 | 1 | 0 | localhost | 57638
|
(0 rows)
|
||||||
1370013 | 1 | 0 | localhost | 57638
|
|
||||||
1370014 | 1 | 0 | localhost | 57638
|
|
||||||
(3 rows)
|
|
||||||
|
|
||||||
-- verify constraints have been created on the new node
|
-- verify constraints have been created on the new node
|
||||||
SELECT run_command_on_workers('select count(*) from pg_constraint where contype=''f'' AND conname like ''ref_table%'';');
|
SELECT run_command_on_workers('select count(*) from pg_constraint where contype=''f'' AND conname like ''ref_table%'';');
|
||||||
run_command_on_workers
|
run_command_on_workers
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
(localhost,57637,t,2)
|
(localhost,57637,t,2)
|
||||||
(localhost,57638,t,2)
|
(localhost,57638,t,0)
|
||||||
(2 rows)
|
(2 rows)
|
||||||
|
|
||||||
DROP TABLE ref_table_1, ref_table_2, ref_table_3;
|
DROP TABLE ref_table_1, ref_table_2, ref_table_3;
|
||||||
|
@ -737,7 +721,7 @@ SELECT 1 FROM master_add_inactive_node('localhost', :worker_2_port);
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
shardid IN (SELECT
|
shardid IN (SELECT
|
||||||
shardid
|
shardid
|
||||||
|
@ -754,7 +738,6 @@ ORDER BY 1,4,5;
|
||||||
|
|
||||||
-- we should see the two shard placements after activation
|
-- we should see the two shard placements after activation
|
||||||
SELECT 1 FROM master_activate_node('localhost', :worker_2_port);
|
SELECT 1 FROM master_activate_node('localhost', :worker_2_port);
|
||||||
NOTICE: Replicating reference table "initially_not_replicated_reference_table" to the node localhost:xxxxx
|
|
||||||
?column?
|
?column?
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
1
|
||||||
|
@ -763,7 +746,7 @@ NOTICE: Replicating reference table "initially_not_replicated_reference_table"
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
shardid IN (SELECT
|
shardid IN (SELECT
|
||||||
shardid
|
shardid
|
||||||
|
@ -776,16 +759,161 @@ ORDER BY 1,4,5;
|
||||||
shardid | shardstate | shardlength | nodename | nodeport
|
shardid | shardstate | shardlength | nodename | nodeport
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1370015 | 1 | 0 | localhost | 57637
|
1370015 | 1 | 0 | localhost | 57637
|
||||||
1370015 | 1 | 0 | localhost | 57638
|
(1 row)
|
||||||
(2 rows)
|
|
||||||
|
|
||||||
-- this should have no effect
|
SELECT 1 FROM master_remove_node('localhost', :worker_2_port);
|
||||||
|
?column?
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
1
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
CREATE TABLE ref_table(a int);
|
||||||
|
SELECT create_reference_table('ref_table');
|
||||||
|
create_reference_table
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
INSERT INTO ref_table SELECT * FROM generate_series(1, 10);
|
||||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
?column?
|
?column?
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
1
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
-- verify we cannot replicate reference tables in a transaction modifying pg_dist_node
|
||||||
|
BEGIN;
|
||||||
|
SELECT 1 FROM master_add_inactive_node('invalid-node-name', 9999);
|
||||||
|
?column?
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
1
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT replicate_reference_tables();
|
||||||
|
ERROR: cannot replicate reference tables in a transaction that modified node metadata
|
||||||
|
ROLLBACK;
|
||||||
|
-- verify we cannot replicate reference tables in a transaction which
|
||||||
|
-- modified reference tables
|
||||||
|
BEGIN;
|
||||||
|
DELETE FROM ref_table;
|
||||||
|
SELECT replicate_reference_tables();
|
||||||
|
ERROR: cannot replicate reference tables in a transaction that modified a reference table
|
||||||
|
ROLLBACK;
|
||||||
|
BEGIN;
|
||||||
|
ALTER TABLE ref_table ADD COLUMN b int;
|
||||||
|
SELECT replicate_reference_tables();
|
||||||
|
ERROR: cannot replicate reference tables in a transaction that modified a reference table
|
||||||
|
ROLLBACK;
|
||||||
|
BEGIN;
|
||||||
|
CREATE INDEX ref_idx ON ref_table(a);
|
||||||
|
SELECT replicate_reference_tables();
|
||||||
|
ERROR: cannot replicate reference tables in a transaction that modified a reference table
|
||||||
|
ROLLBACK;
|
||||||
|
--
|
||||||
|
-- read from reference table, then replicate, then write. verify
|
||||||
|
-- placements are consistent.
|
||||||
|
--
|
||||||
|
BEGIN;
|
||||||
|
SELECT count(*) FROM ref_table;
|
||||||
|
count
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
10
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT replicate_reference_tables();
|
||||||
|
replicate_reference_tables
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
INSERT INTO ref_table VALUES (11);
|
||||||
|
SELECT count(*), sum(a) FROM ref_table;
|
||||||
|
count | sum
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
11 | 66
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
UPDATE ref_table SET a = a + 1;
|
||||||
|
SELECT sum(a) FROM ref_table;
|
||||||
|
sum
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
77
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
COMMIT;
|
||||||
|
SELECT min(result) = max(result) AS consistent FROM run_command_on_placements('ref_table', 'SELECT sum(a) FROM %s');
|
||||||
|
consistent
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
t
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET client_min_messages TO WARNING;
|
||||||
|
SELECT count(*) AS ref_table_placements FROM pg_dist_shard NATURAL JOIN pg_dist_shard_placement WHERE logicalrelid = 'ref_table'::regclass \gset
|
||||||
|
-- remove reference table replica from worker 2
|
||||||
|
SELECT 1 FROM master_remove_node('localhost', :worker_2_port);
|
||||||
|
?column?
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
1
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT count(*) - :ref_table_placements FROM pg_dist_shard NATURAL JOIN pg_dist_shard_placement WHERE logicalrelid = 'ref_table'::regclass;
|
||||||
|
?column?
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
-1
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- test setting citus.replicate_reference_tables_on_activate to on
|
||||||
|
-- master_add_node
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO on;
|
||||||
|
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
|
?column?
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
1
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT count(*) - :ref_table_placements FROM pg_dist_shard NATURAL JOIN pg_dist_shard_placement WHERE logicalrelid = 'ref_table'::regclass;
|
||||||
|
?column?
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
0
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- master_activate_node
|
||||||
|
SELECT 1 FROM master_remove_node('localhost', :worker_2_port);
|
||||||
|
?column?
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
1
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT 1 FROM master_add_inactive_node('localhost', :worker_2_port);
|
||||||
|
?column?
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
1
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT count(*) - :ref_table_placements FROM pg_dist_shard NATURAL JOIN pg_dist_shard_placement WHERE logicalrelid = 'ref_table'::regclass;
|
||||||
|
?column?
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
-1
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT 1 FROM master_activate_node('localhost', :worker_2_port);
|
||||||
|
?column?
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
1
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT count(*) - :ref_table_placements FROM pg_dist_shard NATURAL JOIN pg_dist_shard_placement WHERE logicalrelid = 'ref_table'::regclass;
|
||||||
|
?column?
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
0
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT min(result) = max(result) AS consistent FROM run_command_on_placements('ref_table', 'SELECT sum(a) FROM %s');
|
||||||
|
consistent
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
t
|
||||||
|
(1 row)
|
||||||
|
|
||||||
-- test adding an invalid node while we have reference tables to replicate
|
-- test adding an invalid node while we have reference tables to replicate
|
||||||
-- set client message level to ERROR and verbosity to terse to supporess
|
-- set client message level to ERROR and verbosity to terse to supporess
|
||||||
-- OS-dependent host name resolution warnings
|
-- OS-dependent host name resolution warnings
|
||||||
|
@ -793,12 +921,9 @@ SET client_min_messages to ERROR;
|
||||||
\set VERBOSITY terse
|
\set VERBOSITY terse
|
||||||
SELECT master_add_node('invalid-node-name', 9999);
|
SELECT master_add_node('invalid-node-name', 9999);
|
||||||
ERROR: connection error: invalid-node-name:9999
|
ERROR: connection error: invalid-node-name:9999
|
||||||
SET client_min_messages to DEFAULT;
|
|
||||||
\set VERBOSITY default
|
|
||||||
-- drop unnecassary tables
|
-- drop unnecassary tables
|
||||||
DROP TABLE initially_not_replicated_reference_table;
|
DROP TABLE initially_not_replicated_reference_table;
|
||||||
-- reload pg_dist_shard_placement table
|
-- reload pg_dist_shard_placement table
|
||||||
INSERT INTO pg_dist_shard_placement (SELECT * FROM tmp_shard_placement);
|
INSERT INTO pg_dist_shard_placement (SELECT * FROM tmp_shard_placement);
|
||||||
DROP TABLE tmp_shard_placement;
|
DROP TABLE tmp_shard_placement;
|
||||||
RESET citus.shard_replication_factor;
|
DROP SCHEMA replicate_reference_table CASCADE;
|
||||||
RESET citus.replication_model;
|
|
||||||
|
|
|
@ -4,6 +4,7 @@
|
||||||
-- Tests that check the metadata returned by the master node.
|
-- Tests that check the metadata returned by the master node.
|
||||||
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1410000;
|
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1410000;
|
||||||
SET citus.shard_count TO 4;
|
SET citus.shard_count TO 4;
|
||||||
|
SET client_min_messages TO WARNING;
|
||||||
-- test DROP TABLE(ergo master_drop_all_shards) in transaction, then ROLLBACK
|
-- test DROP TABLE(ergo master_drop_all_shards) in transaction, then ROLLBACK
|
||||||
CREATE TABLE transactional_drop_shards(column1 int);
|
CREATE TABLE transactional_drop_shards(column1 int);
|
||||||
SELECT create_distributed_table('transactional_drop_shards', 'column1');
|
SELECT create_distributed_table('transactional_drop_shards', 'column1');
|
||||||
|
@ -653,9 +654,9 @@ ORDER BY
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
|
SET client_min_messages TO WARNING;
|
||||||
-- try using the coordinator as a worker and then dropping the table
|
-- try using the coordinator as a worker and then dropping the table
|
||||||
SELECT 1 FROM master_add_node('localhost', :master_port);
|
SELECT 1 FROM master_add_node('localhost', :master_port);
|
||||||
NOTICE: Replicating reference table "transactional_drop_reference" to the node localhost:xxxxx
|
|
||||||
?column?
|
?column?
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
1
|
||||||
|
@ -686,8 +687,6 @@ SELECT stop_metadata_sync_to_node('localhost', :worker_1_port);
|
||||||
|
|
||||||
-- test DROP TABLE as a non-superuser in a transaction block
|
-- test DROP TABLE as a non-superuser in a transaction block
|
||||||
CREATE USER try_drop_table WITH LOGIN;
|
CREATE USER try_drop_table WITH LOGIN;
|
||||||
NOTICE: not propagating CREATE ROLE/USER commands to worker nodes
|
|
||||||
HINT: Connect to worker nodes directly to manually create all necessary users and roles.
|
|
||||||
SELECT run_command_on_workers('CREATE USER try_drop_table WITH LOGIN');
|
SELECT run_command_on_workers('CREATE USER try_drop_table WITH LOGIN');
|
||||||
run_command_on_workers
|
run_command_on_workers
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
|
|
|
@ -1,6 +1,7 @@
|
||||||
CREATE SCHEMA "extension'test";
|
CREATE SCHEMA "extension'test";
|
||||||
-- use a schema name with escape character
|
-- use a schema name with escape character
|
||||||
SET search_path TO "extension'test";
|
SET search_path TO "extension'test";
|
||||||
|
SET client_min_messages TO WARNING;
|
||||||
-- create an extension on the given search_path
|
-- create an extension on the given search_path
|
||||||
-- the extension is on contrib, so should be avaliable for the regression tests
|
-- the extension is on contrib, so should be avaliable for the regression tests
|
||||||
CREATE EXTENSION seg;
|
CREATE EXTENSION seg;
|
||||||
|
@ -88,10 +89,8 @@ SELECT create_reference_table('ref_table');
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
-- now, drop the extension, recreate it with an older version and update it to latest version
|
-- now, drop the extension, recreate it with an older version and update it to latest version
|
||||||
SET client_min_messages TO WARNING;
|
|
||||||
DROP EXTENSION isn CASCADE;
|
DROP EXTENSION isn CASCADE;
|
||||||
CREATE EXTENSION isn WITH VERSION "1.1";
|
CREATE EXTENSION isn WITH VERSION "1.1";
|
||||||
RESET client_min_messages;
|
|
||||||
-- before updating the version, ensure the current version
|
-- before updating the version, ensure the current version
|
||||||
SELECT run_command_on_workers($$SELECT extversion FROM pg_extension WHERE extname = 'isn'$$);
|
SELECT run_command_on_workers($$SELECT extversion FROM pg_extension WHERE extname = 'isn'$$);
|
||||||
run_command_on_workers
|
run_command_on_workers
|
||||||
|
@ -137,22 +136,16 @@ SELECT run_command_on_workers($$SELECT nspname from pg_namespace where oid=(SELE
|
||||||
(localhost,57638,t,public)
|
(localhost,57638,t,public)
|
||||||
(2 rows)
|
(2 rows)
|
||||||
|
|
||||||
-- SET client_min_messages TO WARNING before executing a DROP EXTENSION statement
|
|
||||||
SET client_min_messages TO WARNING;
|
|
||||||
-- drop the extension finally
|
-- drop the extension finally
|
||||||
DROP EXTENSION isn CASCADE;
|
DROP EXTENSION isn CASCADE;
|
||||||
-- restore client_min_messages after DROP EXTENSION
|
|
||||||
RESET client_min_messages;
|
|
||||||
-- now make sure that the reference tables depending on an extension can be succesfully created.
|
-- now make sure that the reference tables depending on an extension can be succesfully created.
|
||||||
-- we should also ensure that we replicate this reference table (and hence the extension)
|
-- we should also ensure that we replicate this reference table (and hence the extension)
|
||||||
-- to new nodes after calling master_activate_node.
|
-- to new nodes after calling master_activate_node.
|
||||||
-- now, first drop seg and existing objects before next test
|
-- now, first drop seg and existing objects before next test
|
||||||
SET client_min_messages TO WARNING;
|
|
||||||
DROP EXTENSION seg CASCADE;
|
DROP EXTENSION seg CASCADE;
|
||||||
-- but as we have only 2 ports in postgresql tests, let's remove one of the nodes first
|
-- but as we have only 2 ports in postgresql tests, let's remove one of the nodes first
|
||||||
-- before remove, first remove the existing relations (due to the other tests)
|
-- before remove, first remove the existing relations (due to the other tests)
|
||||||
DROP SCHEMA "extension'test" CASCADE;
|
DROP SCHEMA "extension'test" CASCADE;
|
||||||
RESET client_min_messages;
|
|
||||||
SELECT 1 from master_remove_node('localhost', :worker_2_port);
|
SELECT 1 from master_remove_node('localhost', :worker_2_port);
|
||||||
?column?
|
?column?
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
|
@ -247,7 +240,6 @@ SELECT run_command_on_workers($$SELECT extversion FROM pg_extension WHERE extnam
|
||||||
|
|
||||||
-- and add the other node
|
-- and add the other node
|
||||||
SELECT 1 from master_add_node('localhost', :worker_2_port);
|
SELECT 1 from master_add_node('localhost', :worker_2_port);
|
||||||
NOTICE: Replicating reference table "ref_table_2" to the node localhost:xxxxx
|
|
||||||
?column?
|
?column?
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
1
|
||||||
|
@ -322,11 +314,7 @@ SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname
|
||||||
CREATE TABLE t1 (A int);
|
CREATE TABLE t1 (A int);
|
||||||
CREATE VIEW v1 AS select * from t1;
|
CREATE VIEW v1 AS select * from t1;
|
||||||
ALTER EXTENSION seg ADD VIEW v1;
|
ALTER EXTENSION seg ADD VIEW v1;
|
||||||
NOTICE: Citus does not propagate adding/dropping member objects
|
|
||||||
HINT: You can add/drop the member objects on the workers as well.
|
|
||||||
ALTER EXTENSION seg DROP VIEW v1;
|
ALTER EXTENSION seg DROP VIEW v1;
|
||||||
NOTICE: Citus does not propagate adding/dropping member objects
|
|
||||||
HINT: You can add/drop the member objects on the workers as well.
|
|
||||||
DROP VIEW v1;
|
DROP VIEW v1;
|
||||||
DROP TABLE t1;
|
DROP TABLE t1;
|
||||||
-- drop multiple extensions at the same time
|
-- drop multiple extensions at the same time
|
||||||
|
@ -335,8 +323,6 @@ CREATE EXTENSION isn WITH VERSION '1.1' SCHEMA public;
|
||||||
set citus.enable_ddl_propagation to 'off';
|
set citus.enable_ddl_propagation to 'off';
|
||||||
CREATE EXTENSION pg_buffercache;
|
CREATE EXTENSION pg_buffercache;
|
||||||
set citus.enable_ddl_propagation to 'on';
|
set citus.enable_ddl_propagation to 'on';
|
||||||
-- SET client_min_messages TO WARNING before executing a DROP EXTENSION statement
|
|
||||||
SET client_min_messages TO WARNING;
|
|
||||||
DROP EXTENSION pg_buffercache, isn CASCADE;
|
DROP EXTENSION pg_buffercache, isn CASCADE;
|
||||||
SELECT count(*) FROM pg_extension WHERE extname IN ('pg_buffercache', 'isn');
|
SELECT count(*) FROM pg_extension WHERE extname IN ('pg_buffercache', 'isn');
|
||||||
count
|
count
|
||||||
|
@ -344,10 +330,6 @@ SELECT count(*) FROM pg_extension WHERE extname IN ('pg_buffercache', 'isn');
|
||||||
0
|
0
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
-- restore client_min_messages after DROP EXTENSION
|
|
||||||
RESET client_min_messages;
|
|
||||||
-- SET client_min_messages TO WARNING before executing a DROP EXTENSION statement
|
|
||||||
SET client_min_messages TO WARNING;
|
|
||||||
-- drop extension should just work
|
-- drop extension should just work
|
||||||
DROP EXTENSION seg CASCADE;
|
DROP EXTENSION seg CASCADE;
|
||||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||||
|
@ -363,8 +345,6 @@ SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname
|
||||||
(localhost,57638,t,0)
|
(localhost,57638,t,0)
|
||||||
(2 rows)
|
(2 rows)
|
||||||
|
|
||||||
-- restore client_min_messages after DROP EXTENSION
|
|
||||||
RESET client_min_messages;
|
|
||||||
-- make sure that the extension is not avaliable anymore as a distributed object
|
-- make sure that the extension is not avaliable anymore as a distributed object
|
||||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn'));
|
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn'));
|
||||||
count
|
count
|
||||||
|
@ -403,13 +383,11 @@ SELECT run_command_on_workers($$SELECT extversion FROM pg_extension WHERE extnam
|
||||||
(2 rows)
|
(2 rows)
|
||||||
|
|
||||||
-- drop the schema and all the objects
|
-- drop the schema and all the objects
|
||||||
SET client_min_messages TO WARNING;
|
|
||||||
DROP SCHEMA "extension'test" CASCADE;
|
DROP SCHEMA "extension'test" CASCADE;
|
||||||
-- recreate for the next tests
|
-- recreate for the next tests
|
||||||
CREATE SCHEMA "extension'test";
|
CREATE SCHEMA "extension'test";
|
||||||
-- use a schema name with escape character
|
-- use a schema name with escape character
|
||||||
SET search_path TO "extension'test";
|
SET search_path TO "extension'test";
|
||||||
RESET client_min_messages;
|
|
||||||
-- remove the node, we'll add back again
|
-- remove the node, we'll add back again
|
||||||
SELECT 1 from master_remove_node('localhost', :worker_2_port);
|
SELECT 1 from master_remove_node('localhost', :worker_2_port);
|
||||||
?column?
|
?column?
|
||||||
|
@ -443,7 +421,6 @@ BEGIN;
|
||||||
COMMIT;
|
COMMIT;
|
||||||
-- add the node back
|
-- add the node back
|
||||||
SELECT 1 from master_add_node('localhost', :worker_2_port);
|
SELECT 1 from master_add_node('localhost', :worker_2_port);
|
||||||
NOTICE: Replicating reference table "t3" to the node localhost:xxxxx
|
|
||||||
?column?
|
?column?
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1
|
1
|
||||||
|
@ -464,5 +441,4 @@ SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname
|
||||||
(2 rows)
|
(2 rows)
|
||||||
|
|
||||||
-- drop the schema and all the objects
|
-- drop the schema and all the objects
|
||||||
SET client_min_messages TO WARNING;
|
|
||||||
DROP SCHEMA "extension'test" CASCADE;
|
DROP SCHEMA "extension'test" CASCADE;
|
||||||
|
|
|
@ -502,7 +502,9 @@ SELECT shardid, nodename, nodeport
|
||||||
WHERE logicalrelid = 'numbers_append'::regclass order by placementid;
|
WHERE logicalrelid = 'numbers_append'::regclass order by placementid;
|
||||||
|
|
||||||
-- add the node back
|
-- add the node back
|
||||||
|
SET client_min_messages TO ERROR;
|
||||||
SELECT 1 FROM master_activate_node('localhost', :worker_1_port);
|
SELECT 1 FROM master_activate_node('localhost', :worker_1_port);
|
||||||
|
RESET client_min_messages;
|
||||||
RESET citus.shard_replication_factor;
|
RESET citus.shard_replication_factor;
|
||||||
-- add two new shards and verify they are created at both workers
|
-- add two new shards and verify they are created at both workers
|
||||||
COPY numbers_append FROM STDIN WITH (FORMAT 'csv');
|
COPY numbers_append FROM STDIN WITH (FORMAT 'csv');
|
||||||
|
|
|
@ -674,19 +674,14 @@ SELECT shardid, nodename, nodeport
|
||||||
(4 rows)
|
(4 rows)
|
||||||
|
|
||||||
-- add the node back
|
-- add the node back
|
||||||
|
SET client_min_messages TO ERROR;
|
||||||
SELECT 1 FROM master_activate_node('localhost', :worker_1_port);
|
SELECT 1 FROM master_activate_node('localhost', :worker_1_port);
|
||||||
NOTICE: Replicating reference table "orders_reference" to the node localhost:57637
|
|
||||||
NOTICE: Replicating reference table "customer" to the node localhost:57637
|
|
||||||
NOTICE: Replicating reference table "nation" to the node localhost:57637
|
|
||||||
NOTICE: Replicating reference table "part" to the node localhost:57637
|
|
||||||
NOTICE: Replicating reference table "supplier" to the node localhost:57637
|
|
||||||
NOTICE: Replicating reference table "multi_outer_join_right_reference" to the node localhost:57637
|
|
||||||
NOTICE: Replicating reference table "multi_outer_join_third_reference" to the node localhost:57637
|
|
||||||
?column?
|
?column?
|
||||||
----------
|
----------
|
||||||
1
|
1
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
RESET client_min_messages;
|
||||||
RESET citus.shard_replication_factor;
|
RESET citus.shard_replication_factor;
|
||||||
-- add two new shards and verify they are created at both workers
|
-- add two new shards and verify they are created at both workers
|
||||||
COPY numbers_append FROM STDIN WITH (FORMAT 'csv');
|
COPY numbers_append FROM STDIN WITH (FORMAT 'csv');
|
||||||
|
|
|
@ -16,9 +16,9 @@ setup
|
||||||
// ensure neither node's added for the remaining of the isolation tests
|
// ensure neither node's added for the remaining of the isolation tests
|
||||||
teardown
|
teardown
|
||||||
{
|
{
|
||||||
DROP TABLE test_reference_table;
|
DROP TABLE IF EXISTS test_reference_table;
|
||||||
DROP TABLE test_reference_table_2;
|
DROP TABLE IF EXISTS test_reference_table_2;
|
||||||
DROP TABLE test_table;
|
DROP TABLE IF EXISTS test_table;
|
||||||
SELECT master_remove_node(nodename, nodeport) FROM pg_dist_node;
|
SELECT master_remove_node(nodename, nodeport) FROM pg_dist_node;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -29,6 +29,16 @@ step "s1-begin"
|
||||||
BEGIN;
|
BEGIN;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
step "s1-replicate-on-activate"
|
||||||
|
{
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO on;
|
||||||
|
}
|
||||||
|
|
||||||
|
step "s1-do-not-replicate-on-activate"
|
||||||
|
{
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
}
|
||||||
|
|
||||||
step "s1-add-second-worker"
|
step "s1-add-second-worker"
|
||||||
{
|
{
|
||||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||||
|
@ -39,6 +49,11 @@ step "s1-remove-second-worker"
|
||||||
SELECT master_remove_node('localhost', 57638);
|
SELECT master_remove_node('localhost', 57638);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
step "s1-drop-reference-table"
|
||||||
|
{
|
||||||
|
DROP TABLE test_reference_table;
|
||||||
|
}
|
||||||
|
|
||||||
step "s1-commit"
|
step "s1-commit"
|
||||||
{
|
{
|
||||||
COMMIT;
|
COMMIT;
|
||||||
|
@ -58,6 +73,13 @@ step "s2-copy-to-reference-table"
|
||||||
COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5';
|
COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5';
|
||||||
}
|
}
|
||||||
|
|
||||||
|
step "s2-replicate-reference-tables"
|
||||||
|
{
|
||||||
|
SET client_min_messages TO DEBUG2;
|
||||||
|
SELECT replicate_reference_tables();
|
||||||
|
RESET client_min_messages;
|
||||||
|
}
|
||||||
|
|
||||||
step "s2-insert-to-reference-table"
|
step "s2-insert-to-reference-table"
|
||||||
{
|
{
|
||||||
INSERT INTO test_reference_table VALUES (6);
|
INSERT INTO test_reference_table VALUES (6);
|
||||||
|
@ -117,23 +139,35 @@ step "s2-print-index-count"
|
||||||
// note that we need to run "s1-load-metadata-cache" and "s2-load-metadata-cache"
|
// note that we need to run "s1-load-metadata-cache" and "s2-load-metadata-cache"
|
||||||
// to ensure that metadata is cached otherwise the test would be useless since
|
// to ensure that metadata is cached otherwise the test would be useless since
|
||||||
// the cache would be empty and the metadata data is gathered from the tables directly
|
// the cache would be empty and the metadata data is gathered from the tables directly
|
||||||
permutation "s2-load-metadata-cache" "s1-begin" "s1-add-second-worker" "s2-copy-to-reference-table" "s1-commit" "s2-print-content"
|
permutation "s1-do-not-replicate-on-activate" "s2-load-metadata-cache" "s1-begin" "s1-add-second-worker" "s2-copy-to-reference-table" "s1-commit" "s2-print-content"
|
||||||
permutation "s2-load-metadata-cache" "s2-begin" "s2-copy-to-reference-table" "s1-add-second-worker" "s2-commit" "s2-print-content"
|
permutation "s1-do-not-replicate-on-activate" "s2-load-metadata-cache" "s2-begin" "s2-copy-to-reference-table" "s1-add-second-worker" "s2-commit" "s2-print-content"
|
||||||
permutation "s2-load-metadata-cache" "s1-begin" "s1-add-second-worker" "s2-insert-to-reference-table" "s1-commit" "s2-print-content"
|
permutation "s1-do-not-replicate-on-activate" "s2-load-metadata-cache" "s1-begin" "s1-add-second-worker" "s2-insert-to-reference-table" "s1-commit" "s2-print-content"
|
||||||
permutation "s2-load-metadata-cache" "s2-begin" "s2-insert-to-reference-table" "s1-add-second-worker" "s2-commit" "s2-print-content"
|
permutation "s1-do-not-replicate-on-activate" "s2-load-metadata-cache" "s2-begin" "s2-insert-to-reference-table" "s1-add-second-worker" "s2-commit" "s2-print-content"
|
||||||
permutation "s2-load-metadata-cache" "s1-begin" "s1-add-second-worker" "s2-ddl-on-reference-table" "s1-commit" "s2-print-index-count"
|
permutation "s1-do-not-replicate-on-activate" "s2-load-metadata-cache" "s1-begin" "s1-add-second-worker" "s2-ddl-on-reference-table" "s1-commit" "s2-print-index-count"
|
||||||
permutation "s2-load-metadata-cache" "s2-begin" "s2-ddl-on-reference-table" "s1-add-second-worker" "s2-commit" "s2-print-index-count"
|
permutation "s1-do-not-replicate-on-activate" "s2-load-metadata-cache" "s2-begin" "s2-ddl-on-reference-table" "s1-add-second-worker" "s2-commit" "s2-print-index-count"
|
||||||
permutation "s2-load-metadata-cache" "s1-begin" "s1-add-second-worker" "s2-create-reference-table-2" "s1-commit" "s2-print-content-2"
|
permutation "s1-do-not-replicate-on-activate" "s2-load-metadata-cache" "s1-begin" "s1-add-second-worker" "s2-create-reference-table-2" "s1-commit" "s2-print-content-2"
|
||||||
permutation "s2-load-metadata-cache" "s2-begin" "s2-create-reference-table-2" "s1-add-second-worker" "s2-commit" "s2-print-content-2"
|
permutation "s1-do-not-replicate-on-activate" "s2-load-metadata-cache" "s2-begin" "s2-create-reference-table-2" "s1-add-second-worker" "s2-commit" "s2-print-content-2"
|
||||||
|
|
||||||
|
|
||||||
// same tests without loading the cache
|
// same tests without loading the cache
|
||||||
permutation "s1-begin" "s1-add-second-worker" "s2-copy-to-reference-table" "s1-commit" "s2-print-content"
|
permutation "s1-do-not-replicate-on-activate" "s1-begin" "s1-add-second-worker" "s2-copy-to-reference-table" "s1-commit" "s2-print-content"
|
||||||
permutation "s2-begin" "s2-copy-to-reference-table" "s1-add-second-worker" "s2-commit" "s2-print-content"
|
permutation "s1-do-not-replicate-on-activate" "s2-begin" "s2-copy-to-reference-table" "s1-add-second-worker" "s2-commit" "s2-print-content"
|
||||||
permutation "s1-begin" "s1-add-second-worker" "s2-insert-to-reference-table" "s1-commit" "s2-print-content"
|
permutation "s1-do-not-replicate-on-activate" "s1-begin" "s1-add-second-worker" "s2-insert-to-reference-table" "s1-commit" "s2-print-content"
|
||||||
permutation "s2-begin" "s2-insert-to-reference-table" "s1-add-second-worker" "s2-commit" "s2-print-content"
|
permutation "s1-do-not-replicate-on-activate" "s2-begin" "s2-insert-to-reference-table" "s1-add-second-worker" "s2-commit" "s2-print-content"
|
||||||
permutation "s1-begin" "s1-add-second-worker" "s2-ddl-on-reference-table" "s1-commit" "s2-print-index-count"
|
permutation "s1-do-not-replicate-on-activate" "s1-begin" "s1-add-second-worker" "s2-ddl-on-reference-table" "s1-commit" "s2-print-index-count"
|
||||||
permutation "s2-begin" "s2-ddl-on-reference-table" "s1-add-second-worker" "s2-commit" "s2-print-index-count"
|
permutation "s1-do-not-replicate-on-activate" "s2-begin" "s2-ddl-on-reference-table" "s1-add-second-worker" "s2-commit" "s2-print-index-count"
|
||||||
permutation "s1-begin" "s1-add-second-worker" "s2-create-reference-table-2" "s1-commit" "s2-print-content-2"
|
permutation "s1-do-not-replicate-on-activate" "s1-begin" "s1-add-second-worker" "s2-create-reference-table-2" "s1-commit" "s2-print-content-2"
|
||||||
permutation "s2-begin" "s2-create-reference-table-2" "s1-add-second-worker" "s2-commit" "s2-print-content-2"
|
permutation "s1-do-not-replicate-on-activate" "s2-begin" "s2-create-reference-table-2" "s1-add-second-worker" "s2-commit" "s2-print-content-2"
|
||||||
|
|
||||||
|
// same tests with replicate on activate
|
||||||
|
permutation "s1-replicate-on-activate" "s1-begin" "s1-add-second-worker" "s2-copy-to-reference-table" "s1-commit" "s2-print-content"
|
||||||
|
permutation "s1-replicate-on-activate" "s2-begin" "s2-copy-to-reference-table" "s1-add-second-worker" "s2-commit" "s2-print-content"
|
||||||
|
permutation "s1-replicate-on-activate" "s1-begin" "s1-add-second-worker" "s2-insert-to-reference-table" "s1-commit" "s2-print-content"
|
||||||
|
permutation "s1-replicate-on-activate" "s2-begin" "s2-insert-to-reference-table" "s1-add-second-worker" "s2-commit" "s2-print-content"
|
||||||
|
permutation "s1-replicate-on-activate" "s1-begin" "s1-add-second-worker" "s2-ddl-on-reference-table" "s1-commit" "s2-print-index-count"
|
||||||
|
permutation "s1-replicate-on-activate" "s2-begin" "s2-ddl-on-reference-table" "s1-add-second-worker" "s2-commit" "s2-print-index-count"
|
||||||
|
permutation "s1-replicate-on-activate" "s1-begin" "s1-add-second-worker" "s2-create-reference-table-2" "s1-commit" "s2-print-content-2"
|
||||||
|
permutation "s1-replicate-on-activate" "s2-begin" "s2-create-reference-table-2" "s1-add-second-worker" "s2-commit" "s2-print-content-2"
|
||||||
|
|
||||||
|
// verify drop table blocks replicate reference tables
|
||||||
|
permutation "s1-do-not-replicate-on-activate" "s1-add-second-worker" "s2-begin" "s1-begin" "s1-drop-reference-table" "s2-replicate-reference-tables" "s1-commit" "s2-commit"
|
||||||
|
|
|
@ -206,7 +206,7 @@ permutation "s1-print-distributed-objects" "s1-begin" "s2-begin" "s2-create-sche
|
||||||
// concurrency tests with multi schema distribution
|
// concurrency tests with multi schema distribution
|
||||||
permutation "s1-print-distributed-objects" "s2-create-schema" "s1-begin" "s2-begin" "s3-begin" "s1-add-worker" "s2-create-table" "s3-use-schema" "s3-create-table" "s1-commit" "s2-commit" "s3-commit" "s2-print-distributed-objects"
|
permutation "s1-print-distributed-objects" "s2-create-schema" "s1-begin" "s2-begin" "s3-begin" "s1-add-worker" "s2-create-table" "s3-use-schema" "s3-create-table" "s1-commit" "s2-commit" "s3-commit" "s2-print-distributed-objects"
|
||||||
permutation "s1-print-distributed-objects" "s1-add-worker" "s2-create-schema" "s2-begin" "s3-begin" "s3-use-schema" "s2-create-table" "s3-create-table" "s2-commit" "s3-commit" "s2-print-distributed-objects"
|
permutation "s1-print-distributed-objects" "s1-add-worker" "s2-create-schema" "s2-begin" "s3-begin" "s3-use-schema" "s2-create-table" "s3-create-table" "s2-commit" "s3-commit" "s2-print-distributed-objects"
|
||||||
permutation "s1-print-distributed-objects" "s1-begin" "s2-begin" "s3-begin" "s1-add-worker" "s2-create-schema" "s3-create-schema2" "s2-create-table" "s3-create-table" "s1-commit" "s2-commit" "s3-commit" "s2-print-distributed-objects"
|
permutation "s1-print-distributed-objects" "s1-begin" "s2-begin" "s3-begin" "s1-add-worker" "s2-create-schema" "s3-create-schema2" "s2-create-table" "s3-create-table" "s1-commit" "s3-commit" "s2-commit" "s2-print-distributed-objects"
|
||||||
|
|
||||||
// type and schema tests
|
// type and schema tests
|
||||||
permutation "s1-print-distributed-objects" "s1-begin" "s1-add-worker" "s2-public-schema" "s2-create-type" "s1-commit" "s2-print-distributed-objects"
|
permutation "s1-print-distributed-objects" "s1-begin" "s1-add-worker" "s2-public-schema" "s2-create-type" "s1-commit" "s2-print-distributed-objects"
|
||||||
|
|
|
@ -8,6 +8,7 @@
|
||||||
SELECT citus.mitmproxy('conn.allow()');
|
SELECT citus.mitmproxy('conn.allow()');
|
||||||
|
|
||||||
SET citus.next_shard_id TO 200000;
|
SET citus.next_shard_id TO 200000;
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
-- verify we have all worker nodes present
|
-- verify we have all worker nodes present
|
||||||
SELECT * FROM master_get_active_worker_nodes()
|
SELECT * FROM master_get_active_worker_nodes()
|
||||||
|
@ -39,13 +40,6 @@ FROM pg_dist_placement p JOIN pg_dist_shard s USING (shardid)
|
||||||
WHERE s.logicalrelid = 'user_table'::regclass
|
WHERE s.logicalrelid = 'user_table'::regclass
|
||||||
ORDER BY placementid;
|
ORDER BY placementid;
|
||||||
|
|
||||||
-- fail activate node by failing reference table creation
|
|
||||||
SELECT citus.mitmproxy('conn.onQuery(query="CREATE TABLE").kill()');
|
|
||||||
|
|
||||||
SELECT master_activate_node('localhost', :worker_2_proxy_port);
|
|
||||||
|
|
||||||
SELECT citus.mitmproxy('conn.allow()');
|
|
||||||
|
|
||||||
-- verify node is not activated
|
-- verify node is not activated
|
||||||
SELECT * FROM master_get_active_worker_nodes()
|
SELECT * FROM master_get_active_worker_nodes()
|
||||||
ORDER BY 1, 2;
|
ORDER BY 1, 2;
|
||||||
|
@ -69,22 +63,6 @@ FROM pg_dist_placement p JOIN pg_dist_shard s USING (shardid)
|
||||||
WHERE s.logicalrelid = 'user_table'::regclass
|
WHERE s.logicalrelid = 'user_table'::regclass
|
||||||
ORDER BY placementid;
|
ORDER BY placementid;
|
||||||
|
|
||||||
-- fail activate node by failing reference table creation
|
|
||||||
SELECT citus.mitmproxy('conn.onQuery(query="CREATE TABLE").cancel(' || pg_backend_pid() || ')');
|
|
||||||
|
|
||||||
SELECT master_activate_node('localhost', :worker_2_proxy_port);
|
|
||||||
|
|
||||||
-- verify node is not activated
|
|
||||||
SELECT * FROM master_get_active_worker_nodes()
|
|
||||||
ORDER BY 1, 2;
|
|
||||||
|
|
||||||
SELECT shardid, shardstate
|
|
||||||
FROM pg_dist_placement p JOIN pg_dist_shard s USING (shardid)
|
|
||||||
WHERE s.logicalrelid = 'user_table'::regclass
|
|
||||||
ORDER BY placementid;
|
|
||||||
|
|
||||||
SELECT citus.mitmproxy('conn.allow()');
|
|
||||||
|
|
||||||
-- master_remove_node fails when there are shards on that worker
|
-- master_remove_node fails when there are shards on that worker
|
||||||
SELECT master_remove_node('localhost', :worker_2_proxy_port);
|
SELECT master_remove_node('localhost', :worker_2_proxy_port);
|
||||||
|
|
||||||
|
@ -113,34 +91,6 @@ FROM pg_dist_placement p JOIN pg_dist_shard s USING (shardid)
|
||||||
WHERE s.logicalrelid = 'user_table'::regclass
|
WHERE s.logicalrelid = 'user_table'::regclass
|
||||||
ORDER BY placementid;
|
ORDER BY placementid;
|
||||||
|
|
||||||
-- test master_add_node replicated a reference table
|
|
||||||
-- to newly added node.
|
|
||||||
SELECT citus.mitmproxy('conn.onQuery(query="CREATE TABLE").kill()');
|
|
||||||
|
|
||||||
SELECT master_add_node('localhost', :worker_2_proxy_port);
|
|
||||||
|
|
||||||
-- verify node is not added
|
|
||||||
SELECT * FROM master_get_active_worker_nodes()
|
|
||||||
ORDER BY 1, 2;
|
|
||||||
|
|
||||||
SELECT shardid, shardstate
|
|
||||||
FROM pg_dist_placement p JOIN pg_dist_shard s USING (shardid)
|
|
||||||
WHERE s.logicalrelid = 'user_table'::regclass
|
|
||||||
ORDER BY placementid;
|
|
||||||
|
|
||||||
SELECT citus.mitmproxy('conn.onQuery(query="CREATE TABLE").cancel(' || pg_backend_pid() || ')');
|
|
||||||
|
|
||||||
SELECT master_add_node('localhost', :worker_2_proxy_port);
|
|
||||||
|
|
||||||
-- verify node is not added
|
|
||||||
SELECT * FROM master_get_active_worker_nodes()
|
|
||||||
ORDER BY 1, 2;
|
|
||||||
|
|
||||||
SELECT shardid, shardstate
|
|
||||||
FROM pg_dist_placement p JOIN pg_dist_shard s USING (shardid)
|
|
||||||
WHERE s.logicalrelid = 'user_table'::regclass
|
|
||||||
ORDER BY placementid;
|
|
||||||
|
|
||||||
-- reset cluster to original state
|
-- reset cluster to original state
|
||||||
SELECT citus.mitmproxy('conn.allow()');
|
SELECT citus.mitmproxy('conn.allow()');
|
||||||
SELECT master_add_node('localhost', :worker_2_proxy_port);
|
SELECT master_add_node('localhost', :worker_2_proxy_port);
|
||||||
|
@ -154,15 +104,6 @@ FROM pg_dist_placement p JOIN pg_dist_shard s USING (shardid)
|
||||||
WHERE s.logicalrelid = 'user_table'::regclass
|
WHERE s.logicalrelid = 'user_table'::regclass
|
||||||
ORDER BY placementid;
|
ORDER BY placementid;
|
||||||
|
|
||||||
-- fail master_add_node by failing copy out operation
|
|
||||||
SELECT master_remove_node('localhost', :worker_1_port);
|
|
||||||
SELECT citus.mitmproxy('conn.onQuery(query="COPY").kill()');
|
|
||||||
SELECT master_add_node('localhost', :worker_1_port);
|
|
||||||
|
|
||||||
-- verify node is not added
|
|
||||||
SELECT * FROM master_get_active_worker_nodes()
|
|
||||||
ORDER BY 1, 2;
|
|
||||||
|
|
||||||
SELECT citus.mitmproxy('conn.allow()');
|
SELECT citus.mitmproxy('conn.allow()');
|
||||||
SELECT master_add_node('localhost', :worker_1_port);
|
SELECT master_add_node('localhost', :worker_1_port);
|
||||||
|
|
||||||
|
|
|
@ -3,6 +3,7 @@ SET search_path TO local_shard_copy;
|
||||||
|
|
||||||
SET client_min_messages TO DEBUG;
|
SET client_min_messages TO DEBUG;
|
||||||
SET citus.next_shard_id TO 1570000;
|
SET citus.next_shard_id TO 1570000;
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
SELECT * FROM master_add_node('localhost', :master_port, groupid := 0);
|
SELECT * FROM master_add_node('localhost', :master_port, groupid := 0);
|
||||||
|
|
||||||
|
|
|
@ -7,6 +7,7 @@
|
||||||
|
|
||||||
|
|
||||||
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1310000;
|
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1310000;
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
SELECT nextval('pg_catalog.pg_dist_placement_placementid_seq') AS last_placement_id
|
SELECT nextval('pg_catalog.pg_dist_placement_placementid_seq') AS last_placement_id
|
||||||
\gset
|
\gset
|
||||||
|
|
|
@ -11,6 +11,7 @@ SELECT nextval('pg_catalog.pg_dist_shardid_seq') AS last_shard_id \gset
|
||||||
SET citus.replication_model TO streaming;
|
SET citus.replication_model TO streaming;
|
||||||
SET citus.shard_count TO 8;
|
SET citus.shard_count TO 8;
|
||||||
SET citus.shard_replication_factor TO 1;
|
SET citus.shard_replication_factor TO 1;
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
-- Simulates a readonly node by setting default_transaction_read_only.
|
-- Simulates a readonly node by setting default_transaction_read_only.
|
||||||
CREATE FUNCTION mark_node_readonly(hostname TEXT, port INTEGER, isreadonly BOOLEAN)
|
CREATE FUNCTION mark_node_readonly(hostname TEXT, port INTEGER, isreadonly BOOLEAN)
|
||||||
|
|
|
@ -5,6 +5,7 @@
|
||||||
|
|
||||||
|
|
||||||
SET citus.next_shard_id TO 1380000;
|
SET citus.next_shard_id TO 1380000;
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 1380000;
|
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 1380000;
|
||||||
ALTER SEQUENCE pg_catalog.pg_dist_groupid_seq RESTART 1380000;
|
ALTER SEQUENCE pg_catalog.pg_dist_groupid_seq RESTART 1380000;
|
||||||
ALTER SEQUENCE pg_catalog.pg_dist_node_nodeid_seq RESTART 1380000;
|
ALTER SEQUENCE pg_catalog.pg_dist_node_nodeid_seq RESTART 1380000;
|
||||||
|
@ -115,6 +116,7 @@ WHERE
|
||||||
nodeport = :worker_2_port;
|
nodeport = :worker_2_port;
|
||||||
|
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
-- remove same node twice
|
-- remove same node twice
|
||||||
SELECT master_remove_node('localhost', :worker_2_port);
|
SELECT master_remove_node('localhost', :worker_2_port);
|
||||||
|
@ -260,6 +262,7 @@ WHERE
|
||||||
nodeport = :worker_2_port;
|
nodeport = :worker_2_port;
|
||||||
|
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
-- re-add the node for next tests
|
-- re-add the node for next tests
|
||||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
|
@ -335,6 +338,7 @@ WHERE
|
||||||
SELECT * FROM remove_node_reference_table;
|
SELECT * FROM remove_node_reference_table;
|
||||||
|
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
-- re-add the node for next tests
|
-- re-add the node for next tests
|
||||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
|
@ -406,6 +410,7 @@ WHERE
|
||||||
nodeport = :worker_2_port;
|
nodeport = :worker_2_port;
|
||||||
|
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
SET citus.next_shard_id TO 1380001;
|
SET citus.next_shard_id TO 1380001;
|
||||||
|
|
||||||
|
@ -528,6 +533,7 @@ WHERE
|
||||||
nodeport = :worker_2_port;
|
nodeport = :worker_2_port;
|
||||||
|
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
-- re-add the node for next tests
|
-- re-add the node for next tests
|
||||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
|
@ -599,6 +605,7 @@ WHERE
|
||||||
nodeport = :worker_2_port;
|
nodeport = :worker_2_port;
|
||||||
|
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
-- re-add the node for next tests
|
-- re-add the node for next tests
|
||||||
SELECT 1 FROM master_activate_node('localhost', :worker_2_port);
|
SELECT 1 FROM master_activate_node('localhost', :worker_2_port);
|
||||||
|
|
|
@ -3,11 +3,19 @@
|
||||||
--
|
--
|
||||||
-- Tests that check that reference tables are replicated when adding new nodes.
|
-- Tests that check that reference tables are replicated when adding new nodes.
|
||||||
|
|
||||||
|
CREATE SCHEMA replicate_reference_table;
|
||||||
|
SET search_path TO replicate_reference_table;
|
||||||
|
|
||||||
SET citus.next_shard_id TO 1370000;
|
SET citus.next_shard_id TO 1370000;
|
||||||
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 1370000;
|
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 1370000;
|
||||||
ALTER SEQUENCE pg_catalog.pg_dist_groupid_seq RESTART 1370000;
|
ALTER SEQUENCE pg_catalog.pg_dist_groupid_seq RESTART 1370000;
|
||||||
ALTER SEQUENCE pg_catalog.pg_dist_node_nodeid_seq RESTART 1370000;
|
ALTER SEQUENCE pg_catalog.pg_dist_node_nodeid_seq RESTART 1370000;
|
||||||
|
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO off;
|
||||||
|
|
||||||
|
-- only query shards created in this test
|
||||||
|
CREATE VIEW pg_dist_shard_placement_view AS
|
||||||
|
SELECT * FROM pg_dist_shard_placement WHERE shardid BETWEEN 1370000 AND 1380000;
|
||||||
|
|
||||||
-- remove a node for testing purposes
|
-- remove a node for testing purposes
|
||||||
CREATE TABLE tmp_shard_placement AS SELECT * FROM pg_dist_shard_placement WHERE nodeport = :worker_2_port;
|
CREATE TABLE tmp_shard_placement AS SELECT * FROM pg_dist_shard_placement WHERE nodeport = :worker_2_port;
|
||||||
|
@ -29,7 +37,7 @@ SELECT COUNT(*) FROM pg_dist_node WHERE nodeport = :worker_2_port;
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port;
|
nodeport = :worker_2_port;
|
||||||
|
|
||||||
|
@ -53,7 +61,7 @@ SELECT COUNT(*) FROM pg_dist_node WHERE nodeport = :worker_2_port;
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port;
|
nodeport = :worker_2_port;
|
||||||
|
|
||||||
|
@ -68,13 +76,12 @@ SELECT create_reference_table('replicate_reference_table_valid');
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
|
|
||||||
SELECT *
|
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation
|
||||||
FROM pg_dist_colocation
|
|
||||||
WHERE colocationid IN
|
WHERE colocationid IN
|
||||||
(SELECT colocationid
|
(SELECT colocationid
|
||||||
FROM pg_dist_partition
|
FROM pg_dist_partition
|
||||||
|
@ -86,13 +93,12 @@ SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
|
|
||||||
SELECT *
|
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation
|
||||||
FROM pg_dist_colocation
|
|
||||||
WHERE colocationid IN
|
WHERE colocationid IN
|
||||||
(SELECT colocationid
|
(SELECT colocationid
|
||||||
FROM pg_dist_partition
|
FROM pg_dist_partition
|
||||||
|
@ -105,13 +111,12 @@ WHERE colocationid IN
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
|
|
||||||
SELECT *
|
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation
|
||||||
FROM pg_dist_colocation
|
|
||||||
WHERE colocationid IN
|
WHERE colocationid IN
|
||||||
(SELECT colocationid
|
(SELECT colocationid
|
||||||
FROM pg_dist_partition
|
FROM pg_dist_partition
|
||||||
|
@ -123,13 +128,12 @@ SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
|
|
||||||
SELECT *
|
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation
|
||||||
FROM pg_dist_colocation
|
|
||||||
WHERE colocationid IN
|
WHERE colocationid IN
|
||||||
(SELECT colocationid
|
(SELECT colocationid
|
||||||
FROM pg_dist_partition
|
FROM pg_dist_partition
|
||||||
|
@ -148,13 +152,12 @@ SELECT create_reference_table('replicate_reference_table_rollback');
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
|
|
||||||
SELECT *
|
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation
|
||||||
FROM pg_dist_colocation
|
|
||||||
WHERE colocationid IN
|
WHERE colocationid IN
|
||||||
(SELECT colocationid
|
(SELECT colocationid
|
||||||
FROM pg_dist_partition
|
FROM pg_dist_partition
|
||||||
|
@ -168,13 +171,12 @@ ROLLBACK;
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
|
|
||||||
SELECT *
|
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation
|
||||||
FROM pg_dist_colocation
|
|
||||||
WHERE colocationid IN
|
WHERE colocationid IN
|
||||||
(SELECT colocationid
|
(SELECT colocationid
|
||||||
FROM pg_dist_partition
|
FROM pg_dist_partition
|
||||||
|
@ -191,13 +193,12 @@ SELECT create_reference_table('replicate_reference_table_commit');
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
|
|
||||||
SELECT *
|
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation
|
||||||
FROM pg_dist_colocation
|
|
||||||
WHERE colocationid IN
|
WHERE colocationid IN
|
||||||
(SELECT colocationid
|
(SELECT colocationid
|
||||||
FROM pg_dist_partition
|
FROM pg_dist_partition
|
||||||
|
@ -211,13 +212,12 @@ COMMIT;
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
|
|
||||||
SELECT *
|
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation
|
||||||
FROM pg_dist_colocation
|
|
||||||
WHERE colocationid IN
|
WHERE colocationid IN
|
||||||
(SELECT colocationid
|
(SELECT colocationid
|
||||||
FROM pg_dist_partition
|
FROM pg_dist_partition
|
||||||
|
@ -247,12 +247,11 @@ CREATE TABLE replicate_reference_table_reference_two(column1 int);
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port;
|
nodeport = :worker_2_port;
|
||||||
|
|
||||||
SELECT *
|
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation
|
||||||
FROM pg_dist_colocation
|
|
||||||
WHERE colocationid IN
|
WHERE colocationid IN
|
||||||
(SELECT colocationid
|
(SELECT colocationid
|
||||||
FROM pg_dist_partition
|
FROM pg_dist_partition
|
||||||
|
@ -268,24 +267,22 @@ WHERE
|
||||||
logicalrelid IN ('replicate_reference_table_reference_one', 'replicate_reference_table_hash', 'replicate_reference_table_reference_two')
|
logicalrelid IN ('replicate_reference_table_reference_one', 'replicate_reference_table_hash', 'replicate_reference_table_reference_two')
|
||||||
ORDER BY logicalrelid;
|
ORDER BY logicalrelid;
|
||||||
|
|
||||||
BEGIN;
|
SET client_min_messages TO WARNING;
|
||||||
SET LOCAL client_min_messages TO ERROR;
|
|
||||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
SELECT upgrade_to_reference_table('replicate_reference_table_hash');
|
SELECT upgrade_to_reference_table('replicate_reference_table_hash');
|
||||||
SELECT create_reference_table('replicate_reference_table_reference_two');
|
SELECT create_reference_table('replicate_reference_table_reference_two');
|
||||||
COMMIT;
|
RESET client_min_messages;
|
||||||
|
|
||||||
-- status after master_add_node
|
-- status after master_add_node
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
|
|
||||||
SELECT *
|
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation
|
||||||
FROM pg_dist_colocation
|
|
||||||
WHERE colocationid IN
|
WHERE colocationid IN
|
||||||
(SELECT colocationid
|
(SELECT colocationid
|
||||||
FROM pg_dist_partition
|
FROM pg_dist_partition
|
||||||
|
@ -360,13 +357,12 @@ SELECT create_reference_table('replicate_reference_table_drop');
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
|
|
||||||
SELECT *
|
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation
|
||||||
FROM pg_dist_colocation
|
|
||||||
WHERE colocationid IN
|
WHERE colocationid IN
|
||||||
(SELECT colocationid
|
(SELECT colocationid
|
||||||
FROM pg_dist_partition
|
FROM pg_dist_partition
|
||||||
|
@ -381,7 +377,7 @@ COMMIT;
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
|
@ -399,13 +395,12 @@ SELECT create_reference_table('replicate_reference_table_schema.table1');
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
|
|
||||||
SELECT *
|
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation
|
||||||
FROM pg_dist_colocation
|
|
||||||
WHERE colocationid IN
|
WHERE colocationid IN
|
||||||
(SELECT colocationid
|
(SELECT colocationid
|
||||||
FROM pg_dist_partition
|
FROM pg_dist_partition
|
||||||
|
@ -417,13 +412,12 @@ SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
|
|
||||||
SELECT *
|
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation
|
||||||
FROM pg_dist_colocation
|
|
||||||
WHERE colocationid IN
|
WHERE colocationid IN
|
||||||
(SELECT colocationid
|
(SELECT colocationid
|
||||||
FROM pg_dist_partition
|
FROM pg_dist_partition
|
||||||
|
@ -448,7 +442,7 @@ SELECT create_reference_table('ref_table_1'),
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
|
@ -459,7 +453,7 @@ SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
nodeport = :worker_2_port
|
nodeport = :worker_2_port
|
||||||
ORDER BY shardid, nodeport;
|
ORDER BY shardid, nodeport;
|
||||||
|
@ -481,7 +475,7 @@ SELECT 1 FROM master_add_inactive_node('localhost', :worker_2_port);
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
shardid IN (SELECT
|
shardid IN (SELECT
|
||||||
shardid
|
shardid
|
||||||
|
@ -498,7 +492,7 @@ SELECT 1 FROM master_activate_node('localhost', :worker_2_port);
|
||||||
SELECT
|
SELECT
|
||||||
shardid, shardstate, shardlength, nodename, nodeport
|
shardid, shardstate, shardlength, nodename, nodeport
|
||||||
FROM
|
FROM
|
||||||
pg_dist_shard_placement
|
pg_dist_shard_placement_view
|
||||||
WHERE
|
WHERE
|
||||||
shardid IN (SELECT
|
shardid IN (SELECT
|
||||||
shardid
|
shardid
|
||||||
|
@ -509,9 +503,80 @@ WHERE
|
||||||
AND nodeport != :master_port
|
AND nodeport != :master_port
|
||||||
ORDER BY 1,4,5;
|
ORDER BY 1,4,5;
|
||||||
|
|
||||||
-- this should have no effect
|
SELECT 1 FROM master_remove_node('localhost', :worker_2_port);
|
||||||
|
|
||||||
|
CREATE TABLE ref_table(a int);
|
||||||
|
SELECT create_reference_table('ref_table');
|
||||||
|
INSERT INTO ref_table SELECT * FROM generate_series(1, 10);
|
||||||
|
|
||||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
|
|
||||||
|
-- verify we cannot replicate reference tables in a transaction modifying pg_dist_node
|
||||||
|
BEGIN;
|
||||||
|
SELECT 1 FROM master_add_inactive_node('invalid-node-name', 9999);
|
||||||
|
SELECT replicate_reference_tables();
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
-- verify we cannot replicate reference tables in a transaction which
|
||||||
|
-- modified reference tables
|
||||||
|
BEGIN;
|
||||||
|
DELETE FROM ref_table;
|
||||||
|
SELECT replicate_reference_tables();
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
BEGIN;
|
||||||
|
ALTER TABLE ref_table ADD COLUMN b int;
|
||||||
|
SELECT replicate_reference_tables();
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
BEGIN;
|
||||||
|
CREATE INDEX ref_idx ON ref_table(a);
|
||||||
|
SELECT replicate_reference_tables();
|
||||||
|
ROLLBACK;
|
||||||
|
|
||||||
|
--
|
||||||
|
-- read from reference table, then replicate, then write. verify
|
||||||
|
-- placements are consistent.
|
||||||
|
--
|
||||||
|
BEGIN;
|
||||||
|
SELECT count(*) FROM ref_table;
|
||||||
|
SELECT replicate_reference_tables();
|
||||||
|
INSERT INTO ref_table VALUES (11);
|
||||||
|
SELECT count(*), sum(a) FROM ref_table;
|
||||||
|
UPDATE ref_table SET a = a + 1;
|
||||||
|
SELECT sum(a) FROM ref_table;
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
SELECT min(result) = max(result) AS consistent FROM run_command_on_placements('ref_table', 'SELECT sum(a) FROM %s');
|
||||||
|
|
||||||
|
SET client_min_messages TO WARNING;
|
||||||
|
|
||||||
|
SELECT count(*) AS ref_table_placements FROM pg_dist_shard NATURAL JOIN pg_dist_shard_placement WHERE logicalrelid = 'ref_table'::regclass \gset
|
||||||
|
|
||||||
|
-- remove reference table replica from worker 2
|
||||||
|
SELECT 1 FROM master_remove_node('localhost', :worker_2_port);
|
||||||
|
|
||||||
|
SELECT count(*) - :ref_table_placements FROM pg_dist_shard NATURAL JOIN pg_dist_shard_placement WHERE logicalrelid = 'ref_table'::regclass;
|
||||||
|
|
||||||
|
-- test setting citus.replicate_reference_tables_on_activate to on
|
||||||
|
-- master_add_node
|
||||||
|
SET citus.replicate_reference_tables_on_activate TO on;
|
||||||
|
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||||
|
|
||||||
|
SELECT count(*) - :ref_table_placements FROM pg_dist_shard NATURAL JOIN pg_dist_shard_placement WHERE logicalrelid = 'ref_table'::regclass;
|
||||||
|
|
||||||
|
-- master_activate_node
|
||||||
|
SELECT 1 FROM master_remove_node('localhost', :worker_2_port);
|
||||||
|
SELECT 1 FROM master_add_inactive_node('localhost', :worker_2_port);
|
||||||
|
|
||||||
|
SELECT count(*) - :ref_table_placements FROM pg_dist_shard NATURAL JOIN pg_dist_shard_placement WHERE logicalrelid = 'ref_table'::regclass;
|
||||||
|
|
||||||
|
SELECT 1 FROM master_activate_node('localhost', :worker_2_port);
|
||||||
|
|
||||||
|
SELECT count(*) - :ref_table_placements FROM pg_dist_shard NATURAL JOIN pg_dist_shard_placement WHERE logicalrelid = 'ref_table'::regclass;
|
||||||
|
|
||||||
|
SELECT min(result) = max(result) AS consistent FROM run_command_on_placements('ref_table', 'SELECT sum(a) FROM %s');
|
||||||
|
|
||||||
-- test adding an invalid node while we have reference tables to replicate
|
-- test adding an invalid node while we have reference tables to replicate
|
||||||
-- set client message level to ERROR and verbosity to terse to supporess
|
-- set client message level to ERROR and verbosity to terse to supporess
|
||||||
-- OS-dependent host name resolution warnings
|
-- OS-dependent host name resolution warnings
|
||||||
|
@ -520,9 +585,6 @@ SET client_min_messages to ERROR;
|
||||||
|
|
||||||
SELECT master_add_node('invalid-node-name', 9999);
|
SELECT master_add_node('invalid-node-name', 9999);
|
||||||
|
|
||||||
SET client_min_messages to DEFAULT;
|
|
||||||
\set VERBOSITY default
|
|
||||||
|
|
||||||
-- drop unnecassary tables
|
-- drop unnecassary tables
|
||||||
DROP TABLE initially_not_replicated_reference_table;
|
DROP TABLE initially_not_replicated_reference_table;
|
||||||
|
|
||||||
|
@ -530,5 +592,4 @@ DROP TABLE initially_not_replicated_reference_table;
|
||||||
INSERT INTO pg_dist_shard_placement (SELECT * FROM tmp_shard_placement);
|
INSERT INTO pg_dist_shard_placement (SELECT * FROM tmp_shard_placement);
|
||||||
DROP TABLE tmp_shard_placement;
|
DROP TABLE tmp_shard_placement;
|
||||||
|
|
||||||
RESET citus.shard_replication_factor;
|
DROP SCHEMA replicate_reference_table CASCADE;
|
||||||
RESET citus.replication_model;
|
|
||||||
|
|
|
@ -7,6 +7,7 @@
|
||||||
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1410000;
|
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1410000;
|
||||||
|
|
||||||
SET citus.shard_count TO 4;
|
SET citus.shard_count TO 4;
|
||||||
|
SET client_min_messages TO WARNING;
|
||||||
|
|
||||||
-- test DROP TABLE(ergo master_drop_all_shards) in transaction, then ROLLBACK
|
-- test DROP TABLE(ergo master_drop_all_shards) in transaction, then ROLLBACK
|
||||||
CREATE TABLE transactional_drop_shards(column1 int);
|
CREATE TABLE transactional_drop_shards(column1 int);
|
||||||
|
@ -362,6 +363,7 @@ ORDER BY
|
||||||
shardid, nodename, nodeport;
|
shardid, nodename, nodeport;
|
||||||
|
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
|
SET client_min_messages TO WARNING;
|
||||||
|
|
||||||
-- try using the coordinator as a worker and then dropping the table
|
-- try using the coordinator as a worker and then dropping the table
|
||||||
SELECT 1 FROM master_add_node('localhost', :master_port);
|
SELECT 1 FROM master_add_node('localhost', :master_port);
|
||||||
|
|
|
@ -3,6 +3,8 @@ CREATE SCHEMA "extension'test";
|
||||||
-- use a schema name with escape character
|
-- use a schema name with escape character
|
||||||
SET search_path TO "extension'test";
|
SET search_path TO "extension'test";
|
||||||
|
|
||||||
|
SET client_min_messages TO WARNING;
|
||||||
|
|
||||||
-- create an extension on the given search_path
|
-- create an extension on the given search_path
|
||||||
-- the extension is on contrib, so should be avaliable for the regression tests
|
-- the extension is on contrib, so should be avaliable for the regression tests
|
||||||
CREATE EXTENSION seg;
|
CREATE EXTENSION seg;
|
||||||
|
@ -48,10 +50,8 @@ CREATE TABLE ref_table (a public.issn);
|
||||||
SELECT create_reference_table('ref_table');
|
SELECT create_reference_table('ref_table');
|
||||||
|
|
||||||
-- now, drop the extension, recreate it with an older version and update it to latest version
|
-- now, drop the extension, recreate it with an older version and update it to latest version
|
||||||
SET client_min_messages TO WARNING;
|
|
||||||
DROP EXTENSION isn CASCADE;
|
DROP EXTENSION isn CASCADE;
|
||||||
CREATE EXTENSION isn WITH VERSION "1.1";
|
CREATE EXTENSION isn WITH VERSION "1.1";
|
||||||
RESET client_min_messages;
|
|
||||||
|
|
||||||
-- before updating the version, ensure the current version
|
-- before updating the version, ensure the current version
|
||||||
SELECT run_command_on_workers($$SELECT extversion FROM pg_extension WHERE extname = 'isn'$$);
|
SELECT run_command_on_workers($$SELECT extversion FROM pg_extension WHERE extname = 'isn'$$);
|
||||||
|
@ -77,26 +77,20 @@ SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_exte
|
||||||
-- show that the ALTER EXTENSION command is propagated
|
-- show that the ALTER EXTENSION command is propagated
|
||||||
SELECT run_command_on_workers($$SELECT nspname from pg_namespace where oid=(SELECT extnamespace FROM pg_extension WHERE extname = 'isn')$$);
|
SELECT run_command_on_workers($$SELECT nspname from pg_namespace where oid=(SELECT extnamespace FROM pg_extension WHERE extname = 'isn')$$);
|
||||||
|
|
||||||
-- SET client_min_messages TO WARNING before executing a DROP EXTENSION statement
|
|
||||||
SET client_min_messages TO WARNING;
|
|
||||||
-- drop the extension finally
|
-- drop the extension finally
|
||||||
DROP EXTENSION isn CASCADE;
|
DROP EXTENSION isn CASCADE;
|
||||||
-- restore client_min_messages after DROP EXTENSION
|
|
||||||
RESET client_min_messages;
|
|
||||||
|
|
||||||
-- now make sure that the reference tables depending on an extension can be succesfully created.
|
-- now make sure that the reference tables depending on an extension can be succesfully created.
|
||||||
-- we should also ensure that we replicate this reference table (and hence the extension)
|
-- we should also ensure that we replicate this reference table (and hence the extension)
|
||||||
-- to new nodes after calling master_activate_node.
|
-- to new nodes after calling master_activate_node.
|
||||||
|
|
||||||
-- now, first drop seg and existing objects before next test
|
-- now, first drop seg and existing objects before next test
|
||||||
SET client_min_messages TO WARNING;
|
|
||||||
DROP EXTENSION seg CASCADE;
|
DROP EXTENSION seg CASCADE;
|
||||||
|
|
||||||
-- but as we have only 2 ports in postgresql tests, let's remove one of the nodes first
|
-- but as we have only 2 ports in postgresql tests, let's remove one of the nodes first
|
||||||
-- before remove, first remove the existing relations (due to the other tests)
|
-- before remove, first remove the existing relations (due to the other tests)
|
||||||
|
|
||||||
DROP SCHEMA "extension'test" CASCADE;
|
DROP SCHEMA "extension'test" CASCADE;
|
||||||
RESET client_min_messages;
|
|
||||||
SELECT 1 from master_remove_node('localhost', :worker_2_port);
|
SELECT 1 from master_remove_node('localhost', :worker_2_port);
|
||||||
|
|
||||||
-- then create the extension
|
-- then create the extension
|
||||||
|
@ -189,15 +183,8 @@ set citus.enable_ddl_propagation to 'off';
|
||||||
CREATE EXTENSION pg_buffercache;
|
CREATE EXTENSION pg_buffercache;
|
||||||
set citus.enable_ddl_propagation to 'on';
|
set citus.enable_ddl_propagation to 'on';
|
||||||
|
|
||||||
-- SET client_min_messages TO WARNING before executing a DROP EXTENSION statement
|
|
||||||
SET client_min_messages TO WARNING;
|
|
||||||
DROP EXTENSION pg_buffercache, isn CASCADE;
|
DROP EXTENSION pg_buffercache, isn CASCADE;
|
||||||
SELECT count(*) FROM pg_extension WHERE extname IN ('pg_buffercache', 'isn');
|
SELECT count(*) FROM pg_extension WHERE extname IN ('pg_buffercache', 'isn');
|
||||||
-- restore client_min_messages after DROP EXTENSION
|
|
||||||
RESET client_min_messages;
|
|
||||||
|
|
||||||
-- SET client_min_messages TO WARNING before executing a DROP EXTENSION statement
|
|
||||||
SET client_min_messages TO WARNING;
|
|
||||||
|
|
||||||
-- drop extension should just work
|
-- drop extension should just work
|
||||||
DROP EXTENSION seg CASCADE;
|
DROP EXTENSION seg CASCADE;
|
||||||
|
@ -205,9 +192,6 @@ DROP EXTENSION seg CASCADE;
|
||||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname = 'seg'$$);
|
SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname = 'seg'$$);
|
||||||
|
|
||||||
-- restore client_min_messages after DROP EXTENSION
|
|
||||||
RESET client_min_messages;
|
|
||||||
|
|
||||||
-- make sure that the extension is not avaliable anymore as a distributed object
|
-- make sure that the extension is not avaliable anymore as a distributed object
|
||||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn'));
|
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn'));
|
||||||
|
|
||||||
|
@ -229,7 +213,6 @@ ROLLBACK;
|
||||||
SELECT run_command_on_workers($$SELECT extversion FROM pg_extension WHERE extname = 'seg'$$);
|
SELECT run_command_on_workers($$SELECT extversion FROM pg_extension WHERE extname = 'seg'$$);
|
||||||
|
|
||||||
-- drop the schema and all the objects
|
-- drop the schema and all the objects
|
||||||
SET client_min_messages TO WARNING;
|
|
||||||
DROP SCHEMA "extension'test" CASCADE;
|
DROP SCHEMA "extension'test" CASCADE;
|
||||||
|
|
||||||
-- recreate for the next tests
|
-- recreate for the next tests
|
||||||
|
@ -238,8 +221,6 @@ CREATE SCHEMA "extension'test";
|
||||||
-- use a schema name with escape character
|
-- use a schema name with escape character
|
||||||
SET search_path TO "extension'test";
|
SET search_path TO "extension'test";
|
||||||
|
|
||||||
RESET client_min_messages;
|
|
||||||
|
|
||||||
-- remove the node, we'll add back again
|
-- remove the node, we'll add back again
|
||||||
SELECT 1 from master_remove_node('localhost', :worker_2_port);
|
SELECT 1 from master_remove_node('localhost', :worker_2_port);
|
||||||
|
|
||||||
|
@ -269,5 +250,4 @@ SELECT count(*) FROM citus.pg_dist_object WHERE objid IN (SELECT oid FROM pg_ext
|
||||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname IN ('seg', 'isn')$$);
|
SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname IN ('seg', 'isn')$$);
|
||||||
|
|
||||||
-- drop the schema and all the objects
|
-- drop the schema and all the objects
|
||||||
SET client_min_messages TO WARNING;
|
|
||||||
DROP SCHEMA "extension'test" CASCADE;
|
DROP SCHEMA "extension'test" CASCADE;
|
||||||
|
|
Loading…
Reference in New Issue