Deprecate citus.replicate_reference_tables_on_activate, make it always off (#6474)

Co-authored-by: Marco Slot <marco.slot@gmail.com>
pull/6477/head
Marco Slot 2022-11-04 16:21:10 +01:00 committed by GitHub
parent b8c7a9844c
commit 666696c01c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
67 changed files with 350 additions and 2029 deletions

View File

@ -146,7 +146,6 @@
#include "distributed/intermediate_result_pruning.h"
#include "distributed/listutils.h"
#include "distributed/local_executor.h"
#include "distributed/multi_client_executor.h"
#include "distributed/multi_executor.h"
#include "distributed/multi_explain.h"
#include "distributed/multi_partitioning_utils.h"

View File

@ -26,7 +26,6 @@
#include "distributed/listutils.h"
#include "distributed/metadata_utility.h"
#include "distributed/metadata_cache.h"
#include "distributed/multi_client_executor.h"
#include "distributed/multi_executor.h"
#include "distributed/remote_commands.h"
#include "distributed/transmit.h"
@ -84,6 +83,15 @@ typedef struct RemoteFileDestReceiver
uint64 bytesSent;
} RemoteFileDestReceiver;
/* Enumeration to track one copy query's status on the client */
typedef enum CopyStatus
{
CLIENT_INVALID_COPY = 0,
CLIENT_COPY_MORE = 1,
CLIENT_COPY_FAILED = 2,
CLIENT_COPY_DONE = 3
} CopyStatus;
static void RemoteFileDestReceiverStartup(DestReceiver *dest, int operation,
TupleDesc inputTupleDescriptor);

View File

@ -1,378 +0,0 @@
/*-------------------------------------------------------------------------
*
* multi_client_executor.c
*
* This file contains the libpq-specific parts of executing queries on remote
* nodes.
*
* Copyright (c) Citus Data, Inc.
*
* $Id$
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "fmgr.h"
#include "libpq-fe.h"
#include "miscadmin.h"
#include "commands/dbcommands.h"
#include "distributed/metadata_cache.h"
#include "distributed/connection_management.h"
#include "distributed/multi_executor.h"
#include "distributed/multi_client_executor.h"
#include "distributed/multi_server_executor.h"
#include "distributed/placement_connection.h"
#include "distributed/remote_commands.h"
#include "distributed/subplan_execution.h"
#include <errno.h>
#include <unistd.h>
#ifdef HAVE_POLL_H
#include <poll.h>
#endif
/* Local pool to track active connections */
static MultiConnection *ClientConnectionArray[MAX_CONNECTION_COUNT];
/*
* The value at any position on ClientPollingStatusArray is only defined when
* the corresponding ClientConnectionArray entry exists.
*/
static PostgresPollingStatusType ClientPollingStatusArray[MAX_CONNECTION_COUNT];
/* AllocateConnectionId returns a connection id from the connection pool. */
static int32
AllocateConnectionId(void)
{
int32 connectionId = INVALID_CONNECTION_ID;
/* allocate connectionId from connection pool */
for (int32 connIndex = 0; connIndex < MAX_CONNECTION_COUNT; connIndex++)
{
MultiConnection *connection = ClientConnectionArray[connIndex];
if (connection == NULL)
{
connectionId = connIndex;
break;
}
}
return connectionId;
}
/*
* MultiClientConnect synchronously tries to establish a connection. If it
* succeeds, it returns the connection id. Otherwise, it reports connection
* error and returns INVALID_CONNECTION_ID.
*
* nodeDatabase and userName can be NULL, in which case values from the
* current session are used.
*/
int32
MultiClientConnect(const char *nodeName, uint32 nodePort, const char *nodeDatabase,
const char *userName)
{
int32 connectionId = AllocateConnectionId();
int connectionFlags = FORCE_NEW_CONNECTION; /* no cached connections for now */
if (connectionId == INVALID_CONNECTION_ID)
{
ereport(WARNING, (errmsg("could not allocate connection in connection pool")));
return connectionId;
}
if (XactModificationLevel > XACT_MODIFICATION_NONE)
{
ereport(ERROR, (errcode(ERRCODE_ACTIVE_SQL_TRANSACTION),
errmsg("cannot open new connections after the first modification "
"command within a transaction")));
}
/* establish synchronous connection to worker node */
MultiConnection *connection = GetNodeUserDatabaseConnection(connectionFlags, nodeName,
nodePort,
userName, nodeDatabase);
ConnStatusType connStatusType = PQstatus(connection->pgConn);
if (connStatusType == CONNECTION_OK)
{
ClientConnectionArray[connectionId] = connection;
}
else
{
ReportConnectionError(connection, WARNING);
CloseConnection(connection);
connectionId = INVALID_CONNECTION_ID;
}
return connectionId;
}
/* MultiClientDisconnect disconnects the connection. */
void
MultiClientDisconnect(int32 connectionId)
{
const int InvalidPollingStatus = -1;
Assert(connectionId != INVALID_CONNECTION_ID);
MultiConnection *connection = ClientConnectionArray[connectionId];
Assert(connection != NULL);
CloseConnection(connection);
ClientConnectionArray[connectionId] = NULL;
ClientPollingStatusArray[connectionId] = InvalidPollingStatus;
}
/* MultiClientSendQuery sends the given query over the given connection. */
bool
MultiClientSendQuery(int32 connectionId, const char *query)
{
bool success = true;
Assert(connectionId != INVALID_CONNECTION_ID);
MultiConnection *connection = ClientConnectionArray[connectionId];
Assert(connection != NULL);
int querySent = SendRemoteCommand(connection, query);
if (querySent == 0)
{
char *errorMessage = pchomp(PQerrorMessage(connection->pgConn));
/*
* query might include the user query coming from the taskTracker
* code path, that's why we hash it, too. Otherwise, this code
* path is generally exercised for the kind of errors that
* we cannot send the queries that Citus itself produced.
*/
ereport(WARNING, (errmsg("could not send remote query \"%s\"",
query),
errdetail("Client error: %s",
errorMessage)));
success = false;
}
return success;
}
/* MultiClientResultStatus checks result status for an asynchronous query. */
ResultStatus
MultiClientResultStatus(int32 connectionId)
{
ResultStatus resultStatus = CLIENT_INVALID_RESULT_STATUS;
Assert(connectionId != INVALID_CONNECTION_ID);
MultiConnection *connection = ClientConnectionArray[connectionId];
Assert(connection != NULL);
ConnStatusType connStatusType = PQstatus(connection->pgConn);
if (connStatusType == CONNECTION_BAD)
{
ereport(WARNING, (errmsg("could not maintain connection to worker node")));
return CLIENT_RESULT_UNAVAILABLE;
}
/* consume input to allow status change */
int consumed = PQconsumeInput(connection->pgConn);
if (consumed != 0)
{
int connectionBusy = PQisBusy(connection->pgConn);
if (connectionBusy == 0)
{
resultStatus = CLIENT_RESULT_READY;
}
else
{
resultStatus = CLIENT_RESULT_BUSY;
}
}
else
{
ereport(WARNING, (errmsg("could not consume data from worker node")));
resultStatus = CLIENT_RESULT_UNAVAILABLE;
}
return resultStatus;
}
/* MultiClientQueryStatus returns the query status. */
QueryStatus
MultiClientQueryStatus(int32 connectionId)
{
int tupleCount PG_USED_FOR_ASSERTS_ONLY = 0;
bool copyResults = false;
QueryStatus queryStatus = CLIENT_INVALID_QUERY;
bool raiseInterrupts = true;
Assert(connectionId != INVALID_CONNECTION_ID);
MultiConnection *connection = ClientConnectionArray[connectionId];
Assert(connection != NULL);
ConnStatusType connStatusType = PQstatus(connection->pgConn);
if (connStatusType == CONNECTION_BAD)
{
ereport(WARNING, (errmsg("could not maintain connection to worker node")));
return CLIENT_QUERY_FAILED;
}
/*
* We now read the result object and check its status. If the result object
* isn't ready yet (the caller didn't wait for the connection to be ready),
* we will block on this call.
*/
PGresult *result = GetRemoteCommandResult(connection, raiseInterrupts);
ExecStatusType resultStatus = PQresultStatus(result);
if (resultStatus == PGRES_COMMAND_OK)
{
queryStatus = CLIENT_QUERY_DONE;
}
else if (resultStatus == PGRES_TUPLES_OK)
{
queryStatus = CLIENT_QUERY_DONE;
/*
* We use the client executor to only issue a select query that returns
* a void value. We therefore should not have more than one value here.
*/
tupleCount = PQntuples(result);
Assert(tupleCount <= 1);
}
else if (resultStatus == PGRES_COPY_OUT)
{
queryStatus = CLIENT_QUERY_COPY;
copyResults = true;
}
else
{
queryStatus = CLIENT_QUERY_FAILED;
if (resultStatus == PGRES_COPY_IN)
{
copyResults = true;
}
ReportResultError(connection, result, WARNING);
}
/* clear the result object */
PQclear(result);
/*
* When using the async query mechanism, we need to keep reading results
* until we get null. The exception to this rule is the copy protocol.
*/
if (!copyResults)
{
ForgetResults(connection);
}
return queryStatus;
}
/* MultiClientCopyData copies data from the file. */
CopyStatus
MultiClientCopyData(int32 connectionId, int32 fileDescriptor, uint64 *returnBytesReceived)
{
char *receiveBuffer = NULL;
const int asynchronous = 1;
CopyStatus copyStatus = CLIENT_INVALID_COPY;
Assert(connectionId != INVALID_CONNECTION_ID);
MultiConnection *connection = ClientConnectionArray[connectionId];
Assert(connection != NULL);
/*
* Consume input to handle the case where previous copy operation might have
* received zero bytes.
*/
int consumed = PQconsumeInput(connection->pgConn);
if (consumed == 0)
{
ereport(WARNING, (errmsg("could not read data from worker node")));
return CLIENT_COPY_FAILED;
}
/* receive copy data message in an asynchronous manner */
int receiveLength = PQgetCopyData(connection->pgConn, &receiveBuffer, asynchronous);
while (receiveLength > 0)
{
/* received copy data; append these data to file */
errno = 0;
if (returnBytesReceived)
{
*returnBytesReceived += receiveLength;
}
int appended = write(fileDescriptor, receiveBuffer, receiveLength);
if (appended != receiveLength)
{
/* if write didn't set errno, assume problem is no disk space */
if (errno == 0)
{
errno = ENOSPC;
}
ereport(FATAL, (errcode_for_file_access(),
errmsg("could not append to copied file: %m")));
}
PQfreemem(receiveBuffer);
receiveLength = PQgetCopyData(connection->pgConn, &receiveBuffer, asynchronous);
}
/* we now check the last received length returned by copy data */
if (receiveLength == 0)
{
/* we cannot read more data without blocking */
copyStatus = CLIENT_COPY_MORE;
}
else if (receiveLength == -1)
{
/* received copy done message */
bool raiseInterrupts = true;
PGresult *result = GetRemoteCommandResult(connection, raiseInterrupts);
ExecStatusType resultStatus = PQresultStatus(result);
if (resultStatus == PGRES_COMMAND_OK)
{
copyStatus = CLIENT_COPY_DONE;
}
else
{
copyStatus = CLIENT_COPY_FAILED;
ReportResultError(connection, result, WARNING);
}
PQclear(result);
}
else if (receiveLength == -2)
{
/* received an error */
copyStatus = CLIENT_COPY_FAILED;
ReportConnectionError(connection, WARNING);
}
/* if copy out completed, make sure we drain all results from libpq */
if (receiveLength < 0)
{
ForgetResults(connection);
}
return copyStatus;
}

View File

@ -21,7 +21,6 @@
#include "distributed/listutils.h"
#include "distributed/log_utils.h"
#include "distributed/multi_client_executor.h"
#include "distributed/multi_executor.h"
#include "distributed/multi_physical_planner.h"
#include "distributed/multi_server_executor.h"

View File

@ -121,7 +121,6 @@ static List * GenerateGrantOnFunctionQueriesFromAclItem(Oid schemaOid,
static List * GrantOnSequenceDDLCommands(Oid sequenceOid);
static List * GenerateGrantOnSequenceQueriesFromAclItem(Oid sequenceOid,
AclItem *aclItem);
static void SetLocalReplicateReferenceTablesOnActivate(bool state);
static char * GenerateSetRoleQuery(Oid roleOid);
static void MetadataSyncSigTermHandler(SIGNAL_ARGS);
static void MetadataSyncSigAlrmHandler(SIGNAL_ARGS);
@ -193,14 +192,9 @@ start_metadata_sync_to_node(PG_FUNCTION_ARGS)
char *nodeNameString = text_to_cstring(nodeName);
bool prevReplicateRefTablesOnActivate = ReplicateReferenceTablesOnActivate;
SetLocalReplicateReferenceTablesOnActivate(false);
ActivateNode(nodeNameString, nodePort);
TransactionModifiedNodeMetadata = true;
SetLocalReplicateReferenceTablesOnActivate(prevReplicateRefTablesOnActivate);
PG_RETURN_VOID();
}
@ -220,14 +214,9 @@ start_metadata_sync_to_all_nodes(PG_FUNCTION_ARGS)
List *workerNodes = ActivePrimaryNonCoordinatorNodeList(RowShareLock);
bool prevReplicateRefTablesOnActivate = ReplicateReferenceTablesOnActivate;
SetLocalReplicateReferenceTablesOnActivate(false);
ActivateNodeList(workerNodes);
TransactionModifiedNodeMetadata = true;
SetLocalReplicateReferenceTablesOnActivate(prevReplicateRefTablesOnActivate);
PG_RETURN_BOOL(true);
}
@ -2470,20 +2459,6 @@ SetLocalEnableMetadataSync(bool state)
}
/*
* SetLocalReplicateReferenceTablesOnActivate sets the
* replicate_reference_tables_on_activate locally
*/
void
SetLocalReplicateReferenceTablesOnActivate(bool state)
{
set_config_option("citus.replicate_reference_tables_on_activate",
state == true ? "on" : "off",
(superuser() ? PGC_SUSET : PGC_USERSET), PGC_S_SESSION,
GUC_ACTION_LOCAL, true, 0, false);
}
static char *
GenerateSetRoleQuery(Oid roleOid)
{

View File

@ -68,12 +68,6 @@ int GroupSize = 1;
/* config variable managed via guc.c */
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;
@ -1230,22 +1224,6 @@ ActivateNodeList(List *nodeList)
*/
SyncDistributedObjectsToNodeList(nodeToSyncMetadata);
if (ReplicateReferenceTablesOnActivate)
{
foreach_ptr(node, nodeList)
{
/*
* We need to replicate reference tables before syncing node metadata, otherwise
* reference table replication logic would try to get lock on the new node before
* having the shard placement on it
*/
if (NodeIsPrimary(node))
{
ReplicateAllReferenceTablesToNode(node);
}
}
}
/*
* Sync node metadata. We must sync node metadata before syncing table
* related pg_dist_xxx metadata. Since table related metadata requires

View File

@ -17,7 +17,6 @@
#include "distributed/backend_data.h"
#include "distributed/connection_management.h"
#include "distributed/metadata_cache.h"
#include "distributed/multi_client_executor.h"
#include "distributed/multi_server_executor.h"
#include "distributed/remote_commands.h"
#include "distributed/utils/array_type.h"

View File

@ -35,7 +35,6 @@
#include "distributed/local_executor.h"
#include "distributed/coordinator_protocol.h"
#include "distributed/metadata_sync.h"
#include "distributed/multi_client_executor.h"
#include "distributed/multi_join_order.h"
#include "distributed/multi_logical_planner.h"
#include "distributed/multi_physical_planner.h"

View File

@ -32,7 +32,6 @@
#include "distributed/coordinator_protocol.h"
#include "distributed/metadata_cache.h"
#include "distributed/metadata_sync.h"
#include "distributed/multi_client_executor.h"
#include "distributed/multi_executor.h"
#include "distributed/multi_physical_planner.h"
#include "distributed/multi_server_executor.h"

View File

@ -38,7 +38,6 @@
#include "distributed/coordinator_protocol.h"
#include "distributed/metadata_cache.h"
#include "distributed/metadata_utility.h"
#include "distributed/multi_client_executor.h"
#include "distributed/multi_logical_replication.h"
#include "distributed/multi_progress.h"
#include "distributed/multi_server_executor.h"

View File

@ -120,9 +120,6 @@ static void EnsureEnoughDiskSpaceForShardMove(List *colocatedShardList,
static List * RecreateShardDDLCommandList(ShardInterval *shardInterval,
const char *sourceNodeName,
int32 sourceNodePort);
static List * CopyShardContentsCommandList(ShardInterval *shardInterval,
const char *sourceNodeName,
int32 sourceNodePort);
static List * PostLoadShardCreationCommandList(ShardInterval *shardInterval,
const char *sourceNodeName,
int32 sourceNodePort);
@ -1577,31 +1574,6 @@ SearchShardPlacementInListOrError(List *shardPlacementList, const char *nodeName
}
/*
* CopyShardCommandList generates command list to copy the given shard placement
* from the source node to the target node. To do this it recreates the shard
* on the target, and then copies the data. Caller could optionally skip
* copying the data by the flag includeDataCopy.
*/
List *
CopyShardCommandList(ShardInterval *shardInterval, const char *sourceNodeName,
int32 sourceNodePort, bool includeDataCopy)
{
List *copyShardToNodeCommandsList = RecreateShardDDLCommandList(
shardInterval, sourceNodeName, sourceNodePort);
if (includeDataCopy)
{
copyShardToNodeCommandsList = list_concat(
copyShardToNodeCommandsList,
CopyShardContentsCommandList(shardInterval, sourceNodeName,
sourceNodePort));
}
return list_concat(copyShardToNodeCommandsList,
PostLoadShardCreationCommandList(shardInterval, sourceNodeName,
sourceNodePort));
}
/*
* RecreateShardDDLCommandList generates a command list to recreate a shard,
* but without any data init and without the post-load table creation commands.
@ -1618,28 +1590,6 @@ RecreateShardDDLCommandList(ShardInterval *shardInterval, const char *sourceNode
}
/*
* CopyShardContentsCommandList generates a command list to copy the data of the
* given shard placement from the source node to the target node. This copying
* requires a precreated table for the shard on the target node to have been
* created already (using RecreateShardDDLCommandList).
*/
static List *
CopyShardContentsCommandList(ShardInterval *shardInterval, const char *sourceNodeName,
int32 sourceNodePort)
{
char *shardName = ConstructQualifiedShardName(shardInterval);
StringInfo copyShardDataCommand = makeStringInfo();
appendStringInfo(copyShardDataCommand, WORKER_APPEND_TABLE_TO_SHARD,
quote_literal_cstr(shardName), /* table to append */
quote_literal_cstr(shardName), /* remote table name */
quote_literal_cstr(sourceNodeName), /* remote host */
sourceNodePort); /* remote port */
return list_make1(copyShardDataCommand->data);
}
/*
* PostLoadShardCreationCommandList generates a command list to finalize the
* creation of a shard after the data has been loaded. This creates stuff like

View File

@ -36,7 +36,6 @@
#include "distributed/hash_helpers.h"
#include "distributed/listutils.h"
#include "distributed/lock_graph.h"
#include "distributed/multi_client_executor.h"
#include "distributed/multi_executor.h"
#include "distributed/metadata_utility.h"
#include "distributed/coordinator_protocol.h"

View File

@ -19,7 +19,6 @@
#include "distributed/hash_helpers.h"
#include "distributed/listutils.h"
#include "distributed/metadata_cache.h"
#include "distributed/multi_client_executor.h"
#include "distributed/worker_manager.h"
#include "libpq/hba.h"
#include "common/ip.h"

View File

@ -32,7 +32,6 @@
#include "distributed/insert_select_planner.h"
#include "distributed/insert_select_executor.h"
#include "distributed/listutils.h"
#include "distributed/multi_client_executor.h"
#include "distributed/multi_executor.h"
#include "distributed/multi_explain.h"
#include "distributed/multi_logical_optimizer.h"

View File

@ -151,6 +151,7 @@ static char *DeprecatedEmptyString = "";
static char *MitmfifoEmptyString = "";
static bool DeprecatedDeferShardDeleteOnMove = true;
static bool DeprecatedDeferShardDeleteOnSplit = true;
static bool DeprecatedReplicateReferenceTablesOnActivate = false;
/* deprecated GUC value that should not be used anywhere outside this file */
static int ReplicationModel = REPLICATION_MODEL_STREAMING;
@ -2058,11 +2059,12 @@ RegisterCitusConfigVariables(void)
GUC_STANDARD | GUC_NO_SHOW_ALL,
NULL, NULL, NULL);
/* deprecated setting */
DefineCustomBoolVariable(
"citus.replicate_reference_tables_on_activate",
NULL,
NULL,
&ReplicateReferenceTablesOnActivate,
&DeprecatedReplicateReferenceTablesOnActivate,
true,
PGC_USERSET,
GUC_NO_SHOW_ALL,

View File

@ -1,6 +1,6 @@
-- citus--11.1-1--11.2-1
-- bump version to 11.2-1
DROP FUNCTION pg_catalog.worker_append_table_to_shard(text, text, text, integer);
#include "udfs/get_rebalance_progress/11.2-1.sql"
#include "udfs/citus_isolation_test_session_is_blocked/11.2-1.sql"

View File

@ -10,3 +10,10 @@ DROP SEQUENCE pg_catalog.pg_dist_clock_logical_seq;
DROP OPERATOR CLASS pg_catalog.cluster_clock_ops USING btree CASCADE;
DROP OPERATOR FAMILY pg_catalog.cluster_clock_ops USING btree CASCADE;
DROP TYPE pg_catalog.cluster_clock CASCADE;
CREATE FUNCTION pg_catalog.worker_append_table_to_shard(text, text, text, integer)
RETURNS void
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$worker_append_table_to_shard$$;
COMMENT ON FUNCTION pg_catalog.worker_append_table_to_shard(text, text, text, integer)
IS 'append a regular table''s contents to the shard';

View File

@ -45,9 +45,6 @@ static StringInfo CopyShardPlacementToWorkerNodeQuery(
ShardPlacement *sourceShardPlacement,
WorkerNode *workerNode,
char transferMode);
static void ReplicateReferenceTableShardToNode(ShardInterval *shardInterval,
char *nodeName,
int nodePort);
static bool AnyRelationsModifiedInTransaction(List *relationIdList);
static List * ReplicatedMetadataSyncedDistributedTableList(void);
static bool NodeHasAllReferenceTableReplicas(WorkerNode *workerNode);
@ -447,56 +444,6 @@ upgrade_to_reference_table(PG_FUNCTION_ARGS)
}
/*
* ReplicateShardToNode function replicates given shard to the given worker node
* in a separate transaction. If the worker already has
* a replica of the shard this is a no-op. This function also modifies metadata
* by inserting/updating related rows in pg_dist_placement.
*
* IMPORTANT: This should only be used to replicate shards of a reference
* table.
*/
static void
ReplicateReferenceTableShardToNode(ShardInterval *shardInterval, char *nodeName,
int nodePort)
{
uint64 shardId = shardInterval->shardId;
bool missingOk = false;
ShardPlacement *sourceShardPlacement = ActiveShardPlacement(shardId, missingOk);
char *srcNodeName = sourceShardPlacement->nodeName;
uint32 srcNodePort = sourceShardPlacement->nodePort;
bool includeDataCopy = true; /* TODO: consider using logical replication */
List *ddlCommandList = CopyShardCommandList(shardInterval, srcNodeName, srcNodePort,
includeDataCopy);
ereport(NOTICE, (errmsg("Replicating reference table \"%s\" to the node %s:%d",
get_rel_name(shardInterval->relationId), nodeName,
nodePort)));
/* send commands to new workers, the current user should be a superuser */
Assert(superuser());
WorkerNode *workerNode = FindWorkerNode(nodeName, nodePort);
SendMetadataCommandListToWorkerListInCoordinatedTransaction(list_make1(workerNode),
CurrentUserName(),
ddlCommandList);
int32 groupId = GroupForNode(nodeName, nodePort);
uint64 placementId = GetNextPlacementId();
InsertShardPlacementRow(shardId, placementId, SHARD_STATE_ACTIVE, 0,
groupId);
if (ShouldSyncTableMetadata(shardInterval->relationId))
{
char *placementCommand = PlacementUpsertCommand(shardId, placementId,
SHARD_STATE_ACTIVE, 0,
groupId);
SendCommandToWorkersWithMetadata(placementCommand);
}
}
/*
* CreateReferenceTableColocationId creates a new co-location id for reference tables and
* writes it into pg_dist_colocation, then returns the created co-location id. Since there
@ -659,95 +606,6 @@ CompareOids(const void *leftElement, const void *rightElement)
}
/*
* 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(WorkerNode *workerNode)
{
int colocationId = GetReferenceTableColocationId();
if (colocationId == INVALID_COLOCATION_ID)
{
/* no reference tables in system */
return;
}
/* prevent changes in table set while replicating reference tables */
LockColocationId(colocationId, RowExclusiveLock);
List *referenceTableList = CitusTableTypeIdList(REFERENCE_TABLE);
/* 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);
List *shardPlacementList =
ShardPlacementListIncludingOrphanedPlacements(shardInterval->shardId);
ShardPlacement *targetPlacement =
SearchShardPlacementInList(shardPlacementList,
workerNode->workerName,
workerNode->workerPort);
if (targetPlacement != NULL &&
targetPlacement->shardState == SHARD_STATE_ACTIVE)
{
/* We already have the shard, nothing to do */
continue;
}
referenceShardIntervalList = lappend(referenceShardIntervalList,
shardInterval);
}
if (ClusterHasKnownMetadataWorkers())
{
BlockWritesToShardList(referenceShardIntervalList);
}
ShardInterval *shardInterval = NULL;
foreach_ptr(shardInterval, referenceShardIntervalList)
{
uint64 shardId = shardInterval->shardId;
LockShardDistributionMetadata(shardId, ExclusiveLock);
ReplicateReferenceTableShardToNode(shardInterval,
workerNode->workerName,
workerNode->workerPort);
}
/* create foreign constraints between reference tables */
foreach_ptr(shardInterval, referenceShardIntervalList)
{
List *commandList = CopyShardForeignConstraintCommandList(shardInterval);
/* send commands to new workers, the current user should be a superuser */
Assert(superuser());
SendMetadataCommandListToWorkerListInCoordinatedTransaction(
list_make1(workerNode),
CurrentUserName(),
commandList);
}
}
}
/*
* ErrorIfNotAllNodesHaveReferenceTableReplicas throws an error when one of the
* nodes in the list does not have reference table replicas.

View File

@ -20,7 +20,6 @@
#include "distributed/listutils.h"
#include "distributed/local_executor.h"
#include "distributed/metadata_cache.h"
#include "distributed/multi_client_executor.h"
#include "distributed/multi_executor.h"
#include "distributed/multi_physical_planner.h"
#include "distributed/multi_server_executor.h"

View File

@ -38,7 +38,6 @@
#include "distributed/listutils.h"
#include "distributed/metadata_cache.h"
#include "distributed/metadata_sync.h"
#include "distributed/multi_client_executor.h"
#include "distributed/multi_logical_optimizer.h"
#include "distributed/multi_partitioning_utils.h"
#include "distributed/multi_server_executor.h"
@ -62,13 +61,6 @@
/* Local functions forward declarations */
static bool ReceiveRegularFile(const char *nodeName, uint32 nodePort,
const char *nodeUser, StringInfo transmitCommand,
StringInfo filePath);
static void ReceiveResourceCleanup(int32 connectionId, const char *filename,
int32 fileDescriptor);
static CopyStmt * CopyStatement(RangeVar *relation, char *sourceFilename);
static void CitusDeleteFile(const char *filename);
static bool check_log_statement(List *stmt_list);
static void AlterSequenceMinMax(Oid sequenceId, char *schemaName, char *sequenceName,
Oid sequenceTypeId);
@ -82,175 +74,6 @@ PG_FUNCTION_INFO_V1(worker_append_table_to_shard);
PG_FUNCTION_INFO_V1(worker_nextval);
/*
* ReceiveRegularFile creates a local file at the given file path, and connects
* to remote database that has the given node name and port number. The function
* then issues the given transmit command using client-side logic (libpq), reads
* the remote file's contents, and appends these contents to the local file. On
* success, the function returns success; on failure, it cleans up all resources
* and returns false.
*/
static bool
ReceiveRegularFile(const char *nodeName, uint32 nodePort, const char *nodeUser,
StringInfo transmitCommand, StringInfo filePath)
{
char filename[MAXPGPATH];
const int fileFlags = (O_APPEND | O_CREAT | O_RDWR | O_TRUNC | PG_BINARY);
const int fileMode = (S_IRUSR | S_IWUSR);
bool queryReady = false;
bool copyDone = false;
/* create local file to append remote data to */
strlcpy(filename, filePath->data, MAXPGPATH);
int32 fileDescriptor = BasicOpenFilePerm(filename, fileFlags, fileMode);
if (fileDescriptor < 0)
{
ereport(WARNING, (errcode_for_file_access(),
errmsg("could not open file \"%s\": %m", filePath->data)));
return false;
}
/* we use the same database name on the master and worker nodes */
const char *nodeDatabase = CurrentDatabaseName();
/* connect to remote node */
int32 connectionId = MultiClientConnect(nodeName, nodePort, nodeDatabase, nodeUser);
if (connectionId == INVALID_CONNECTION_ID)
{
ReceiveResourceCleanup(connectionId, filename, fileDescriptor);
return false;
}
/* send request to remote node to start transmitting data */
bool querySent = MultiClientSendQuery(connectionId, transmitCommand->data);
if (!querySent)
{
ReceiveResourceCleanup(connectionId, filename, fileDescriptor);
return false;
}
/* loop until the remote node acknowledges our transmit request */
while (!queryReady)
{
ResultStatus resultStatus = MultiClientResultStatus(connectionId);
if (resultStatus == CLIENT_RESULT_READY)
{
queryReady = true;
}
else if (resultStatus == CLIENT_RESULT_BUSY)
{
/* remote node did not respond; wait for longer */
long sleepIntervalPerCycle = RemoteTaskCheckInterval * 1000L;
pg_usleep(sleepIntervalPerCycle);
}
else
{
ReceiveResourceCleanup(connectionId, filename, fileDescriptor);
return false;
}
}
/* check query response is as expected */
QueryStatus queryStatus = MultiClientQueryStatus(connectionId);
if (queryStatus != CLIENT_QUERY_COPY)
{
ReceiveResourceCleanup(connectionId, filename, fileDescriptor);
return false;
}
/* loop until we receive and append all the data from remote node */
while (!copyDone)
{
CopyStatus copyStatus = MultiClientCopyData(connectionId, fileDescriptor, NULL);
if (copyStatus == CLIENT_COPY_DONE)
{
copyDone = true;
}
else if (copyStatus == CLIENT_COPY_MORE)
{
/* remote node will continue to send more data */
}
else
{
ReceiveResourceCleanup(connectionId, filename, fileDescriptor);
return false;
}
}
/* we are done executing; release the connection and the file handle */
MultiClientDisconnect(connectionId);
int closed = close(fileDescriptor);
if (closed < 0)
{
ereport(WARNING, (errcode_for_file_access(),
errmsg("could not close file \"%s\": %m", filename)));
/* if we failed to close file, try to delete it before erroring out */
CitusDeleteFile(filename);
return false;
}
/* we successfully received the remote file */
ereport(DEBUG2, (errmsg("received remote file \"%s\"", filename)));
return true;
}
/*
* ReceiveResourceCleanup gets called if an error occurs during file receiving.
* The function closes the connection, and closes and deletes the local file.
*/
static void
ReceiveResourceCleanup(int32 connectionId, const char *filename, int32 fileDescriptor)
{
if (connectionId != INVALID_CONNECTION_ID)
{
MultiClientDisconnect(connectionId);
}
if (fileDescriptor != -1)
{
int closed = close(fileDescriptor);
if (closed < 0)
{
ereport(WARNING, (errcode_for_file_access(),
errmsg("could not close file \"%s\": %m", filename)));
}
int deleted = unlink(filename);
if (deleted != 0)
{
ereport(WARNING, (errcode_for_file_access(),
errmsg("could not delete file \"%s\": %m", filename)));
}
}
}
/* Deletes file with the given filename. */
static void
CitusDeleteFile(const char *filename)
{
int deleted = unlink(filename);
if (deleted != 0)
{
ereport(WARNING, (errcode_for_file_access(),
errmsg("could not delete file \"%s\": %m", filename)));
}
}
/*
* worker_apply_shard_ddl_command extends table, index, or constraint names in
* the given DDL command. The function then applies this extended DDL command
@ -454,150 +277,12 @@ ParseTreeRawStmt(const char *ddlCommand)
/*
* worker_append_table_to_shard fetches the given remote table's data into the
* local file system. The function then appends this file data into the given
* shard.
* worker_append_table_to_shard is deprecated.
*/
Datum
worker_append_table_to_shard(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
text *shardQualifiedNameText = PG_GETARG_TEXT_P(0);
text *sourceQualifiedNameText = PG_GETARG_TEXT_P(1);
text *sourceNodeNameText = PG_GETARG_TEXT_P(2);
uint32 sourceNodePort = PG_GETARG_UINT32(3);
List *shardQualifiedNameList = textToQualifiedNameList(shardQualifiedNameText);
List *sourceQualifiedNameList = textToQualifiedNameList(sourceQualifiedNameText);
char *sourceNodeName = text_to_cstring(sourceNodeNameText);
char *shardTableName = NULL;
char *shardSchemaName = NULL;
char *sourceSchemaName = NULL;
char *sourceTableName = NULL;
/* We extract schema names and table names from qualified names */
DeconstructQualifiedName(shardQualifiedNameList, &shardSchemaName, &shardTableName);
DeconstructQualifiedName(sourceQualifiedNameList, &sourceSchemaName,
&sourceTableName);
/*
* We lock on the shardId, but do not unlock. When the function returns, and
* the transaction for this function commits, this lock will automatically
* be released. This ensures appends to a shard happen in a serial manner.
*/
uint64 shardId = ExtractShardIdFromTableName(shardTableName, false);
LockShardResource(shardId, AccessExclusiveLock);
/*
* Copy into intermediate results directory, which is automatically cleaned on
* error.
*/
StringInfo localFilePath = makeStringInfo();
appendStringInfo(localFilePath, "%s/worker_append_table_to_shard_" UINT64_FORMAT,
CreateIntermediateResultsDirectory(), shardId);
char *sourceQualifiedName = quote_qualified_identifier(sourceSchemaName,
sourceTableName);
StringInfo sourceCopyCommand = makeStringInfo();
/*
* Partitioned tables do not support "COPY table TO STDOUT". Thus, we use
* "COPY (SELECT * FROM table) TO STDOUT" for partitioned tables.
*
* If the schema name is not explicitly set, we use the public schema.
*/
sourceSchemaName = sourceSchemaName ? sourceSchemaName : "public";
Oid sourceSchemaId = get_namespace_oid(sourceSchemaName, false);
Oid sourceShardRelationId = get_relname_relid(sourceTableName, sourceSchemaId);
if (PartitionedTableNoLock(sourceShardRelationId))
{
appendStringInfo(sourceCopyCommand, COPY_SELECT_ALL_OUT_COMMAND,
sourceQualifiedName);
}
else
{
appendStringInfo(sourceCopyCommand, COPY_OUT_COMMAND, sourceQualifiedName);
}
char *userName = CurrentUserName();
bool received = ReceiveRegularFile(sourceNodeName, sourceNodePort, userName,
sourceCopyCommand,
localFilePath);
if (!received)
{
ereport(ERROR, (errmsg("could not copy table \"%s\" from \"%s:%u\"",
sourceTableName, sourceNodeName, sourceNodePort)));
}
/* copy local file into the given shard */
RangeVar *localTable = makeRangeVar(shardSchemaName, shardTableName, -1);
CopyStmt *localCopyCommand = CopyStatement(localTable, localFilePath->data);
char *shardQualifiedName = quote_qualified_identifier(shardSchemaName,
shardTableName);
StringInfo queryString = makeStringInfo();
appendStringInfo(queryString, COPY_IN_COMMAND, shardQualifiedName,
localFilePath->data);
/* make sure we are allowed to execute the COPY command */
CheckCopyPermissions(localCopyCommand);
Relation shardRelation = table_openrv(localCopyCommand->relation, RowExclusiveLock);
/* mimic check from copy.c */
if (XactReadOnly && !shardRelation->rd_islocaltemp)
{
PreventCommandIfReadOnly("COPY FROM");
}
ParseState *parseState = make_parsestate(NULL);
(void) addRangeTableEntryForRelation(parseState, shardRelation, RowExclusiveLock,
NULL, false, false);
CopyFromState copyState = BeginCopyFrom_compat(parseState,
shardRelation,
NULL,
localCopyCommand->filename,
localCopyCommand->is_program,
NULL,
localCopyCommand->attlist,
localCopyCommand->options);
CopyFrom(copyState);
EndCopyFrom(copyState);
free_parsestate(parseState);
/* finally delete the temporary file we created */
CitusDeleteFile(localFilePath->data);
table_close(shardRelation, NoLock);
PG_RETURN_VOID();
}
/*
* CopyStatement creates and initializes a copy statement to read the given
* file's contents into the given table, using copy's standard text format.
*/
static CopyStmt *
CopyStatement(RangeVar *relation, char *sourceFilename)
{
CopyStmt *copyStatement = makeNode(CopyStmt);
copyStatement->relation = relation;
copyStatement->query = NULL;
copyStatement->attlist = NIL;
copyStatement->options = NIL;
copyStatement->is_from = true;
copyStatement->is_program = false;
copyStatement->filename = sourceFilename;
return copyStatement;
ereport(ERROR, (errmsg("worker_append_table_to_shard has been deprecated")));
}

View File

@ -60,8 +60,6 @@
"SELECT worker_apply_shard_ddl_command (" UINT64_FORMAT ", %s, %s)"
#define WORKER_APPLY_SHARD_DDL_COMMAND_WITHOUT_SCHEMA \
"SELECT worker_apply_shard_ddl_command (" UINT64_FORMAT ", %s)"
#define WORKER_APPEND_TABLE_TO_SHARD \
"SELECT worker_append_table_to_shard (%s, %s, %s, %u)"
#define WORKER_APPLY_INTER_SHARD_DDL_COMMAND \
"SELECT worker_apply_inter_shard_ddl_command (" UINT64_FORMAT ", %s, " UINT64_FORMAT \
", %s, %s)"
@ -287,9 +285,6 @@ extern Datum isolate_tenant_to_new_shard(PG_FUNCTION_ARGS);
extern Datum citus_split_shard_by_split_points(PG_FUNCTION_ARGS);
/* function declarations for shard copy functinality */
extern List * CopyShardCommandList(ShardInterval *shardInterval, const
char *sourceNodeName,
int32 sourceNodePort, bool includeData);
extern List * CopyShardForeignConstraintCommandList(ShardInterval *shardInterval);
extern void CopyShardForeignConstraintCommandListGrouped(ShardInterval *shardInterval,
List **

View File

@ -1,111 +0,0 @@
/*-------------------------------------------------------------------------
*
* multi_client_executor.h
* Type and function pointer declarations for executing client-side (libpq)
* logic.
*
* Copyright (c) Citus Data, Inc.
*
* $Id$
*
*-------------------------------------------------------------------------
*/
#ifndef MULTI_CLIENT_EXECUTOR_H
#define MULTI_CLIENT_EXECUTOR_H
#include "distributed/connection_management.h"
#include "nodes/pg_list.h"
#ifdef HAVE_POLL_H
#include <poll.h>
#endif
#define INVALID_CONNECTION_ID -1 /* identifies an invalid connection */
#define MAX_CONNECTION_COUNT 2048 /* simultaneous client connection count */
#define STRING_BUFFER_SIZE 1024 /* buffer size for character arrays */
/* Enumeration to track one client connection's status */
typedef enum
{
CLIENT_INVALID_CONNECT = 0,
CLIENT_CONNECTION_BAD = 1,
CLIENT_CONNECTION_BUSY = 2,
CLIENT_CONNECTION_BUSY_READ = 3,
CLIENT_CONNECTION_BUSY_WRITE = 4,
CLIENT_CONNECTION_READY = 5
} ConnectStatus;
/* Enumeration to see if we can read query results without blocking */
typedef enum
{
CLIENT_INVALID_RESULT_STATUS = 0,
CLIENT_RESULT_UNAVAILABLE = 1,
CLIENT_RESULT_BUSY = 2,
CLIENT_RESULT_READY = 3
} ResultStatus;
/* Enumeration to track one execution query's status on the client */
typedef enum
{
CLIENT_INVALID_QUERY = 0,
CLIENT_QUERY_FAILED = 1,
CLIENT_QUERY_DONE = 2,
CLIENT_QUERY_COPY = 3
} QueryStatus;
/* Enumeration to track one copy query's status on the client */
typedef enum
{
CLIENT_INVALID_COPY = 0,
CLIENT_COPY_MORE = 1,
CLIENT_COPY_FAILED = 2,
CLIENT_COPY_DONE = 3
} CopyStatus;
/* Enumeration to track the status of a query in a batch on the client */
typedef enum
{
CLIENT_INVALID_BATCH_QUERY = 0,
CLIENT_BATCH_QUERY_FAILED = 1,
CLIENT_BATCH_QUERY_CONTINUE = 2,
CLIENT_BATCH_QUERY_DONE = 3
} BatchQueryStatus;
struct pollfd; /* forward declared, to avoid having to include poll.h */
typedef struct WaitInfo
{
int maxWaiters;
#ifdef HAVE_POLL
struct pollfd *pollfds;
#else
fd_set readFileDescriptorSet;
fd_set writeFileDescriptorSet;
fd_set exceptionFileDescriptorSet;
int maxConnectionFileDescriptor;
#endif /* HAVE_POLL*/
int registeredWaiters;
bool haveReadyWaiter;
bool haveFailedWaiter;
} WaitInfo;
/* Function declarations for executing client-side (libpq) logic. */
extern int32 MultiClientConnect(const char *nodeName, uint32 nodePort,
const char *nodeDatabase, const char *nodeUser);
extern void MultiClientDisconnect(int32 connectionId);
extern bool MultiClientSendQuery(int32 connectionId, const char *query);
extern ResultStatus MultiClientResultStatus(int32 connectionId);
extern QueryStatus MultiClientQueryStatus(int32 connectionId);
extern CopyStatus MultiClientCopyData(int32 connectionId, int32 fileDescriptor,
uint64 *returnBytesReceived);
#endif /* MULTI_CLIENT_EXECUTOR_H */

View File

@ -61,7 +61,6 @@ typedef struct WorkerNode
extern int MaxWorkerNodesTracked;
extern char *WorkerListFileName;
extern char *CurrentCluster;
extern bool ReplicateReferenceTablesOnActivate;
extern void ActivateNodeList(List *nodeList);
extern int ActivateNode(char *nodeName, int nodePort);

View File

@ -28,12 +28,6 @@
/* Number of rows to prefetch when reading data with a cursor */
#define ROW_PREFETCH_COUNT 50
/* Defines used for fetching files and tables */
/* the tablename in the overloaded COPY statement is the to-be-transferred file */
#define COPY_OUT_COMMAND "COPY %s TO STDOUT"
#define COPY_SELECT_ALL_OUT_COMMAND "COPY (SELECT * FROM %s) TO STDOUT"
#define COPY_IN_COMMAND "COPY %s FROM '%s'"
/* Defines that relate to creating tables */
#define GET_TABLE_DDL_EVENTS "SELECT master_get_table_ddl_events('%s')"
#define SET_SEARCH_PATH_COMMAND "SET search_path TO %s"

View File

@ -11,7 +11,6 @@ SELECT citus.mitmproxy('conn.allow()');
(1 row)
SET citus.next_shard_id TO 200000;
SET citus.replicate_reference_tables_on_activate TO off;
-- verify we have all worker nodes present
SELECT * FROM master_get_active_worker_nodes()
ORDER BY 1, 2;

View File

@ -128,7 +128,6 @@ SELECT * FROM create_distributed_table('rep1', 'id');
-- Add the coordinator, so we can have a replicated shard
SELECT 1 FROM citus_add_node('localhost', :master_port, 0);
NOTICE: Replicating reference table "ref" to the node localhost:xxxxx
NOTICE: localhost:xxxxx is the coordinator and already contains metadata, skipping syncing the metadata
?column?
---------------------------------------------------------------------

View File

@ -357,7 +357,7 @@ t
(1 row)
starting permutation: s1-begin s2-begin s1-start-metadata-sync s2-create-dist-table s1-commit s2-commit s3-compare-snapshot
starting permutation: s1-begin s2-begin s1-start-metadata-sync s2-create-dist-table s1-commit s2-commit
step s1-begin:
BEGIN;
@ -388,30 +388,6 @@ create_distributed_table
step s2-commit:
COMMIT;
step s3-compare-snapshot:
SELECT count(*) = 0 AS same_metadata_in_workers
FROM
(
(
SELECT unnest(activate_node_snapshot())
EXCEPT
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
)
UNION
(
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
EXCEPT
SELECT unnest(activate_node_snapshot())
)
) AS foo;
same_metadata_in_workers
---------------------------------------------------------------------
t
(1 row)
starting permutation: s1-begin s2-begin s1-start-metadata-sync s2-create-ref-table s1-commit s2-commit s3-compare-snapshot
step s1-begin:

View File

@ -1,6 +1,6 @@
Parsed test spec with 4 sessions
starting permutation: add-node s1-begin s2-begin s1-update-dist-table s2-lock-ref-table-placement-on-coordinator s1-lock-ref-table-placement-on-coordinator s2-update-dist-table deadlock-checker-call s1-end s2-end
starting permutation: add-node replicate-reference-tables s1-begin s2-begin s1-update-dist-table s2-lock-ref-table-placement-on-coordinator s1-lock-ref-table-placement-on-coordinator s2-update-dist-table deadlock-checker-call s1-end s2-end
create_distributed_table
---------------------------------------------------------------------
@ -14,6 +14,14 @@ step add-node:
1
(1 row)
step replicate-reference-tables:
SELECT replicate_reference_tables(shard_transfer_mode := 'block_writes');
replicate_reference_tables
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
@ -67,7 +75,7 @@ master_remove_node
(1 row)
starting permutation: add-node s1-begin s2-begin s1-update-ref-table s2-sleep s2-view-dist s2-view-worker s2-end s1-end
starting permutation: add-node replicate-reference-tables s1-begin s2-begin s1-update-ref-table s2-sleep s2-view-dist s2-view-worker s2-end s1-end
create_distributed_table
---------------------------------------------------------------------
@ -81,6 +89,14 @@ step add-node:
1
(1 row)
step replicate-reference-tables:
SELECT replicate_reference_tables(shard_transfer_mode := 'block_writes');
replicate_reference_tables
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
@ -101,15 +117,18 @@ pg_sleep
step s2-view-dist:
SELECT query, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE backend_type = 'client backend' AND query NOT ILIKE ALL(VALUES('%pg_prepared_xacts%'), ('%COMMIT%'), ('%pg_isolation_test_session_is_blocked%'), ('%BEGIN%'), ('%add_node%')) ORDER BY query DESC;
query |state |wait_event_type|wait_event|usename |datname
query |state |wait_event_type|wait_event|usename |datname
---------------------------------------------------------------------
SELECT check_distributed_deadlocks();
SELECT replicate_reference_tables(shard_transfer_mode := 'block_writes');
|idle |Client |ClientRead|postgres|regression
SELECT check_distributed_deadlocks();
|idle |Client |ClientRead|postgres|regression
update ref_table set a = a + 1;
|idle in transaction|Client |ClientRead|postgres|regression
(2 rows)
|idle in transaction|Client |ClientRead|postgres|regression
(3 rows)
step s2-view-worker:
SELECT query, state, wait_event_type, wait_event, usename, datname
@ -146,7 +165,7 @@ master_remove_node
(1 row)
starting permutation: add-node s1-begin s2-begin s1-update-ref-table s2-active-transactions s1-end s2-end
starting permutation: add-node replicate-reference-tables s1-begin s2-begin s1-update-ref-table s2-active-transactions s1-end s2-end
create_distributed_table
---------------------------------------------------------------------
@ -160,6 +179,14 @@ step add-node:
1
(1 row)
step replicate-reference-tables:
SELECT replicate_reference_tables(shard_transfer_mode := 'block_writes');
replicate_reference_tables
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;

View File

@ -2,7 +2,6 @@ CREATE SCHEMA local_shard_copy;
SET search_path TO local_shard_copy;
SET client_min_messages TO DEBUG;
SET citus.next_shard_id TO 1570000;
SET citus.replicate_reference_tables_on_activate TO off;
SELECT 1 FROM master_add_node('localhost', :master_port, groupid := 0);
NOTICE: localhost:xxxxx is the coordinator and already contains metadata, skipping syncing the metadata
?column?

View File

@ -22,7 +22,6 @@ BEGIN;
SET client_min_messages TO ERROR;
SET search_path TO public;
CREATE EXTENSION citus;
SET citus.replicate_reference_tables_on_activate TO OFF;
SELECT 1 FROM master_add_node('localhost', :master_port, groupId => 0);
?column?
---------------------------------------------------------------------

View File

@ -1199,37 +1199,38 @@ SELECT * FROM multi_extension.print_extension_changes();
-- Snapshot of state at 11.2-1
ALTER EXTENSION citus UPDATE TO '11.2-1';
SELECT * FROM multi_extension.print_extension_changes();
previous_object | current_object
previous_object | current_object
---------------------------------------------------------------------
function get_rebalance_progress() TABLE(sessionid integer, table_name regclass, shardid bigint, shard_size bigint, sourcename text, sourceport integer, targetname text, targetport integer, progress bigint, source_shard_size bigint, target_shard_size bigint, operation_type text) |
| function citus_get_node_clock() cluster_clock
| function citus_get_transaction_clock() cluster_clock
| function citus_internal_adjust_local_clock_to_remote(cluster_clock) void
| function citus_is_clock_after(cluster_clock,cluster_clock) boolean
| function cluster_clock_cmp(cluster_clock,cluster_clock) integer
| function cluster_clock_eq(cluster_clock,cluster_clock) boolean
| function cluster_clock_ge(cluster_clock,cluster_clock) boolean
| function cluster_clock_gt(cluster_clock,cluster_clock) boolean
| function cluster_clock_in(cstring) cluster_clock
| function cluster_clock_le(cluster_clock,cluster_clock) boolean
| function cluster_clock_logical(cluster_clock) bigint
| function cluster_clock_lt(cluster_clock,cluster_clock) boolean
| function cluster_clock_ne(cluster_clock,cluster_clock) boolean
| function cluster_clock_out(cluster_clock) cstring
| function cluster_clock_recv(internal) cluster_clock
| function cluster_clock_send(cluster_clock) bytea
function worker_append_table_to_shard(text,text,text,integer) void |
| function citus_get_node_clock() cluster_clock
| function citus_get_transaction_clock() cluster_clock
| function citus_internal_adjust_local_clock_to_remote(cluster_clock) void
| function citus_is_clock_after(cluster_clock,cluster_clock) boolean
| function cluster_clock_cmp(cluster_clock,cluster_clock) integer
| function cluster_clock_eq(cluster_clock,cluster_clock) boolean
| function cluster_clock_ge(cluster_clock,cluster_clock) boolean
| function cluster_clock_gt(cluster_clock,cluster_clock) boolean
| function cluster_clock_in(cstring) cluster_clock
| function cluster_clock_le(cluster_clock,cluster_clock) boolean
| function cluster_clock_logical(cluster_clock) bigint
| function cluster_clock_lt(cluster_clock,cluster_clock) boolean
| function cluster_clock_ne(cluster_clock,cluster_clock) boolean
| function cluster_clock_out(cluster_clock) cstring
| function cluster_clock_recv(internal) cluster_clock
| function cluster_clock_send(cluster_clock) bytea
| function get_rebalance_progress() TABLE(sessionid integer, table_name regclass, shardid bigint, shard_size bigint, sourcename text, sourceport integer, targetname text, targetport integer, progress bigint, source_shard_size bigint, target_shard_size bigint, operation_type text, source_lsn pg_lsn, target_lsn pg_lsn, status text)
| operator <(cluster_clock,cluster_clock)
| operator <=(cluster_clock,cluster_clock)
| operator <>(cluster_clock,cluster_clock)
| operator =(cluster_clock,cluster_clock)
| operator >(cluster_clock,cluster_clock)
| operator >=(cluster_clock,cluster_clock)
| operator class cluster_clock_ops for access method btree
| operator family cluster_clock_ops for access method btree
| sequence pg_dist_clock_logical_seq
| type cluster_clock
(28 rows)
| operator <(cluster_clock,cluster_clock)
| operator <=(cluster_clock,cluster_clock)
| operator <>(cluster_clock,cluster_clock)
| operator =(cluster_clock,cluster_clock)
| operator >(cluster_clock,cluster_clock)
| operator >=(cluster_clock,cluster_clock)
| operator class cluster_clock_ops for access method btree
| operator family cluster_clock_ops for access method btree
| sequence pg_dist_clock_logical_seq
| type cluster_clock
(29 rows)
DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff;
-- show running version

View File

@ -30,7 +30,6 @@ NOTICE: dropping metadata on the node (localhost,57638)
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1310000;
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 2;
SET citus.replicate_reference_tables_on_activate TO off;
SELECT nextval('pg_catalog.pg_dist_placement_placementid_seq') AS last_placement_id
\gset
ALTER SEQUENCE pg_catalog.pg_dist_placement_placementid_seq RESTART 100000;
@ -1656,8 +1655,7 @@ ORDER BY shardid, nodeport;
shardid | nodename | nodeport
---------------------------------------------------------------------
1310073 | localhost | 57637
1310073 | localhost | 57638
(2 rows)
(1 row)
\c - - - :worker_1_port
SELECT shardid, nodename, nodeport
@ -1667,8 +1665,7 @@ ORDER BY shardid, nodeport;
shardid | nodename | nodeport
---------------------------------------------------------------------
1310073 | localhost | 57637
1310073 | localhost | 57638
(2 rows)
(1 row)
-- Get the metadata back into a consistent state
\c - - - :master_port

View File

@ -30,7 +30,6 @@ NOTICE: dropping metadata on the node (localhost,57638)
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1310000;
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 2;
SET citus.replicate_reference_tables_on_activate TO off;
SELECT nextval('pg_catalog.pg_dist_placement_placementid_seq') AS last_placement_id
\gset
ALTER SEQUENCE pg_catalog.pg_dist_placement_placementid_seq RESTART 100000;
@ -1656,8 +1655,7 @@ ORDER BY shardid, nodeport;
shardid | nodename | nodeport
---------------------------------------------------------------------
1310073 | localhost | 57637
1310073 | localhost | 57638
(2 rows)
(1 row)
\c - - - :worker_1_port
SELECT shardid, nodename, nodeport
@ -1667,8 +1665,7 @@ ORDER BY shardid, nodeport;
shardid | nodename | nodeport
---------------------------------------------------------------------
1310073 | localhost | 57637
1310073 | localhost | 57638
(2 rows)
(1 row)
-- Get the metadata back into a consistent state
\c - - - :master_port

View File

@ -533,41 +533,6 @@ SELECT create_distributed_table('full_access_user_schema.t2', 'id');
(1 row)
RESET ROLE;
\c - - - :worker_2_port
-- non-superuser should be able to use worker_append_table_to_shard on their own shard
SET ROLE full_access;
CREATE TABLE full_access_user_schema.source_table (id int);
INSERT INTO full_access_user_schema.source_table VALUES (1);
CREATE TABLE full_access_user_schema.shard_0 (id int);
SELECT worker_append_table_to_shard('full_access_user_schema.shard_0', 'full_access_user_schema.source_table', 'localhost', :worker_2_port);
worker_append_table_to_shard
---------------------------------------------------------------------
(1 row)
SELECT * FROM full_access_user_schema.shard_0;
id
---------------------------------------------------------------------
1
(1 row)
RESET ROLE;
-- other users should not be able to read from a table they have no access to via worker_append_table_to_shard
SET ROLE usage_access;
SELECT worker_append_table_to_shard('full_access_user_schema.shard_0', 'full_access_user_schema.source_table', 'localhost', :worker_2_port);
WARNING: permission denied for table source_table
CONTEXT: while executing command on localhost:xxxxx
ERROR: could not copy table "source_table" from "localhost:xxxxx"
RESET ROLE;
-- allow usage_access to read from table
GRANT SELECT ON full_access_user_schema.source_table TO usage_access;
-- other users should not be able to write to a table they do not have write access to
SET ROLE usage_access;
SELECT worker_append_table_to_shard('full_access_user_schema.shard_0', 'full_access_user_schema.source_table', 'localhost', :worker_2_port);
ERROR: permission denied for table shard_0
RESET ROLE;
DROP TABLE full_access_user_schema.source_table, full_access_user_schema.shard_0;
\c - - - :master_port
DROP SCHEMA full_access_user_schema CASCADE;
NOTICE: drop cascades to 2 other objects
DETAIL: drop cascades to table full_access_user_schema.t1

View File

@ -7,7 +7,6 @@ SELECT nextval('pg_catalog.pg_dist_colocationid_seq') AS last_colocation_id \gse
SELECT nextval('pg_catalog.pg_dist_shardid_seq') AS last_shard_id \gset
SET citus.shard_count TO 8;
SET citus.shard_replication_factor TO 1;
SET citus.replicate_reference_tables_on_activate TO off;
\set VERBOSITY terse
-- Simulates a readonly node by setting default_transaction_read_only.
CREATE FUNCTION mark_node_readonly(hostname TEXT, port INTEGER, isreadonly BOOLEAN)

View File

@ -3773,14 +3773,6 @@ ROLLBACK;
DROP TABLE pi_table;
-- 6) test with citus local table
select 1 from citus_add_node('localhost', :master_port, groupid=>0);
NOTICE: Replicating reference table "orders_reference" to the node localhost:xxxxx
NOTICE: Replicating reference table "customer" to the node localhost:xxxxx
NOTICE: Replicating reference table "nation" to the node localhost:xxxxx
NOTICE: Replicating reference table "part" to the node localhost:xxxxx
NOTICE: Replicating reference table "supplier" to the node localhost:xxxxx
NOTICE: Replicating reference table "users_ref_test_table" to the node localhost:xxxxx
NOTICE: Replicating reference table "events_reference_table" to the node localhost:xxxxx
NOTICE: Replicating reference table "users_reference_table" to the node localhost:xxxxx
NOTICE: localhost:xxxxx is the coordinator and already contains metadata, skipping syncing the metadata
?column?
---------------------------------------------------------------------

View File

@ -3,7 +3,6 @@
--
-- Tests that check the metadata after master_remove_node.
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_groupid_seq RESTART 1380000;
ALTER SEQUENCE pg_catalog.pg_dist_node_nodeid_seq RESTART 1380000;
@ -241,7 +240,6 @@ WHERE
(0 rows)
\c - - - :master_port
SET citus.replicate_reference_tables_on_activate TO off;
-- remove same node twice
SELECT master_remove_node('localhost', :worker_2_port);
ERROR: node at "localhost:xxxxx" does not exist
@ -480,7 +478,6 @@ WHERE
(0 rows)
\c - - - :master_port
SET citus.replicate_reference_tables_on_activate TO off;
-- re-add the node for next tests
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
?column?
@ -603,7 +600,6 @@ SELECT * FROM remove_node_reference_table;
(1 row)
\c - - - :master_port
SET citus.replicate_reference_tables_on_activate TO off;
-- re-add the node for next tests
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
?column?
@ -713,7 +709,6 @@ WHERE
(0 rows)
\c - - - :master_port
SET citus.replicate_reference_tables_on_activate TO off;
SET citus.next_shard_id TO 1380001;
-- verify table structure is changed
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.remove_node_reference_table'::regclass;
@ -920,7 +915,6 @@ WHERE
(0 rows)
\c - - - :master_port
SET citus.replicate_reference_tables_on_activate TO off;
-- re-add the node for next tests
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
?column?
@ -1055,7 +1049,6 @@ WHERE
(0 rows)
\c - - - :master_port
SET citus.replicate_reference_tables_on_activate TO off;
-- re-add the node for next tests
SELECT 1 FROM master_activate_node('localhost', :worker_2_port);
?column?

View File

@ -8,7 +8,6 @@ SET citus.next_shard_id TO 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_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;
@ -1007,67 +1006,8 @@ SELECT 1 FROM master_remove_node('localhost', :worker_2_port);
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_placement WHERE shardid = :ref_table_shard;
?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_placement WHERE shardid = :ref_table_shard;
?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_placement WHERE shardid = :ref_table_shard;
?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 that metadata is synced when citus_copy_shard_placement replicates
-- reference table shards
SET citus.replicate_reference_tables_on_activate TO off;
SELECT 1 FROM master_remove_node('localhost', :worker_2_port);
?column?
---------------------------------------------------------------------
1
(1 row)
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
?column?
---------------------------------------------------------------------
@ -1175,7 +1115,6 @@ SET search_path TO replicate_reference_table;
-- The following case used to get stuck on create_distributed_table() instead
-- of detecting the distributed deadlock.
--
SET citus.replicate_reference_tables_on_activate TO off;
SET citus.shard_replication_factor TO 1;
SELECT master_remove_node('localhost', :worker_2_port);
master_remove_node
@ -1219,7 +1158,7 @@ SELECT create_distributed_table('test','x');
SELECT citus_add_node('localhost', :worker_2_port);
citus_add_node
---------------------------------------------------------------------
1370024
1370022
(1 row)
SELECT

View File

@ -1076,7 +1076,6 @@ ORDER BY 1, 2;
SET search_path to "Tenant Isolation";
--
-- Make sure that isolate_tenant_to_new_shard() replicats reference tables
-- when replicate_reference_tables_on_activate is off.
--
CREATE TABLE ref_table(a int);
SELECT create_reference_table('ref_table');
@ -1146,7 +1145,6 @@ SELECT count(*) FROM partitioning_test;
4
(1 row)
SET citus.replicate_reference_tables_on_activate TO off;
SET client_min_messages TO WARNING;
SELECT 1 FROM master_add_node('localhost', :master_port, groupId=>0);
?column?

View File

@ -1199,7 +1199,6 @@ ORDER BY 1, 2;
SET search_path to "Tenant Isolation";
--
-- Make sure that isolate_tenant_to_new_shard() replicats reference tables
-- when replicate_reference_tables_on_activate is off.
--
CREATE TABLE ref_table(a int);
SELECT create_reference_table('ref_table');
@ -1269,7 +1268,6 @@ SELECT count(*) FROM partitioning_test;
4
(1 row)
SET citus.replicate_reference_tables_on_activate TO off;
SET client_min_messages TO WARNING;
SELECT 1 FROM master_add_node('localhost', :master_port, groupId=>0);
?column?

View File

@ -404,8 +404,6 @@ where val = 'asdf';
3
(1 row)
-- not replicate reference tables from other test files
SET citus.replicate_reference_tables_on_activate TO off;
SELECT 1 FROM citus_add_node('localhost', :master_port, groupId => 0);
NOTICE: localhost:xxxxx is the coordinator and already contains metadata, skipping syncing the metadata
?column?
@ -641,7 +639,6 @@ NOTICE: renaming the new table to test_pg12.generated_stored_ref
(4 rows)
ROLLBACK;
RESET citus.replicate_reference_tables_on_activate;
SELECT citus_remove_node('localhost', :master_port);
citus_remove_node
---------------------------------------------------------------------
@ -664,4 +661,4 @@ DROP USER read_access;
drop schema test_pg12 cascade;
NOTICE: drop cascades to 16 other objects
\set VERBOSITY default
SET citus.shard_replication_factor to 2;
SET citus.shard_replication_factor to 2;

View File

@ -1312,6 +1312,15 @@ SELECT 1 FROM citus_add_node('localhost', :master_port, groupId => 0);
(1 row)
RESET client_min_messages;
-- this works around bug #6476: the CREATE TABLE below will
-- self-deadlock on PG15 if it also replicates reference
-- tables to the coordinator.
SELECT replicate_reference_tables(shard_transfer_mode := 'block_writes');
replicate_reference_tables
---------------------------------------------------------------------
(1 row)
-- should error since col_3 defaults to a sequence
CREATE TABLE set_on_default_test_referencing(
col_1 int, col_2 int, col_3 serial, col_4 int,

View File

@ -196,7 +196,6 @@ ERROR: insert or update on table "reference_table_1506003" violates foreign key
-- between reference tables and citus local tables in worker_2 placements of
-- the reference tables
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
NOTICE: Replicating reference table "reference_table" to the node localhost:xxxxx
?column?
---------------------------------------------------------------------
1

View File

@ -509,10 +509,25 @@ SELECT master_remove_node('localhost', :master_port);
(1 row)
-- add the coordinator as a worker node and verify that the reference tables are replicated
SELECT master_add_node('localhost', :master_port, groupid => 0) AS master_nodeid \gset
NOTICE: Replicating reference table "squares" to the node localhost:xxxxx
NOTICE: Replicating reference table "numbers" to the node localhost:xxxxx
SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0);
NOTICE: localhost:xxxxx is the coordinator and already contains metadata, skipping syncing the metadata
?column?
---------------------------------------------------------------------
1
(1 row)
SELECT replicate_reference_tables(shard_transfer_mode := 'force_logical');
replicate_reference_tables
---------------------------------------------------------------------
(1 row)
SELECT count(*) > 0 FROM pg_dist_shard_placement WHERE nodename = 'localhost' AND nodeport = :master_port;
?column?
---------------------------------------------------------------------
t
(1 row)
-- clean-up
SET client_min_messages TO ERROR;
DROP SCHEMA replicate_ref_to_coordinator CASCADE;

View File

@ -20,7 +20,6 @@ CREATE TABLE postgres_table_test(a int primary key);
-- make sure that all rebalance operations works fine when
-- reference tables are replicated to the coordinator
SELECT 1 FROM master_add_node('localhost', :master_port, groupId=>0);
NOTICE: Replicating reference table "ref_table_test" to the node localhost:xxxxx
NOTICE: localhost:xxxxx is the coordinator and already contains metadata, skipping syncing the metadata
?column?
---------------------------------------------------------------------
@ -2177,10 +2176,8 @@ SELECT public.wait_until_metadata_sync(30000);
--
-- Make sure that rebalance_table_shards() and replicate_table_shards() replicate
-- reference tables to the coordinator when replicate_reference_tables_on_activate
-- is off.
-- reference tables to the coordinator
--
SET citus.replicate_reference_tables_on_activate TO off;
SET client_min_messages TO WARNING;
CREATE TABLE dist_table_test_3(a int);
SET citus.shard_count TO 4;

View File

@ -219,7 +219,6 @@ SELECT isolate_tenant_to_new_shard('test', 5, 'CASCADE', shard_transfer_mode =>
-- and on the workers
SELECT 1 FROM master_add_node('localhost', :worker_1_port);
NOTICE: shards are still on the coordinator after adding the new node
NOTICE: Replicating reference table "ref" to the node localhost:xxxxx
?column?
---------------------------------------------------------------------
1

View File

@ -79,7 +79,7 @@ ORDER BY 1;
function citus_is_coordinator()
function citus_isolation_test_session_is_blocked(integer,integer[])
function citus_job_cancel(bigint)
function citus_job_wait(bigint, citus_job_status)
function citus_job_wait(bigint,citus_job_status)
function citus_json_concatenate(json,json)
function citus_json_concatenate_final(json)
function citus_jsonb_concatenate(jsonb,jsonb)
@ -223,7 +223,6 @@ ORDER BY 1;
function truncate_local_data_after_distributing_table(regclass)
function undistribute_table(regclass,boolean)
function update_distributed_table_colocation(regclass,text)
function worker_append_table_to_shard(text,text,text,integer)
function worker_apply_inter_shard_ddl_command(bigint,text,bigint,text,text)
function worker_apply_sequence_command(text)
function worker_apply_sequence_command(text,regtype)
@ -312,5 +311,5 @@ ORDER BY 1;
view citus_stat_statements
view pg_dist_shard_placement
view time_partitions
(304 rows)
(303 rows)

View File

@ -98,7 +98,7 @@ CREATE OR REPLACE VIEW prop_view_6 AS
INNER JOIN view_prop_schema_inner.view_table_4 AS vt4 ON vt1.id = vt4.id;
-- Show that all views are propagated as distributed object
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%prop_view_%' ORDER BY 1;
obj_identifier
obj_identifier
---------------------------------------------------------------------
(view,"{view_prop_schema,prop_view_1}",{})
(view,"{view_prop_schema,prop_view_2}",{})
@ -209,8 +209,8 @@ CREATE USER view_creation_user;
SELECT 1 FROM run_command_on_workers($$CREATE USER view_creation_user;$$);
?column?
---------------------------------------------------------------------
1
1
1
1
(2 rows)
GRANT ALL PRIVILEGES ON SCHEMA view_prop_schema to view_creation_user;
@ -468,7 +468,7 @@ SELECT definition FROM pg_views WHERE viewname = 'alter_view_1';
---------------------------------------------------------------------
SELECT alter_view_table.id,+
alter_view_table.val1 +
FROM alter_view_table;
FROM alter_view_table;
(1 row)
SELECT relname, reloptions
@ -481,11 +481,11 @@ WHERE oid = 'view_prop_schema.alter_view_1'::regclass::oid;
\c - - - :worker_1_port
SELECT definition FROM pg_views WHERE viewname = 'alter_view_1';
definition
definition
---------------------------------------------------------------------
SELECT alter_view_table.id,+
alter_view_table.val1 +
FROM view_prop_schema.alter_view_table;
SELECT alter_view_table.id, +
alter_view_table.val1 +
FROM view_prop_schema.alter_view_table;
(1 row)
SELECT relname, reloptions
@ -879,7 +879,7 @@ CREATE VIEW v_dist AS SELECT 1;
SELECT COUNT(*) FROM pg_dist_object WHERE objid = 'v_dist'::regclass;
count
---------------------------------------------------------------------
1
1
(1 row)
-- tests when citus.enforce_object_restrictions_for_local_objects=false
@ -893,7 +893,7 @@ CREATE OR REPLACE VIEW vv3 as SELECT * FROM vv4;
SELECT COUNT(*) FROM pg_dist_object WHERE objid IN ('vv3'::regclass, 'vv4'::regclass);
count
---------------------------------------------------------------------
0
0
(1 row)
-- local view with no citus relation dependency will NOT be distributed
@ -902,7 +902,7 @@ CREATE VIEW v_local_only AS SELECT 1;
SELECT COUNT(*) FROM pg_dist_object WHERE objid = 'v_local_only'::regclass;
count
---------------------------------------------------------------------
0
0
(1 row)
-- distribute the local table and check the distribution of dependent views
@ -916,14 +916,14 @@ SELECT create_distributed_table('local_t', 'a');
SELECT COUNT(*) FROM pg_dist_object WHERE objid IN ('vv1'::regclass, 'vv2'::regclass);
count
---------------------------------------------------------------------
2
2
(1 row)
-- show that views with circular dependency are NOT in pg_dist_object
SELECT COUNT(*) FROM pg_dist_object WHERE objid IN ('vv3'::regclass, 'vv4'::regclass);
count
---------------------------------------------------------------------
0
0
(1 row)
-- show that we cannot re-create the circular views ever

View File

@ -58,7 +58,8 @@ test: cte_inline recursive_view_local_table values sequences_with_different_type
test: pg13 pg12
# run pg14 sequentially as it syncs metadata
test: pg14
test: pg15 pg15_jsonpath detect_conn_close
test: pg15
test: pg15_jsonpath detect_conn_close
test: drop_column_partitioned_table
test: tableam

View File

@ -29,16 +29,6 @@ step "s1-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"
{
SELECT 1 FROM master_add_node('localhost', 57638);
@ -134,35 +124,25 @@ step "s2-print-index-count"
// 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
// the cache would be empty and the metadata data is gathered from the tables directly
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 "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 "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 "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 "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 "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 "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 "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"
permutation "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 "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 "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 "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"
// same tests without loading the cache
permutation "s1-do-not-replicate-on-activate" "s1-begin" "s1-add-second-worker" "s2-copy-to-reference-table" "s1-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-do-not-replicate-on-activate" "s1-begin" "s1-add-second-worker" "s2-insert-to-reference-table" "s1-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-do-not-replicate-on-activate" "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-begin" "s2-ddl-on-reference-table" "s1-add-second-worker" "s2-commit" "s2-print-index-count"
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 "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"
permutation "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-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-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-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"
// 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"
permutation "s1-add-second-worker" "s2-begin" "s1-begin" "s1-drop-reference-table" "s2-replicate-reference-tables" "s1-commit" "s2-commit"

View File

@ -219,7 +219,8 @@ permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-alter-table" "s1-
permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-drop-table" "s1-commit" "s2-commit" "s3-compare-snapshot"
permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-create-schema" "s1-commit" "s2-commit" "s3-compare-snapshot" "s2-drop-schema"
permutation "s2-create-schema" "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-drop-schema" "s1-commit" "s2-commit" "s3-compare-snapshot"
permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-create-dist-table" "s1-commit" "s2-commit" "s3-compare-snapshot"
// currently snapshot comparison is omitted because independent pg_dist_object records do not have a deterministic order
permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-create-dist-table" "s1-commit" "s2-commit"
permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-create-ref-table" "s1-commit" "s2-commit" "s3-compare-snapshot"
permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-attach-partition" "s1-commit" "s2-commit" "s3-compare-snapshot"
permutation "s2-attach-partition" "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-detach-partition" "s1-commit" "s2-commit" "s3-compare-snapshot"
@ -236,7 +237,6 @@ permutation "s2-create-dist-func" "s1-begin" "s2-begin" "s2-drop-dist-func" "s1-
permutation "s2-create-schema" "s1-begin" "s2-begin" "s2-drop-schema" "s1-start-metadata-sync" "s2-commit" "s1-commit" "s3-compare-snapshot"
permutation "s2-create-type" "s1-begin" "s2-begin" "s2-alter-type" "s1-start-metadata-sync" "s2-commit" "s1-commit" "s3-compare-snapshot" "s3-compare-type-definition"
// the following operations do not get blocked
permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-create-type" "s1-commit" "s2-commit" "s3-compare-snapshot"
permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-create-dist-func" "s1-commit" "s2-commit" "s3-compare-snapshot"

View File

@ -136,14 +136,19 @@ step "add-node"
SELECT 1 FROM master_add_node('localhost', 57636, groupid => 0);
}
step "replicate-reference-tables"
{
SELECT replicate_reference_tables(shard_transfer_mode := 'block_writes');
}
// verify that locks on the placement of the reference table on the coordinator is
// taken into account when looking for distributed deadlocks
permutation "add-node" "s1-begin" "s2-begin" "s1-update-dist-table" "s2-lock-ref-table-placement-on-coordinator" "s1-lock-ref-table-placement-on-coordinator" "s2-update-dist-table" ("s1-lock-ref-table-placement-on-coordinator") "deadlock-checker-call" "s1-end" "s2-end"
permutation "add-node" "replicate-reference-tables" "s1-begin" "s2-begin" "s1-update-dist-table" "s2-lock-ref-table-placement-on-coordinator" "s1-lock-ref-table-placement-on-coordinator" "s2-update-dist-table" ("s1-lock-ref-table-placement-on-coordinator") "deadlock-checker-call" "s1-end" "s2-end"
// verify that *_dist_stat_activity() functions return the correct result when query
// has a task on the coordinator.
permutation "add-node" "s1-begin" "s2-begin" "s1-update-ref-table" "s2-sleep" "s2-view-dist" "s2-view-worker" "s2-end" "s1-end"
permutation "add-node" "replicate-reference-tables" "s1-begin" "s2-begin" "s1-update-ref-table" "s2-sleep" "s2-view-dist" "s2-view-worker" "s2-end" "s1-end"
// verify that get_*_active_transactions() functions return the correct result when
// the query has a task on the coordinator.
permutation "add-node" "s1-begin" "s2-begin" "s1-update-ref-table" "s2-active-transactions" "s1-end" "s2-end"
permutation "add-node" "replicate-reference-tables" "s1-begin" "s2-begin" "s1-update-ref-table" "s2-active-transactions" "s1-end" "s2-end"

View File

@ -8,7 +8,6 @@
SELECT citus.mitmproxy('conn.allow()');
SET citus.next_shard_id TO 200000;
SET citus.replicate_reference_tables_on_activate TO off;
-- verify we have all worker nodes present
SELECT * FROM master_get_active_worker_nodes()

View File

@ -3,7 +3,6 @@ SET search_path TO local_shard_copy;
SET client_min_messages TO DEBUG;
SET citus.next_shard_id TO 1570000;
SET citus.replicate_reference_tables_on_activate TO off;
SELECT 1 FROM master_add_node('localhost', :master_port, groupid := 0);

View File

@ -24,8 +24,6 @@ BEGIN;
SET search_path TO public;
CREATE EXTENSION citus;
-- not wait for replicating reference tables from other test files
SET citus.replicate_reference_tables_on_activate TO OFF;
SELECT 1 FROM master_add_node('localhost', :master_port, groupId => 0);
create table l1 (a int unique);

View File

@ -16,7 +16,6 @@ SELECT stop_metadata_sync_to_node('localhost', :worker_2_port);
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1310000;
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 2;
SET citus.replicate_reference_tables_on_activate TO off;
SELECT nextval('pg_catalog.pg_dist_placement_placementid_seq') AS last_placement_id
\gset

View File

@ -318,34 +318,8 @@ CREATE TABLE full_access_user_schema.t2(id int);
SELECT create_distributed_table('full_access_user_schema.t2', 'id');
RESET ROLE;
\c - - - :worker_2_port
-- non-superuser should be able to use worker_append_table_to_shard on their own shard
SET ROLE full_access;
CREATE TABLE full_access_user_schema.source_table (id int);
INSERT INTO full_access_user_schema.source_table VALUES (1);
CREATE TABLE full_access_user_schema.shard_0 (id int);
SELECT worker_append_table_to_shard('full_access_user_schema.shard_0', 'full_access_user_schema.source_table', 'localhost', :worker_2_port);
SELECT * FROM full_access_user_schema.shard_0;
RESET ROLE;
-- other users should not be able to read from a table they have no access to via worker_append_table_to_shard
SET ROLE usage_access;
SELECT worker_append_table_to_shard('full_access_user_schema.shard_0', 'full_access_user_schema.source_table', 'localhost', :worker_2_port);
RESET ROLE;
-- allow usage_access to read from table
GRANT SELECT ON full_access_user_schema.source_table TO usage_access;
-- other users should not be able to write to a table they do not have write access to
SET ROLE usage_access;
SELECT worker_append_table_to_shard('full_access_user_schema.shard_0', 'full_access_user_schema.source_table', 'localhost', :worker_2_port);
RESET ROLE;
DROP TABLE full_access_user_schema.source_table, full_access_user_schema.shard_0;
\c - - - :master_port
DROP SCHEMA full_access_user_schema CASCADE;
DROP TABLE
my_table,
my_table_with_data,

View File

@ -10,7 +10,6 @@ SELECT nextval('pg_catalog.pg_dist_shardid_seq') AS last_shard_id \gset
SET citus.shard_count TO 8;
SET citus.shard_replication_factor TO 1;
SET citus.replicate_reference_tables_on_activate TO off;
\set VERBOSITY terse

View File

@ -5,7 +5,6 @@
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_groupid_seq RESTART 1380000;
ALTER SEQUENCE pg_catalog.pg_dist_node_nodeid_seq RESTART 1380000;
@ -122,7 +121,6 @@ WHERE
nodeport = :worker_2_port;
\c - - - :master_port
SET citus.replicate_reference_tables_on_activate TO off;
-- remove same node twice
SELECT master_remove_node('localhost', :worker_2_port);
@ -269,7 +267,6 @@ WHERE
nodeport = :worker_2_port;
\c - - - :master_port
SET citus.replicate_reference_tables_on_activate TO off;
-- re-add the node for next tests
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
@ -345,7 +342,6 @@ WHERE
SELECT * FROM remove_node_reference_table;
\c - - - :master_port
SET citus.replicate_reference_tables_on_activate TO off;
-- re-add the node for next tests
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
@ -417,7 +413,6 @@ WHERE
nodeport = :worker_2_port;
\c - - - :master_port
SET citus.replicate_reference_tables_on_activate TO off;
SET citus.next_shard_id TO 1380001;
@ -540,7 +535,6 @@ WHERE
nodeport = :worker_2_port;
\c - - - :master_port
SET citus.replicate_reference_tables_on_activate TO off;
-- re-add the node for next tests
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
@ -619,7 +613,6 @@ WHERE
nodeport = :worker_2_port;
\c - - - :master_port
SET citus.replicate_reference_tables_on_activate TO off;
-- re-add the node for next tests
SELECT 1 FROM master_activate_node('localhost', :worker_2_port);

View File

@ -11,8 +11,6 @@ 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_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;
@ -622,29 +620,8 @@ SELECT count(*) - :ref_table_placements FROM pg_dist_shard_placement WHERE shard
DROP TABLE range_table;
SELECT 1 FROM master_remove_node('localhost', :worker_2_port);
-- 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_placement WHERE shardid = :ref_table_shard;
-- 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_placement WHERE shardid = :ref_table_shard;
SELECT 1 FROM master_activate_node('localhost', :worker_2_port);
SELECT count(*) - :ref_table_placements FROM pg_dist_shard_placement WHERE shardid = :ref_table_shard;
SELECT min(result) = max(result) AS consistent FROM run_command_on_placements('ref_table', 'SELECT sum(a) FROM %s');
-- test that metadata is synced when citus_copy_shard_placement replicates
-- reference table shards
SET citus.replicate_reference_tables_on_activate TO off;
SELECT 1 FROM master_remove_node('localhost', :worker_2_port);
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
SET citus.shard_replication_factor TO 1;
@ -705,7 +682,6 @@ SET search_path TO replicate_reference_table;
-- The following case used to get stuck on create_distributed_table() instead
-- of detecting the distributed deadlock.
--
SET citus.replicate_reference_tables_on_activate TO off;
SET citus.shard_replication_factor TO 1;
SELECT master_remove_node('localhost', :worker_2_port);

View File

@ -540,7 +540,6 @@ SET search_path to "Tenant Isolation";
--
-- Make sure that isolate_tenant_to_new_shard() replicats reference tables
-- when replicate_reference_tables_on_activate is off.
--
@ -580,7 +579,6 @@ SELECT count(*) FROM pg_dist_shard WHERE logicalrelid = 'partitioning_test'::reg
SELECT count(*) FROM partitioning_test;
SET citus.replicate_reference_tables_on_activate TO off;
SET client_min_messages TO WARNING;
SELECT 1 FROM master_add_node('localhost', :master_port, groupId=>0);

View File

@ -578,7 +578,6 @@ SET search_path to "Tenant Isolation";
--
-- Make sure that isolate_tenant_to_new_shard() replicats reference tables
-- when replicate_reference_tables_on_activate is off.
--
@ -618,7 +617,6 @@ SELECT count(*) FROM pg_dist_shard WHERE logicalrelid = 'partitioning_test'::reg
SELECT count(*) FROM partitioning_test;
SET citus.replicate_reference_tables_on_activate TO off;
SET client_min_messages TO WARNING;
SELECT 1 FROM master_add_node('localhost', :master_port, groupId=>0);

View File

@ -267,8 +267,6 @@ select count(*)
from col_test
where val = 'asdf';
-- not replicate reference tables from other test files
SET citus.replicate_reference_tables_on_activate TO off;
SELECT 1 FROM citus_add_node('localhost', :master_port, groupId => 0);
BEGIN;
@ -376,7 +374,6 @@ BEGIN;
SELECT * FROM generated_stored_ref;
ROLLBACK;
RESET citus.replicate_reference_tables_on_activate;
SELECT citus_remove_node('localhost', :master_port);
CREATE TABLE superuser_columnar_table (a int) USING columnar;

View File

@ -839,6 +839,11 @@ SET client_min_messages to ERROR;
SELECT 1 FROM citus_add_node('localhost', :master_port, groupId => 0);
RESET client_min_messages;
-- this works around bug #6476: the CREATE TABLE below will
-- self-deadlock on PG15 if it also replicates reference
-- tables to the coordinator.
SELECT replicate_reference_tables(shard_transfer_mode := 'block_writes');
-- should error since col_3 defaults to a sequence
CREATE TABLE set_on_default_test_referencing(
col_1 int, col_2 int, col_3 serial, col_4 int,

View File

@ -253,7 +253,9 @@ ALTER TABLE squares DROP COLUMN b;
SELECT master_remove_node('localhost', :master_port);
-- add the coordinator as a worker node and verify that the reference tables are replicated
SELECT master_add_node('localhost', :master_port, groupid => 0) AS master_nodeid \gset
SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0);
SELECT replicate_reference_tables(shard_transfer_mode := 'force_logical');
SELECT count(*) > 0 FROM pg_dist_shard_placement WHERE nodename = 'localhost' AND nodeport = :master_port;
-- clean-up
SET client_min_messages TO ERROR;

View File

@ -1220,11 +1220,9 @@ SELECT public.wait_until_metadata_sync(30000);
--
-- Make sure that rebalance_table_shards() and replicate_table_shards() replicate
-- reference tables to the coordinator when replicate_reference_tables_on_activate
-- is off.
-- reference tables to the coordinator
--
SET citus.replicate_reference_tables_on_activate TO off;
SET client_min_messages TO WARNING;
CREATE TABLE dist_table_test_3(a int);