pull/775/merge
Andres Freund 2016-10-17 16:49:47 +00:00 committed by GitHub
commit a1454d0c9b
41 changed files with 3631 additions and 2263 deletions

File diff suppressed because it is too large Load Diff

View File

@ -20,6 +20,7 @@
#include "commands/dbcommands.h"
#include "distributed/metadata_cache.h"
#include "distributed/connection_cache.h"
#include "distributed/connection_management.h"
#include "distributed/multi_client_executor.h"
#include "distributed/multi_server_executor.h"

View File

@ -28,15 +28,18 @@
#include "catalog/pg_type.h"
#include "distributed/citus_clauses.h"
#include "distributed/citus_ruleutils.h"
#include "distributed/connection_cache.h"
#include "distributed/connection_management.h"
#include "distributed/master_metadata_utility.h"
#include "distributed/metadata_cache.h"
#include "distributed/multi_executor.h"
#include "distributed/multi_physical_planner.h"
#include "distributed/multi_planner.h"
#include "distributed/multi_router_executor.h"
#include "distributed/placement_connection.h"
#include "distributed/relay_utility.h"
#include "distributed/remote_commands.h"
#include "distributed/resource_lock.h"
#include "distributed/remote_commands.h"
#include "executor/execdesc.h"
#include "executor/executor.h"
#include "executor/instrument.h"
@ -64,29 +67,9 @@
/* controls use of locks to enforce safe commutativity */
bool AllModificationsCommutative = false;
/*
* The following static variables are necessary to track the progression of
* multi-statement transactions managed by the router executor. After the first
* modification within a transaction, the executor populates a hash with the
* transaction's initial participants (nodes hit by that initial modification).
*
* To keep track of the reverse mapping (from shards to nodes), we have a list
* of XactShardConnSets, which map a shard identifier to a set of connection
* hash entries. This list is walked by MarkRemainingInactivePlacements to
* ensure we mark placements as failed if they reject a COMMIT.
*
* Beyond that, there's a backend hook to register xact callbacks and a flag to
* track when a user tries to roll back to a savepoint (not allowed).
*/
static HTAB *xactParticipantHash = NULL;
static List *xactShardConnSetList = NIL;
static bool subXactAbortAttempted = false;
/* functions needed during start phase */
static void InitTransactionStateForTask(Task *task);
static LOCKMODE CommutativityRuleToLockMode(CmdType commandType, bool upsertQuery);
static void AcquireExecutorShardLock(Task *task, LOCKMODE lockMode);
static HTAB * CreateXactParticipantHash(void);
/* functions needed during run phase */
static bool ExecuteTaskAndStoreResults(QueryDesc *queryDesc,
@ -96,27 +79,14 @@ static bool ExecuteTaskAndStoreResults(QueryDesc *queryDesc,
static uint64 ReturnRowsFromTuplestore(uint64 tupleCount, TupleDesc tupleDescriptor,
DestReceiver *destination,
Tuplestorestate *tupleStore);
static PGconn * GetConnectionForPlacement(ShardPlacement *placement,
bool isModificationQuery);
static void PurgeConnectionForPlacement(ShardPlacement *placement);
static void RemoveXactConnection(PGconn *connection);
static void ExtractParametersFromParamListInfo(ParamListInfo paramListInfo,
Oid **parameterTypes,
const char ***parameterValues);
static bool SendQueryInSingleRowMode(PGconn *connection, char *query,
static bool SendQueryInSingleRowMode(MultiConnection *connection, char *query,
ParamListInfo paramListInfo);
static bool StoreQueryResult(MaterialState *routerState, PGconn *connection,
static bool StoreQueryResult(MaterialState *routerState, MultiConnection *connection,
TupleDesc tupleDescriptor, int64 *rows);
static bool ConsumeQueryResult(PGconn *connection, int64 *rows);
static void RecordShardIdParticipant(uint64 affectedShardId,
NodeConnectionEntry *participantEntry);
/* functions needed by callbacks and hooks */
static void RouterTransactionCallback(XactEvent event, void *arg);
static void RouterSubtransactionCallback(SubXactEvent event, SubTransactionId subId,
SubTransactionId parentSubid, void *arg);
static void ExecuteTransactionEnd(bool commit);
static void MarkRemainingInactivePlacements(void);
static bool ConsumeQueryResult(MultiConnection *connection, int64 *rows);
/*
@ -152,9 +122,9 @@ RouterExecutorStart(QueryDesc *queryDesc, int eflags, Task *task)
* but some customers already use functions that touch multiple shards
* from within a function, so we'll ignore functions for now.
*/
if (IsTransactionBlock() && xactParticipantHash == NULL)
if (IsTransactionBlock())
{
InitTransactionStateForTask(task);
BeginOrContinueCoordinatedTransaction();
}
}
@ -184,62 +154,6 @@ RouterExecutorStart(QueryDesc *queryDesc, int eflags, Task *task)
}
/*
* InitTransactionStateForTask is called during executor start with the first
* modifying (INSERT/UPDATE/DELETE) task during a transaction. It creates the
* transaction participant hash, opens connections to this task's nodes, and
* populates the hash with those connections after sending BEGIN commands to
* each. If a node fails to respond, its connection is set to NULL to prevent
* further interaction with it during the transaction.
*/
static void
InitTransactionStateForTask(Task *task)
{
ListCell *placementCell = NULL;
xactParticipantHash = CreateXactParticipantHash();
foreach(placementCell, task->taskPlacementList)
{
ShardPlacement *placement = (ShardPlacement *) lfirst(placementCell);
NodeConnectionKey participantKey;
NodeConnectionEntry *participantEntry = NULL;
bool entryFound = false;
PGconn *connection = NULL;
MemSet(&participantKey, 0, sizeof(participantKey));
strlcpy(participantKey.nodeName, placement->nodeName,
MAX_NODE_LENGTH + 1);
participantKey.nodePort = placement->nodePort;
participantEntry = hash_search(xactParticipantHash, &participantKey,
HASH_ENTER, &entryFound);
Assert(!entryFound);
connection = GetOrEstablishConnection(placement->nodeName,
placement->nodePort);
if (connection != NULL)
{
PGresult *result = PQexec(connection, "BEGIN");
if (PQresultStatus(result) != PGRES_COMMAND_OK)
{
WarnRemoteError(connection, result);
PurgeConnection(connection);
connection = NULL;
}
PQclear(result);
}
participantEntry->connection = connection;
}
XactModificationLevel = XACT_MODIFICATION_DATA;
}
/*
* CommutativityRuleToLockMode determines the commutativity rule for the given
* command and returns the appropriate lock mode to enforce that rule. The
@ -309,33 +223,6 @@ AcquireExecutorShardLock(Task *task, LOCKMODE lockMode)
}
/*
* CreateXactParticipantHash initializes the map used to store the connections
* needed to process distributed transactions. Unlike the connection cache, we
* permit NULL connections here to signify that a participant has seen an error
* and is no longer receiving commands during a transaction. This hash should
* be walked at transaction end to send final COMMIT or ABORT commands.
*/
static HTAB *
CreateXactParticipantHash(void)
{
HTAB *xactParticipantHash = NULL;
HASHCTL info;
int hashFlags = 0;
MemSet(&info, 0, sizeof(info));
info.keysize = sizeof(NodeConnectionKey);
info.entrysize = sizeof(NodeConnectionEntry);
info.hcxt = TopTransactionContext;
hashFlags = (HASH_ELEM | HASH_CONTEXT | HASH_BLOBS);
xactParticipantHash = hash_create("citus xact participant hash", 32, &info,
hashFlags);
return xactParticipantHash;
}
/*
* RouterExecutorRun actually executes a single task on a worker.
*/
@ -420,6 +307,12 @@ RouterExecutorRun(QueryDesc *queryDesc, ScanDirection direction, long count)
/* mark underlying query as having executed */
routerState->eof_underlying = true;
/* have performed modifications now */
if (isModificationQuery)
{
XactModificationLevel = XACT_MODIFICATION_DATA;
}
}
/* if the underlying query produced output, return it */
@ -487,6 +380,7 @@ ExecuteTaskAndStoreResults(QueryDesc *queryDesc, Task *task,
int64 affectedTupleCount = -1;
bool gotResults = false;
char *queryString = task->queryString;
int connectionFlags = NEW_CONNECTION | CACHED_CONNECTION | SESSION_LIFESPAN;
if (isModificationQuery && task->requiresMasterEvaluation)
{
@ -504,6 +398,11 @@ ExecuteTaskAndStoreResults(QueryDesc *queryDesc, Task *task,
elog(DEBUG4, "query after master evaluation: %s", queryString);
}
if (isModificationQuery)
{
connectionFlags |= FOR_DML;
}
/*
* Try to run the query to completion on one placement. If the query fails
* attempt the query on the next placement.
@ -513,8 +412,16 @@ ExecuteTaskAndStoreResults(QueryDesc *queryDesc, Task *task,
ShardPlacement *taskPlacement = (ShardPlacement *) lfirst(taskPlacementCell);
bool queryOK = false;
int64 currentAffectedTupleCount = 0;
PGconn *connection = GetConnectionForPlacement(taskPlacement,
isModificationQuery);
/*
* FIXME: It's not actually correct to use only one shard placement
* here for router queries involving multiple relations. We should
* check that this connection is the only modifying one associated
* with all the involved shards.
*/
MultiConnection *connection = GetPlacementConnection(connectionFlags,
taskPlacement);
AdjustRemoteTransactionState(connection);
if (connection == NULL)
{
@ -525,7 +432,6 @@ ExecuteTaskAndStoreResults(QueryDesc *queryDesc, Task *task,
queryOK = SendQueryInSingleRowMode(connection, queryString, paramListInfo);
if (!queryOK)
{
PurgeConnectionForPlacement(taskPlacement);
failedPlacementList = lappend(failedPlacementList, taskPlacement);
continue;
}
@ -582,8 +488,6 @@ ExecuteTaskAndStoreResults(QueryDesc *queryDesc, Task *task,
}
else
{
PurgeConnectionForPlacement(taskPlacement);
failedPlacementList = lappend(failedPlacementList, taskPlacement);
continue;
@ -660,145 +564,14 @@ ReturnRowsFromTuplestore(uint64 tupleCount, TupleDesc tupleDescriptor,
}
/*
* GetConnectionForPlacement is the main entry point for acquiring a connection
* within the router executor. By using placements (rather than node names and
* ports) to identify connections, the router executor can keep track of shards
* used by multi-statement transactions and error out if a transaction tries
* to reach a new node altogether). In the single-statement commands context,
* GetConnectionForPlacement simply falls through to GetOrEstablishConnection.
*/
static PGconn *
GetConnectionForPlacement(ShardPlacement *placement, bool isModificationQuery)
{
NodeConnectionKey participantKey;
NodeConnectionEntry *participantEntry = NULL;
bool entryFound = false;
/* if not in a transaction, fall through to connection cache */
if (xactParticipantHash == NULL)
{
PGconn *connection = GetOrEstablishConnection(placement->nodeName,
placement->nodePort);
return connection;
}
Assert(IsTransactionBlock());
MemSet(&participantKey, 0, sizeof(participantKey));
strlcpy(participantKey.nodeName, placement->nodeName, MAX_NODE_LENGTH + 1);
participantKey.nodePort = placement->nodePort;
participantEntry = hash_search(xactParticipantHash, &participantKey, HASH_FIND,
&entryFound);
if (entryFound)
{
if (isModificationQuery)
{
RecordShardIdParticipant(placement->shardId, participantEntry);
}
return participantEntry->connection;
}
else
{
ereport(ERROR, (errcode(ERRCODE_CONNECTION_DOES_NOT_EXIST),
errmsg("no transaction participant matches %s:%d",
placement->nodeName, placement->nodePort),
errdetail("Transactions which modify distributed tables may only "
"target nodes affected by the modification command "
"which began the transaction.")));
}
}
/*
* PurgeConnectionForPlacement provides a way to purge an invalid connection
* from all relevant connection hashes using the placement involved in the
* query at the time of the error. If a transaction is ongoing, this function
* ensures the right node's connection is set to NULL in the participant map
* for the transaction in addition to purging the connection cache's entry.
*/
static void
PurgeConnectionForPlacement(ShardPlacement *placement)
{
NodeConnectionKey nodeKey;
char *currentUser = CurrentUserName();
MemSet(&nodeKey, 0, sizeof(NodeConnectionKey));
strlcpy(nodeKey.nodeName, placement->nodeName, MAX_NODE_LENGTH + 1);
nodeKey.nodePort = placement->nodePort;
strlcpy(nodeKey.nodeUser, currentUser, NAMEDATALEN);
PurgeConnectionByKey(&nodeKey);
/*
* The following is logically identical to RemoveXactConnection, but since
* we have a ShardPlacement to help build a NodeConnectionKey, we avoid
* any penalty incurred by calling BuildKeyForConnection, which must ex-
* tract host, port, and user from the connection options list.
*/
if (xactParticipantHash != NULL)
{
NodeConnectionEntry *participantEntry = NULL;
bool entryFound = false;
Assert(IsTransactionBlock());
/* the participant hash doesn't use the user field */
MemSet(&nodeKey.nodeUser, 0, sizeof(nodeKey.nodeUser));
participantEntry = hash_search(xactParticipantHash, &nodeKey, HASH_FIND,
&entryFound);
Assert(entryFound);
participantEntry->connection = NULL;
}
}
/*
* Removes a given connection from the transaction participant hash, based on
* the host and port of the provided connection. If the hash is not NULL, it
* MUST contain the provided connection, or a FATAL error is raised.
*/
static void
RemoveXactConnection(PGconn *connection)
{
NodeConnectionKey nodeKey;
NodeConnectionEntry *participantEntry = NULL;
bool entryFound = false;
if (xactParticipantHash == NULL)
{
return;
}
BuildKeyForConnection(connection, &nodeKey);
/* the participant hash doesn't use the user field */
MemSet(&nodeKey.nodeUser, 0, sizeof(nodeKey.nodeUser));
participantEntry = hash_search(xactParticipantHash, &nodeKey, HASH_FIND,
&entryFound);
if (!entryFound)
{
ereport(FATAL, (errmsg("could not find specified transaction connection")));
}
participantEntry->connection = NULL;
}
/*
* SendQueryInSingleRowMode sends the given query on the connection in an
* asynchronous way. The function also sets the single-row mode on the
* connection so that we receive results a row at a time.
*/
static bool
SendQueryInSingleRowMode(PGconn *connection, char *query, ParamListInfo paramListInfo)
SendQueryInSingleRowMode(MultiConnection *connection, char *query,
ParamListInfo paramListInfo)
{
int querySent = 0;
int singleRowMode = 0;
@ -812,24 +585,25 @@ SendQueryInSingleRowMode(PGconn *connection, char *query, ParamListInfo paramLis
ExtractParametersFromParamListInfo(paramListInfo, &parameterTypes,
&parameterValues);
querySent = PQsendQueryParams(connection, query, parameterCount, parameterTypes,
parameterValues, NULL, NULL, 0);
querySent = PQsendQueryParams(connection->conn, query, parameterCount,
parameterTypes, parameterValues,
NULL, NULL, 0);
}
else
{
querySent = PQsendQuery(connection, query);
querySent = PQsendQuery(connection->conn, query);
}
if (querySent == 0)
{
WarnRemoteError(connection, NULL);
ReportConnectionError(connection, WARNING);
return false;
}
singleRowMode = PQsetSingleRowMode(connection);
singleRowMode = PQsetSingleRowMode(connection->conn);
if (singleRowMode == 0)
{
WarnRemoteError(connection, NULL);
ReportConnectionError(connection, WARNING);
return false;
}
@ -902,7 +676,7 @@ ExtractParametersFromParamListInfo(ParamListInfo paramListInfo, Oid **parameterT
* the connection.
*/
static bool
StoreQueryResult(MaterialState *routerState, PGconn *connection,
StoreQueryResult(MaterialState *routerState, MultiConnection *connection,
TupleDesc tupleDescriptor, int64 *rows)
{
AttInMetadata *attributeInputMetadata = TupleDescGetAttInMetadata(tupleDescriptor);
@ -937,7 +711,7 @@ StoreQueryResult(MaterialState *routerState, PGconn *connection,
uint32 columnCount = 0;
ExecStatusType resultStatus = 0;
PGresult *result = PQgetResult(connection);
PGresult *result = PQgetResult(connection->conn);
if (result == NULL)
{
break;
@ -960,12 +734,11 @@ StoreQueryResult(MaterialState *routerState, PGconn *connection,
if (raiseError)
{
RemoveXactConnection(connection);
ReraiseRemoteError(connection, result);
ReportResultError(connection, result, ERROR);
}
else
{
WarnRemoteError(connection, result);
ReportResultError(connection, result, WARNING);
}
PQclear(result);
@ -1030,7 +803,7 @@ StoreQueryResult(MaterialState *routerState, PGconn *connection,
* has been an error.
*/
static bool
ConsumeQueryResult(PGconn *connection, int64 *rows)
ConsumeQueryResult(MultiConnection *connection, int64 *rows)
{
bool commandFailed = false;
bool gotResponse = false;
@ -1044,7 +817,7 @@ ConsumeQueryResult(PGconn *connection, int64 *rows)
*/
while (true)
{
PGresult *result = PQgetResult(connection);
PGresult *result = PQgetResult(connection->conn);
ExecStatusType status = PGRES_COMMAND_OK;
if (result == NULL)
@ -1072,12 +845,11 @@ ConsumeQueryResult(PGconn *connection, int64 *rows)
if (raiseError)
{
RemoveXactConnection(connection);
ReraiseRemoteError(connection, result);
ReportResultError(connection, result, ERROR);
}
else
{
WarnRemoteError(connection, result);
ReportResultError(connection, result, WARNING);
}
PQclear(result);
@ -1115,50 +887,6 @@ ConsumeQueryResult(PGconn *connection, int64 *rows)
}
/*
* RecordShardIdParticipant registers a connection as being involved with a
* particular shard during a multi-statement transaction.
*/
static void
RecordShardIdParticipant(uint64 affectedShardId, NodeConnectionEntry *participantEntry)
{
XactShardConnSet *shardConnSetMatch = NULL;
ListCell *listCell = NULL;
MemoryContext oldContext = NULL;
List *connectionEntryList = NIL;
/* check whether an entry already exists for this shard */
foreach(listCell, xactShardConnSetList)
{
XactShardConnSet *shardConnSet = (XactShardConnSet *) lfirst(listCell);
if (shardConnSet->shardId == affectedShardId)
{
shardConnSetMatch = shardConnSet;
}
}
/* entries must last through the whole top-level transaction */
oldContext = MemoryContextSwitchTo(TopTransactionContext);
/* if no entry found, make one */
if (shardConnSetMatch == NULL)
{
shardConnSetMatch = (XactShardConnSet *) palloc0(sizeof(XactShardConnSet));
shardConnSetMatch->shardId = affectedShardId;
xactShardConnSetList = lappend(xactShardConnSetList, shardConnSetMatch);
}
/* add connection, avoiding duplicates */
connectionEntryList = shardConnSetMatch->connectionEntryList;
shardConnSetMatch->connectionEntryList = list_append_unique_ptr(connectionEntryList,
participantEntry);
MemoryContextSwitchTo(oldContext);
}
/*
* RouterExecutorFinish cleans up after a distributed execution.
*/
@ -1193,222 +921,3 @@ RouterExecutorEnd(QueryDesc *queryDesc)
queryDesc->estate = NULL;
queryDesc->totaltime = NULL;
}
/*
* RegisterRouterExecutorXactCallbacks registers this executor's callbacks.
*/
void
RegisterRouterExecutorXactCallbacks(void)
{
RegisterXactCallback(RouterTransactionCallback, NULL);
RegisterSubXactCallback(RouterSubtransactionCallback, NULL);
}
/*
* RouterTransactionCallback handles committing or aborting remote transactions
* after the local one has committed or aborted. It only sends COMMIT or ABORT
* commands to still-healthy remotes; the failed ones are marked as inactive if
* after a successful COMMIT (no need to mark on ABORTs).
*/
static void
RouterTransactionCallback(XactEvent event, void *arg)
{
if (XactModificationLevel != XACT_MODIFICATION_DATA)
{
return;
}
switch (event)
{
case XACT_EVENT_PARALLEL_COMMIT:
case XACT_EVENT_COMMIT:
{
break;
}
case XACT_EVENT_PARALLEL_ABORT:
case XACT_EVENT_ABORT:
{
bool commit = false;
ExecuteTransactionEnd(commit);
break;
}
/* no support for prepare with multi-statement transactions */
case XACT_EVENT_PREPARE:
case XACT_EVENT_PRE_PREPARE:
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot prepare a transaction that modified "
"distributed tables")));
break;
}
case XACT_EVENT_PARALLEL_PRE_COMMIT:
case XACT_EVENT_PRE_COMMIT:
{
bool commit = true;
if (subXactAbortAttempted)
{
subXactAbortAttempted = false;
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot ROLLBACK TO SAVEPOINT in transactions "
"which modify distributed tables")));
}
ExecuteTransactionEnd(commit);
MarkRemainingInactivePlacements();
/* leave early to avoid resetting transaction state */
return;
}
}
/* reset transaction state */
XactModificationLevel = XACT_MODIFICATION_NONE;
xactParticipantHash = NULL;
xactShardConnSetList = NIL;
subXactAbortAttempted = false;
}
/*
* RouterSubtransactionCallback silently keeps track of any attempt to ROLLBACK
* TO SAVEPOINT, which is not permitted by this executor. At transaction end,
* the executor checks whether such a rollback was attempted and, if so, errors
* out entirely (with an appropriate message).
*
* This implementation permits savepoints so long as no rollbacks occur.
*/
static void
RouterSubtransactionCallback(SubXactEvent event, SubTransactionId subId,
SubTransactionId parentSubid, void *arg)
{
if ((xactParticipantHash != NULL) && (event == SUBXACT_EVENT_ABORT_SUB))
{
subXactAbortAttempted = true;
}
}
/*
* ExecuteTransactionEnd ends any remote transactions still taking place on
* remote nodes. It uses xactParticipantHash to know which nodes need any
* final COMMIT or ABORT commands. Nodes which fail a final COMMIT will have
* their connection field set to NULL to permit placement invalidation.
*/
static void
ExecuteTransactionEnd(bool commit)
{
const char *sqlCommand = commit ? "COMMIT TRANSACTION" : "ABORT TRANSACTION";
HASH_SEQ_STATUS scan;
NodeConnectionEntry *participant;
bool completed = !commit; /* aborts are assumed completed */
if (xactParticipantHash == NULL)
{
return;
}
hash_seq_init(&scan, xactParticipantHash);
while ((participant = (NodeConnectionEntry *) hash_seq_search(&scan)))
{
PGconn *connection = participant->connection;
PGresult *result = NULL;
if (PQstatus(connection) != CONNECTION_OK)
{
continue;
}
result = PQexec(connection, sqlCommand);
if (PQresultStatus(result) == PGRES_COMMAND_OK)
{
completed = true;
}
else
{
WarnRemoteError(connection, result);
PurgeConnection(participant->connection);
participant->connection = NULL;
}
PQclear(result);
}
if (!completed)
{
ereport(ERROR, (errmsg("could not commit transaction on any active nodes")));
}
}
/*
* MarkRemainingInactivePlacements takes care of marking placements of a shard
* inactive after some of the placements rejected the final COMMIT phase of a
* transaction. This step is skipped if all placements reject the COMMIT, since
* in that case no modifications to the placement have persisted.
*
* Failures are detected by checking the connection field of the entries in the
* connection set for each shard: it is always set to NULL after errors.
*/
static void
MarkRemainingInactivePlacements(void)
{
ListCell *shardConnSetCell = NULL;
foreach(shardConnSetCell, xactShardConnSetList)
{
XactShardConnSet *shardConnSet = (XactShardConnSet *) lfirst(shardConnSetCell);
List *participantList = shardConnSet->connectionEntryList;
ListCell *participantCell = NULL;
int successes = list_length(participantList); /* assume full success */
/* determine how many actual successes there were: subtract failures */
foreach(participantCell, participantList)
{
NodeConnectionEntry *participant = NULL;
participant = (NodeConnectionEntry *) lfirst(participantCell);
/* other codes sets connection to NULL after errors */
if (participant->connection == NULL)
{
successes--;
}
}
/* if no nodes succeeded for this shard, don't do anything */
if (successes == 0)
{
continue;
}
/* otherwise, ensure failed placements are marked inactive */
foreach(participantCell, participantList)
{
NodeConnectionEntry *participant = NULL;
participant = (NodeConnectionEntry *) lfirst(participantCell);
if (participant->connection == NULL)
{
uint64 shardId = shardConnSet->shardId;
NodeConnectionKey *nodeKey = &participant->cacheKey;
uint64 shardLength = 0;
uint64 placementId = INVALID_PLACEMENT_ID;
placementId = DeleteShardPlacementRow(shardId, nodeKey->nodeName,
nodeKey->nodePort);
InsertShardPlacementRow(shardId, placementId, FILE_INACTIVE, shardLength,
nodeKey->nodeName, nodeKey->nodePort);
}
}
}
}

View File

@ -30,17 +30,15 @@
#include "commands/defrem.h"
#include "commands/tablecmds.h"
#include "distributed/citus_ruleutils.h"
#include "distributed/commit_protocol.h"
#include "distributed/connection_cache.h"
#include "distributed/master_metadata_utility.h"
#include "distributed/master_protocol.h"
#include "distributed/metadata_cache.h"
#include "distributed/multi_copy.h"
#include "distributed/multi_join_order.h"
#include "distributed/multi_shard_transaction.h"
#include "distributed/multi_utility.h" /* IWYU pragma: keep */
#include "distributed/pg_dist_partition.h"
#include "distributed/resource_lock.h"
#include "distributed/remote_commands.h"
#include "distributed/transmit.h"
#include "distributed/worker_protocol.h"
#include "executor/executor.h"
@ -95,14 +93,13 @@ static void VerifyTransmitStmt(CopyStmt *copyStatement);
static Node * ProcessCopyStmt(CopyStmt *copyStatement, char *completionTag,
bool *commandMustRunAsOwner);
static Node * ProcessIndexStmt(IndexStmt *createIndexStatement,
const char *createIndexCommand, bool isTopLevel);
const char *createIndexCommand);
static Node * ProcessDropIndexStmt(DropStmt *dropIndexStatement,
const char *dropIndexCommand, bool isTopLevel);
const char *dropIndexCommand);
static Node * ProcessAlterTableStmt(AlterTableStmt *alterTableStatement,
const char *alterTableCommand, bool isTopLevel);
const char *alterTableCommand);
static Node * ProcessAlterObjectSchemaStmt(AlterObjectSchemaStmt *alterObjectSchemaStmt,
const char *alterObjectSchemaCommand,
bool isTopLevel);
const char *alterObjectSchemaCommand);
/* Local functions forward declarations for unsupported command checks */
static void ErrorIfUnsupportedIndexStmt(IndexStmt *createIndexStatement);
@ -117,12 +114,8 @@ static void ErrorIfDistributedRenameStmt(RenameStmt *renameStatement);
/* Local functions forward declarations for helper functions */
static void CreateLocalTable(RangeVar *relation, char *nodeName, int32 nodePort);
static bool IsAlterTableRenameStmt(RenameStmt *renameStatement);
static void ExecuteDistributedDDLCommand(Oid relationId, const char *ddlCommandString,
bool isTopLevel);
static void ExecuteDistributedDDLCommand(Oid relationId, const char *ddlCommandString);
static void ShowNoticeIfNotUsing2PC(void);
static bool ExecuteCommandOnWorkerShards(Oid relationId, const char *commandString);
static void ExecuteCommandOnShardPlacements(StringInfo applyCommand, uint64 shardId,
ShardConnections *shardConnections);
static void RangeVarCallbackForDropIndex(const RangeVar *rel, Oid relOid, Oid oldRelOid,
void *arg);
static void CheckCopyPermissions(CopyStmt *copyStatement);
@ -210,12 +203,9 @@ multi_ProcessUtility(Node *parsetree,
/* ddl commands are propagated to workers only if EnableDDLPropagation is set */
if (EnableDDLPropagation)
{
bool isTopLevel = (context == PROCESS_UTILITY_TOPLEVEL);
if (IsA(parsetree, IndexStmt))
{
parsetree = ProcessIndexStmt((IndexStmt *) parsetree, queryString,
isTopLevel);
parsetree = ProcessIndexStmt((IndexStmt *) parsetree, queryString);
}
if (IsA(parsetree, DropStmt))
@ -223,7 +213,7 @@ multi_ProcessUtility(Node *parsetree,
DropStmt *dropStatement = (DropStmt *) parsetree;
if (dropStatement->removeType == OBJECT_INDEX)
{
parsetree = ProcessDropIndexStmt(dropStatement, queryString, isTopLevel);
parsetree = ProcessDropIndexStmt(dropStatement, queryString);
}
}
@ -232,8 +222,7 @@ multi_ProcessUtility(Node *parsetree,
AlterTableStmt *alterTableStmt = (AlterTableStmt *) parsetree;
if (alterTableStmt->relkind == OBJECT_TABLE)
{
parsetree = ProcessAlterTableStmt(alterTableStmt, queryString,
isTopLevel);
parsetree = ProcessAlterTableStmt(alterTableStmt, queryString);
}
}
@ -257,8 +246,7 @@ multi_ProcessUtility(Node *parsetree,
if (IsA(parsetree, AlterObjectSchemaStmt))
{
AlterObjectSchemaStmt *setSchemaStmt = (AlterObjectSchemaStmt *) parsetree;
parsetree = ProcessAlterObjectSchemaStmt(setSchemaStmt, queryString,
isTopLevel);
parsetree = ProcessAlterObjectSchemaStmt(setSchemaStmt, queryString);
}
/*
@ -538,8 +526,7 @@ ProcessCopyStmt(CopyStmt *copyStatement, char *completionTag, bool *commandMustR
* master node table.
*/
static Node *
ProcessIndexStmt(IndexStmt *createIndexStatement, const char *createIndexCommand,
bool isTopLevel)
ProcessIndexStmt(IndexStmt *createIndexStatement, const char *createIndexCommand)
{
/*
* We first check whether a distributed relation is affected. For that, we need to
@ -586,7 +573,7 @@ ProcessIndexStmt(IndexStmt *createIndexStatement, const char *createIndexCommand
ErrorIfUnsupportedIndexStmt(createIndexStatement);
/* if it is supported, go ahead and execute the command */
ExecuteDistributedDDLCommand(relationId, createIndexCommand, isTopLevel);
ExecuteDistributedDDLCommand(relationId, createIndexCommand);
}
}
@ -603,8 +590,7 @@ ProcessIndexStmt(IndexStmt *createIndexStatement, const char *createIndexCommand
* master node table.
*/
static Node *
ProcessDropIndexStmt(DropStmt *dropIndexStatement, const char *dropIndexCommand,
bool isTopLevel)
ProcessDropIndexStmt(DropStmt *dropIndexStatement, const char *dropIndexCommand)
{
ListCell *dropObjectCell = NULL;
Oid distributedIndexId = InvalidOid;
@ -673,7 +659,7 @@ ProcessDropIndexStmt(DropStmt *dropIndexStatement, const char *dropIndexCommand,
ErrorIfUnsupportedDropIndexStmt(dropIndexStatement);
/* if it is supported, go ahead and execute the command */
ExecuteDistributedDDLCommand(distributedRelationId, dropIndexCommand, isTopLevel);
ExecuteDistributedDDLCommand(distributedRelationId, dropIndexCommand);
}
return (Node *) dropIndexStatement;
@ -689,8 +675,7 @@ ProcessDropIndexStmt(DropStmt *dropIndexStatement, const char *dropIndexCommand,
* master node table.
*/
static Node *
ProcessAlterTableStmt(AlterTableStmt *alterTableStatement, const char *alterTableCommand,
bool isTopLevel)
ProcessAlterTableStmt(AlterTableStmt *alterTableStatement, const char *alterTableCommand)
{
/* first check whether a distributed relation is affected */
if (alterTableStatement->relation != NULL)
@ -705,7 +690,7 @@ ProcessAlterTableStmt(AlterTableStmt *alterTableStatement, const char *alterTabl
ErrorIfUnsupportedAlterTableStmt(alterTableStatement);
/* if it is supported, go ahead and execute the command */
ExecuteDistributedDDLCommand(relationId, alterTableCommand, isTopLevel);
ExecuteDistributedDDLCommand(relationId, alterTableCommand);
}
}
}
@ -722,7 +707,7 @@ ProcessAlterTableStmt(AlterTableStmt *alterTableStatement, const char *alterTabl
*/
static Node *
ProcessAlterObjectSchemaStmt(AlterObjectSchemaStmt *alterObjectSchemaStmt,
const char *alterObjectSchemaCommand, bool isTopLevel)
const char *alterObjectSchemaCommand)
{
Oid relationId = InvalidOid;
bool noWait = false;
@ -1278,16 +1263,13 @@ IsAlterTableRenameStmt(RenameStmt *renameStmt)
* ExecuteDistributedDDLCommand applies a given DDL command to the given
* distributed table in a distributed transaction. If the multi shard commit protocol is
* in its default value of '1pc', then a notice message indicating that '2pc' might be
* used for extra safety. In the commit protocol, a BEGIN is sent after connection to
* each shard placement and COMMIT/ROLLBACK is handled by
* CompleteShardPlacementTransactions function.
* used for extra safety.
*
* DDL is executed, via worker_apply_shard_ddl_command(), on the workers.
*/
static void
ExecuteDistributedDDLCommand(Oid relationId, const char *ddlCommandString,
bool isTopLevel)
ExecuteDistributedDDLCommand(Oid relationId, const char *ddlCommandString)
{
bool executionOK = false;
if (XactModificationLevel == XACT_MODIFICATION_DATA)
{
ereport(ERROR, (errcode(ERRCODE_ACTIVE_SQL_TRANSACTION),
@ -1295,16 +1277,14 @@ ExecuteDistributedDDLCommand(Oid relationId, const char *ddlCommandString,
"transaction blocks containing data modifications")));
}
BeginOrContinueCoordinatedTransaction();
/* FIXME: Move into ExecuteDDLOnRelationPlacements()? */
ShowNoticeIfNotUsing2PC();
executionOK = ExecuteCommandOnWorkerShards(relationId, ddlCommandString);
/* if command could not be executed on any finalized shard placement, error out */
if (!executionOK)
{
ereport(ERROR, (errmsg("could not execute DDL command on worker node shards")));
}
ExecuteDDLOnRelationPlacements(relationId, ddlCommandString);
/* FIXME: Move into ExecuteDDLOnRelationPlacements()? */
XactModificationLevel = XACT_MODIFICATION_SCHEMA;
}
@ -1328,116 +1308,6 @@ ShowNoticeIfNotUsing2PC(void)
}
/*
* ExecuteCommandOnWorkerShards executes a given command on all the finalized
* shard placements of the given table within a distributed transaction. The
* value of citus.multi_shard_commit_protocol is set to '2pc' by the caller
* ExecuteDistributedDDLCommand function so that two phase commit protocol is used.
*
* ExecuteCommandOnWorkerShards opens an individual connection for each of the
* shard placement. After all connections are opened, a BEGIN command followed by
* a proper "SELECT worker_apply_shard_ddl_command(<shardId>, <DDL Command>)" is
* sent to all open connections in a serial manner.
*
* The opened transactions are handled by the CompleteShardPlacementTransactions
* function.
*
* Note: There are certain errors which would occur on few nodes and not on the
* others. For example, adding a column with a type which exists on some nodes
* and not on the others.
*
* Note: The execution will be blocked if a prepared transaction from previous
* executions exist on the workers. In this case, those prepared transactions should
* be removed by either COMMIT PREPARED or ROLLBACK PREPARED.
*/
static bool
ExecuteCommandOnWorkerShards(Oid relationId, const char *commandString)
{
List *shardIntervalList = LoadShardIntervalList(relationId);
char *tableOwner = TableOwner(relationId);
ListCell *shardIntervalCell = NULL;
Oid schemaId = get_rel_namespace(relationId);
char *schemaName = get_namespace_name(schemaId);
LockShards(shardIntervalList, ShareLock);
OpenTransactionsToAllShardPlacements(shardIntervalList, tableOwner);
foreach(shardIntervalCell, shardIntervalList)
{
ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell);
uint64 shardId = shardInterval->shardId;
ShardConnections *shardConnections = NULL;
bool shardConnectionsFound = false;
char *escapedSchemaName = quote_literal_cstr(schemaName);
char *escapedCommandString = quote_literal_cstr(commandString);
StringInfo applyCommand = makeStringInfo();
shardConnections = GetShardConnections(shardId, &shardConnectionsFound);
Assert(shardConnectionsFound);
/* build the shard ddl command */
appendStringInfo(applyCommand, WORKER_APPLY_SHARD_DDL_COMMAND, shardId,
escapedSchemaName, escapedCommandString);
ExecuteCommandOnShardPlacements(applyCommand, shardId, shardConnections);
FreeStringInfo(applyCommand);
}
/* check for cancellation one last time before returning */
CHECK_FOR_INTERRUPTS();
return true;
}
/*
* ExecuteCommandOnShardPlacements executes the given ddl command on the
* placements of the given shard, using the given shard connections.
*/
static void
ExecuteCommandOnShardPlacements(StringInfo applyCommand, uint64 shardId,
ShardConnections *shardConnections)
{
List *connectionList = shardConnections->connectionList;
ListCell *connectionCell = NULL;
Assert(connectionList != NIL);
foreach(connectionCell, connectionList)
{
TransactionConnection *transactionConnection =
(TransactionConnection *) lfirst(connectionCell);
PGconn *connection = transactionConnection->connection;
PGresult *result = NULL;
/* send the query */
result = PQexec(connection, applyCommand->data);
if (PQresultStatus(result) != PGRES_TUPLES_OK)
{
WarnRemoteError(connection, result);
ereport(ERROR, (errmsg("could not execute DDL command on worker "
"node shards")));
}
else
{
char *workerName = ConnectionGetOptionValue(connection, "host");
char *workerPort = ConnectionGetOptionValue(connection, "port");
ereport(DEBUG2, (errmsg("applied command on shard " UINT64_FORMAT
" on node %s:%s", shardId, workerName,
workerPort)));
}
PQclear(result);
transactionConnection->transactionState = TRANSACTION_STATE_OPEN;
CHECK_FOR_INTERRUPTS();
}
}
/*
* Before acquiring a table lock, check whether we have sufficient rights.
* In the case of DROP INDEX, also try to lock the table before the index.
@ -1761,7 +1631,6 @@ ReplicateGrantStmt(Node *parsetree)
RangeVar *relvar = (RangeVar *) lfirst(objectCell);
Oid relOid = RangeVarGetRelid(relvar, NoLock, false);
const char *grantOption = "";
bool isTopLevel = true;
if (!IsDistributedTable(relOid))
{
@ -1794,7 +1663,7 @@ ReplicateGrantStmt(Node *parsetree)
granteesString.data);
}
ExecuteDistributedDDLCommand(relOid, ddlString.data, isTopLevel);
ExecuteDistributedDDLCommand(relOid, ddlString.data);
resetStringInfo(&ddlString);
}
}

View File

@ -26,7 +26,6 @@
#include "catalog/namespace.h"
#include "catalog/pg_class.h"
#include "distributed/connection_cache.h"
#include "distributed/listutils.h"
#include "distributed/master_metadata_utility.h"
#include "distributed/master_protocol.h"

View File

@ -24,6 +24,7 @@
#include "access/xact.h"
#include "catalog/namespace.h"
#include "commands/dbcommands.h"
#include "distributed/connection_management.h"
#include "distributed/master_protocol.h"
#include "distributed/multi_client_executor.h"
#include "distributed/multi_join_order.h"
@ -32,6 +33,7 @@
#include "distributed/multi_server_executor.h"
#include "distributed/pg_dist_partition.h"
#include "distributed/pg_dist_shard.h"
#include "distributed/remote_commands.h"
#include "distributed/relay_utility.h"
#include "distributed/worker_protocol.h"
#include "lib/stringinfo.h"
@ -190,11 +192,10 @@ master_drop_all_shards(PG_FUNCTION_ARGS)
char *schemaName = NULL;
char *relationName = NULL;
bool isTopLevel = true;
List *shardIntervalList = NIL;
int droppedShardCount = 0;
PreventTransactionChain(isTopLevel, "DROP distributed table");
BeginOrContinueCoordinatedTransaction();
relationName = get_rel_name(relationId);
@ -253,9 +254,11 @@ master_drop_sequences(PG_FUNCTION_ARGS)
ArrayIterator sequenceIterator = NULL;
Datum sequenceText = 0;
bool isNull = false;
MultiConnection *connection = NULL;
StringInfo dropSeqCommand = makeStringInfo();
BeginOrContinueCoordinatedTransaction();
/* iterate over sequence names to build single command to DROP them all */
sequenceIterator = array_create_iterator(sequenceNamesArray, 0, NULL);
while (array_iterate(sequenceIterator, &sequenceText, &isNull))
@ -280,7 +283,9 @@ master_drop_sequences(PG_FUNCTION_ARGS)
appendStringInfo(dropSeqCommand, " %s", TextDatumGetCString(sequenceText));
}
dropSuccessful = ExecuteRemoteCommand(nodeName, nodePort, dropSeqCommand);
connection = GetNodeConnection(NEW_CONNECTION | CACHED_CONNECTION,
nodeName, nodePort);
dropSuccessful = ExecuteCheckStatement(connection, dropSeqCommand->data);
if (!dropSuccessful)
{
ereport(WARNING, (errmsg("could not delete sequences from node \"%s:" INT64_FORMAT
@ -305,15 +310,15 @@ DropShards(Oid relationId, char *schemaName, char *relationName,
{
ListCell *shardIntervalCell = NULL;
int droppedShardCount = 0;
List *commandList = NIL;
ListCell *commandCell = NULL;
BeginOrContinueCoordinatedTransaction();
foreach(shardIntervalCell, deletableShardIntervalList)
{
List *shardPlacementList = NIL;
List *droppedPlacementList = NIL;
List *lingeringPlacementList = NIL;
ListCell *shardPlacementCell = NULL;
ListCell *droppedPlacementCell = NULL;
ListCell *lingeringPlacementCell = NULL;
ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell);
uint64 shardId = shardInterval->shardId;
char *quotedShardName = NULL;
@ -328,14 +333,11 @@ DropShards(Oid relationId, char *schemaName, char *relationName,
shardPlacementList = ShardPlacementList(shardId);
foreach(shardPlacementCell, shardPlacementList)
{
ShardPlacement *shardPlacement =
(ShardPlacement *) lfirst(shardPlacementCell);
char *workerName = shardPlacement->nodeName;
uint32 workerPort = shardPlacement->nodePort;
bool dropSuccessful = false;
ShardPlacement *placement = (ShardPlacement *) lfirst(shardPlacementCell);
BatchCommand *command = (BatchCommand *) palloc0(sizeof(BatchCommand));
StringInfo workerDropQuery = makeStringInfo();
char storageType = shardInterval->storageType;
if (storageType == SHARD_STORAGE_TABLE)
{
appendStringInfo(workerDropQuery, DROP_REGULAR_TABLE_COMMAND,
@ -348,58 +350,45 @@ DropShards(Oid relationId, char *schemaName, char *relationName,
quotedShardName);
}
dropSuccessful = ExecuteRemoteCommand(workerName, workerPort,
workerDropQuery);
if (dropSuccessful)
{
droppedPlacementList = lappend(droppedPlacementList, shardPlacement);
}
else
{
lingeringPlacementList = lappend(lingeringPlacementList, shardPlacement);
}
command->placement = placement;
command->connectionFlags = NEW_CONNECTION | CACHED_CONNECTION | FOR_DDL;
command->commandString = workerDropQuery->data;
command->userData = shardRelationName; /* for failure reporting */
commandList = lappend(commandList, command);
}
/* make sure we don't process cancel signals */
HOLD_INTERRUPTS();
DeleteShardRow(shardId);
}
foreach(droppedPlacementCell, droppedPlacementList)
{
ShardPlacement *placement = (ShardPlacement *) lfirst(droppedPlacementCell);
char *workerName = placement->nodeName;
uint32 workerPort = placement->nodePort;
ExecuteBatchCommands(commandList);
DeleteShardPlacementRow(shardId, workerName, workerPort);
}
foreach(commandCell, commandList)
{
BatchCommand *command = (BatchCommand *) lfirst(commandCell);
ShardPlacement *placement = command->placement;
uint64 shardId = placement->shardId;
uint64 placementId = placement->placementId;
char *workerName = placement->nodeName;
uint32 workerPort = placement->nodePort;
uint64 oldShardLength = placement->shardLength;
const char *shardName = command->userData;
/* mark shard placements that we couldn't drop as to be deleted */
foreach(lingeringPlacementCell, lingeringPlacementList)
if (command->failed)
{
ShardPlacement *placement = (ShardPlacement *) lfirst(lingeringPlacementCell);
uint64 placementId = placement->placementId;
char *workerName = placement->nodeName;
uint32 workerPort = placement->nodePort;
uint64 oldShardLength = placement->shardLength;
DeleteShardPlacementRow(shardId, workerName, workerPort);
InsertShardPlacementRow(shardId, placementId, FILE_TO_DELETE, oldShardLength,
workerName, workerPort);
ereport(WARNING, (errmsg("could not delete shard \"%s\" on node \"%s:%u\"",
shardRelationName, workerName, workerPort),
shardName, workerName, workerPort),
errdetail("Marking this shard placement for deletion")));
}
DeleteShardRow(shardId);
if (QueryCancelPending)
else
{
ereport(WARNING, (errmsg("cancel requests are ignored during shard "
"deletion")));
QueryCancelPending = false;
DeleteShardPlacementRow(shardId, workerName, workerPort);
}
RESUME_INTERRUPTS();
}
droppedShardCount = list_length(deletableShardIntervalList);

View File

@ -26,8 +26,7 @@
#include "commands/event_trigger.h"
#include "distributed/citus_clauses.h"
#include "distributed/citus_ruleutils.h"
#include "distributed/commit_protocol.h"
#include "distributed/connection_cache.h"
#include "distributed/connection_management.h"
#include "distributed/listutils.h"
#include "distributed/master_metadata_utility.h"
#include "distributed/master_protocol.h"
@ -37,9 +36,9 @@
#include "distributed/multi_router_executor.h"
#include "distributed/multi_router_planner.h"
#include "distributed/multi_server_executor.h"
#include "distributed/multi_shard_transaction.h"
#include "distributed/pg_dist_shard.h"
#include "distributed/pg_dist_partition.h"
#include "distributed/remote_commands.h"
#include "distributed/resource_lock.h"
#include "distributed/worker_protocol.h"
#include "optimizer/clauses.h"
@ -57,9 +56,6 @@
static void LockShardsForModify(List *shardIntervalList);
static bool HasReplication(List *shardIntervalList);
static int SendQueryToShards(Query *query, List *shardIntervalList, Oid relationId);
static int SendQueryToPlacements(char *shardQueryString,
ShardConnections *shardConnections);
PG_FUNCTION_INFO_V1(master_modify_multiple_shards);
@ -91,6 +87,8 @@ master_modify_multiple_shards(PG_FUNCTION_ARGS)
PreventTransactionChain(isTopLevel, "master_modify_multiple_shards");
BeginCoordinatedTransaction();
queryTreeNode = ParseTreeNode(queryString);
if (IsA(queryTreeNode, DeleteStmt))
{
@ -161,12 +159,10 @@ master_modify_multiple_shards(PG_FUNCTION_ARGS)
prunedShardIntervalList =
PruneShardList(relationId, tableId, restrictClauseList, shardIntervalList);
CHECK_FOR_INTERRUPTS();
LockShardsForModify(prunedShardIntervalList);
affectedTupleCount = SendQueryToShards(modifyQuery, prunedShardIntervalList,
relationId);
affectedTupleCount = ExecuteQueryOnPlacements(modifyQuery, prunedShardIntervalList,
relationId);
PG_RETURN_INT32(affectedTupleCount);
}
@ -227,119 +223,3 @@ HasReplication(List *shardIntervalList)
return hasReplication;
}
/*
* SendQueryToShards executes the given query in all placements of the given
* shard list and returns the total affected tuple count. The execution is done
* in a distributed transaction and the commit protocol is decided according to
* the value of citus.multi_shard_commit_protocol parameter. SendQueryToShards
* does not acquire locks for the shards so it is advised to acquire locks to
* the shards when necessary before calling SendQueryToShards.
*/
static int
SendQueryToShards(Query *query, List *shardIntervalList, Oid relationId)
{
int affectedTupleCount = 0;
char *relationOwner = TableOwner(relationId);
ListCell *shardIntervalCell = NULL;
OpenTransactionsToAllShardPlacements(shardIntervalList, relationOwner);
foreach(shardIntervalCell, shardIntervalList)
{
ShardInterval *shardInterval = (ShardInterval *) lfirst(
shardIntervalCell);
Oid relationId = shardInterval->relationId;
uint64 shardId = shardInterval->shardId;
bool shardConnectionsFound = false;
ShardConnections *shardConnections = NULL;
StringInfo shardQueryString = makeStringInfo();
char *shardQueryStringData = NULL;
int shardAffectedTupleCount = -1;
shardConnections = GetShardConnections(shardId, &shardConnectionsFound);
Assert(shardConnectionsFound);
deparse_shard_query(query, relationId, shardId, shardQueryString);
shardQueryStringData = shardQueryString->data;
shardAffectedTupleCount = SendQueryToPlacements(shardQueryStringData,
shardConnections);
affectedTupleCount += shardAffectedTupleCount;
}
/* check for cancellation one last time before returning */
CHECK_FOR_INTERRUPTS();
return affectedTupleCount;
}
/*
* SendQueryToPlacements sends the given query string to all given placement
* connections of a shard. CommitRemoteTransactions or AbortRemoteTransactions
* should be called after all queries have been sent successfully.
*/
static int
SendQueryToPlacements(char *shardQueryString, ShardConnections *shardConnections)
{
uint64 shardId = shardConnections->shardId;
List *connectionList = shardConnections->connectionList;
ListCell *connectionCell = NULL;
int32 shardAffectedTupleCount = -1;
Assert(connectionList != NIL);
foreach(connectionCell, connectionList)
{
TransactionConnection *transactionConnection =
(TransactionConnection *) lfirst(connectionCell);
PGconn *connection = transactionConnection->connection;
PGresult *result = NULL;
char *placementAffectedTupleString = NULL;
int32 placementAffectedTupleCount = -1;
CHECK_FOR_INTERRUPTS();
/* send the query */
result = PQexec(connection, shardQueryString);
if (PQresultStatus(result) != PGRES_COMMAND_OK)
{
WarnRemoteError(connection, result);
ereport(ERROR, (errmsg("could not send query to shard placement")));
}
placementAffectedTupleString = PQcmdTuples(result);
/* returned tuple count is empty for utility commands, use 0 as affected count */
if (*placementAffectedTupleString == '\0')
{
placementAffectedTupleCount = 0;
}
else
{
placementAffectedTupleCount = pg_atoi(placementAffectedTupleString,
sizeof(int32), 0);
}
if ((shardAffectedTupleCount == -1) ||
(shardAffectedTupleCount == placementAffectedTupleCount))
{
shardAffectedTupleCount = placementAffectedTupleCount;
}
else
{
ereport(ERROR,
(errmsg("modified %d tuples, but expected to modify %d",
placementAffectedTupleCount, shardAffectedTupleCount),
errdetail("Affected tuple counts at placements of shard "
UINT64_FORMAT " are different.", shardId)));
}
PQclear(result);
transactionConnection->transactionState = TRANSACTION_STATE_OPEN;
}
return shardAffectedTupleCount;
}

View File

@ -16,18 +16,21 @@
#include "postgres.h"
#include "funcapi.h"
#include "libpq-fe.h"
#include "miscadmin.h"
#include "access/htup_details.h"
#include "access/xact.h"
#include "catalog/indexing.h"
#include "distributed/multi_client_executor.h"
#include "distributed/connection_management.h"
#include "distributed/master_metadata_utility.h"
#include "distributed/master_protocol.h"
#include "distributed/metadata_cache.h"
#include "distributed/multi_join_order.h"
#include "distributed/pg_dist_partition.h"
#include "distributed/pg_dist_shard.h"
#include "distributed/remote_commands.h"
#include "distributed/resource_lock.h"
#include "distributed/worker_manager.h"
#include "distributed/worker_protocol.h"
@ -582,11 +585,8 @@ WorkerShardStats(char *nodeName, uint32 nodePort, Oid relationId, char *shardNam
char *partitionColumnName = NULL;
StringInfo partitionValueQuery = makeStringInfo();
int32 connectionId = -1;
bool queryOK = false;
void *queryResult = NULL;
int rowCount = 0;
int columnCount = 0;
MultiConnection *connection = NULL;
PGresult *result = NULL;
const int minValueIndex = 0;
const int maxValueIndex = 1;
@ -600,11 +600,8 @@ WorkerShardStats(char *nodeName, uint32 nodePort, Oid relationId, char *shardNam
*shardMinValue = NULL;
*shardMaxValue = NULL;
connectionId = MultiClientConnect(nodeName, nodePort, NULL, NULL);
if (connectionId == INVALID_CONNECTION_ID)
{
return false;
}
connection = GetNodeConnection(NEW_CONNECTION | CACHED_CONNECTION,
nodeName, nodePort);
quotedShardName = quote_literal_cstr(shardName);
@ -618,18 +615,19 @@ WorkerShardStats(char *nodeName, uint32 nodePort, Oid relationId, char *shardNam
appendStringInfo(tableSizeQuery, SHARD_TABLE_SIZE_QUERY, quotedShardName);
}
queryOK = MultiClientExecute(connectionId, tableSizeQuery->data,
&queryResult, &rowCount, &columnCount);
if (!queryOK)
result = ExecuteStatement(connection, tableSizeQuery->data);
if (!result)
{
MultiClientDisconnect(connectionId);
return false;
}
tableSizeString = MultiClientGetValue(queryResult, 0, 0);
tableSizeString = PQgetvalue(result, 0, 0);
if (tableSizeString == NULL)
{
MultiClientDisconnect(connectionId);
PQclear(result);
result = PQgetResult(connection->conn);
Assert(result == NULL);
return false;
}
@ -637,20 +635,21 @@ WorkerShardStats(char *nodeName, uint32 nodePort, Oid relationId, char *shardNam
tableSize = strtoull(tableSizeString, &tableSizeStringEnd, 0);
if (errno != 0 || (*tableSizeStringEnd) != '\0')
{
MultiClientClearResult(queryResult);
MultiClientDisconnect(connectionId);
PQclear(result);
result = PQgetResult(connection->conn);
Assert(result == NULL);
return false;
}
*shardSize = tableSize;
MultiClientClearResult(queryResult);
PQclear(result);
result = PQgetResult(connection->conn);
Assert(result == NULL);
if (partitionType != DISTRIBUTE_BY_APPEND)
{
/* we don't need min/max for non-append distributed tables */
MultiClientDisconnect(connectionId);
return true;
}
@ -661,28 +660,27 @@ WorkerShardStats(char *nodeName, uint32 nodePort, Oid relationId, char *shardNam
appendStringInfo(partitionValueQuery, SHARD_RANGE_QUERY,
partitionColumnName, partitionColumnName, shardName);
queryOK = MultiClientExecute(connectionId, partitionValueQuery->data,
&queryResult, &rowCount, &columnCount);
if (!queryOK)
result = ExecuteStatement(connection, partitionValueQuery->data);
if (!result)
{
MultiClientDisconnect(connectionId);
return false;
}
minValueIsNull = MultiClientValueIsNull(queryResult, 0, minValueIndex);
maxValueIsNull = MultiClientValueIsNull(queryResult, 0, maxValueIndex);
minValueIsNull = PQgetisnull(result, 0, minValueIndex);
maxValueIsNull = PQgetisnull(result, 0, maxValueIndex);
if (!minValueIsNull && !maxValueIsNull)
{
char *minValueResult = MultiClientGetValue(queryResult, 0, minValueIndex);
char *maxValueResult = MultiClientGetValue(queryResult, 0, maxValueIndex);
char *minValueResult = PQgetvalue(result, 0, minValueIndex);
char *maxValueResult = PQgetvalue(result, 0, maxValueIndex);
*shardMinValue = cstring_to_text(minValueResult);
*shardMaxValue = cstring_to_text(maxValueResult);
}
MultiClientClearResult(queryResult);
MultiClientDisconnect(connectionId);
PQclear(result);
result = PQgetResult(connection->conn);
Assert(result == NULL);
return true;
}

View File

@ -18,7 +18,7 @@
#include "commands/explain.h"
#include "executor/executor.h"
#include "distributed/commit_protocol.h"
#include "distributed/connection_management.h"
#include "distributed/master_protocol.h"
#include "distributed/multi_copy.h"
#include "distributed/multi_executor.h"
@ -29,9 +29,11 @@
#include "distributed/multi_router_executor.h"
#include "distributed/multi_router_planner.h"
#include "distributed/multi_server_executor.h"
#include "distributed/multi_shard_transaction.h"
#include "distributed/multi_utility.h"
#include "distributed/placement_connection.h"
#include "distributed/remote_commands.h"
#include "distributed/task_tracker.h"
#include "distributed/transaction_management.h"
#include "distributed/worker_manager.h"
#include "distributed/worker_protocol.h"
#include "postmaster/postmaster.h"
@ -150,9 +152,10 @@ _PG_init(void)
/* organize that task tracker is started once server is up */
TaskTrackerRegister();
/* initialize transaction callbacks */
RegisterRouterExecutorXactCallbacks();
RegisterShardPlacementXactCallbacks();
/* initialize coordinated transaction management */
InitializeTransactionManagement();
InitializeConnectionManagement();
InitPlacementConnectionManagement();
}
@ -259,6 +262,16 @@ RegisterCitusConfigVariables(void)
GUC_NO_SHOW_ALL,
NULL, NULL, NULL);
DefineCustomBoolVariable(
"citus.log_remote_commands",
gettext_noop("Log queries sent to other nodes in the server log"),
NULL,
&LogRemoteCommands,
false,
PGC_USERSET,
0,
NULL, NULL, NULL);
DefineCustomBoolVariable(
"citus.explain_multi_logical_plan",
gettext_noop("Enables Explain to print out distributed logical plans."),

View File

@ -1,264 +0,0 @@
/*-------------------------------------------------------------------------
*
* commit_protocol.c
* This file contains functions for managing 1PC or 2PC transactions
* across many shard placements.
*
* Copyright (c) 2016, Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "libpq-fe.h"
#include "miscadmin.h"
#include "distributed/commit_protocol.h"
#include "distributed/connection_cache.h"
#include "distributed/master_metadata_utility.h"
#include "distributed/multi_shard_transaction.h"
#include "lib/stringinfo.h"
#include "nodes/pg_list.h"
/* Local functions forward declarations */
static uint32 DistributedTransactionId = 0;
/* Local functions forward declarations */
static StringInfo BuildTransactionName(int connectionId);
/* the commit protocol to use for COPY commands */
int MultiShardCommitProtocol = COMMIT_PROTOCOL_1PC;
/*
* InitializeDistributedTransaction prepares the distributed transaction ID
* used in transaction names.
*/
void
InitializeDistributedTransaction(void)
{
DistributedTransactionId++;
}
/*
* PrepareRemoteTransactions prepares all transactions on connections in
* connectionList for commit if the 2PC commit protocol is enabled.
* On failure, it reports an error and stops.
*/
void
PrepareRemoteTransactions(List *connectionList)
{
ListCell *connectionCell = NULL;
foreach(connectionCell, connectionList)
{
TransactionConnection *transactionConnection =
(TransactionConnection *) lfirst(connectionCell);
PGconn *connection = transactionConnection->connection;
int64 connectionId = transactionConnection->connectionId;
PGresult *result = NULL;
StringInfo command = makeStringInfo();
StringInfo transactionName = BuildTransactionName(connectionId);
appendStringInfo(command, "PREPARE TRANSACTION '%s'", transactionName->data);
result = PQexec(connection, command->data);
if (PQresultStatus(result) != PGRES_COMMAND_OK)
{
/* a failure to prepare is an implicit rollback */
transactionConnection->transactionState = TRANSACTION_STATE_CLOSED;
WarnRemoteError(connection, result);
PQclear(result);
ereport(ERROR, (errcode(ERRCODE_IO_ERROR),
errmsg("failed to prepare transaction")));
}
ereport(DEBUG2, (errmsg("sent PREPARE TRANSACTION over connection %ld",
connectionId)));
PQclear(result);
transactionConnection->transactionState = TRANSACTION_STATE_PREPARED;
}
}
/*
* AbortRemoteTransactions aborts all transactions on connections in connectionList.
* On failure, it reports a warning and continues to abort all of them.
*/
void
AbortRemoteTransactions(List *connectionList)
{
ListCell *connectionCell = NULL;
foreach(connectionCell, connectionList)
{
TransactionConnection *transactionConnection =
(TransactionConnection *) lfirst(connectionCell);
PGconn *connection = transactionConnection->connection;
int64 connectionId = transactionConnection->connectionId;
PGresult *result = NULL;
if (transactionConnection->transactionState == TRANSACTION_STATE_PREPARED)
{
StringInfo command = makeStringInfo();
StringInfo transactionName = BuildTransactionName(connectionId);
appendStringInfo(command, "ROLLBACK PREPARED '%s'", transactionName->data);
result = PQexec(connection, command->data);
if (PQresultStatus(result) != PGRES_COMMAND_OK)
{
char *nodeName = ConnectionGetOptionValue(connection, "host");
char *nodePort = ConnectionGetOptionValue(connection, "port");
/* log a warning so the user may abort the transaction later */
ereport(WARNING, (errmsg("failed to roll back prepared transaction '%s'",
transactionName->data),
errhint("Run \"%s\" on %s:%s",
command->data, nodeName, nodePort)));
}
ereport(DEBUG2, (errmsg("sent ROLLBACK over connection %ld", connectionId)));
PQclear(result);
}
else if (transactionConnection->transactionState == TRANSACTION_STATE_OPEN)
{
/* try to roll back cleanly, if it fails then we won't commit anyway */
result = PQexec(connection, "ROLLBACK");
PQclear(result);
}
transactionConnection->transactionState = TRANSACTION_STATE_CLOSED;
}
}
/*
* CommitRemoteTransactions commits all transactions on connections in connectionList.
* If stopOnFailure is true, then CommitRemoteTransactions reports an error on
* failure, otherwise it reports a warning.
* Note that if the caller of this function wants the transactions to roll back
* on a failing commit, stopOnFailure should be used as true. On the other hand,
* if the caller does not want the transactions to roll back on a failing commit,
* stopOnFailure should be used as false.
*/
void
CommitRemoteTransactions(List *connectionList, bool stopOnFailure)
{
ListCell *connectionCell = NULL;
foreach(connectionCell, connectionList)
{
TransactionConnection *transactionConnection =
(TransactionConnection *) lfirst(connectionCell);
PGconn *connection = transactionConnection->connection;
int64 connectionId = transactionConnection->connectionId;
PGresult *result = NULL;
if (transactionConnection->transactionState == TRANSACTION_STATE_PREPARED)
{
StringInfo command = makeStringInfo();
StringInfo transactionName = BuildTransactionName(connectionId);
/* we shouldn't be committing if any transactions are not prepared */
Assert(transactionConnection->transactionState == TRANSACTION_STATE_PREPARED);
appendStringInfo(command, "COMMIT PREPARED '%s'", transactionName->data);
result = PQexec(connection, command->data);
if (PQresultStatus(result) != PGRES_COMMAND_OK)
{
char *nodeName = ConnectionGetOptionValue(connection, "host");
char *nodePort = ConnectionGetOptionValue(connection, "port");
/*
* If stopOnFailure is false, log a warning so the user may
* commit the transaction later.
*/
if (stopOnFailure)
{
ereport(ERROR, (errmsg("failed to commit prepared transaction '%s'",
transactionName->data),
errhint("Run \"%s\" on %s:%s",
command->data, nodeName, nodePort)));
}
else
{
ereport(WARNING, (errmsg("failed to commit prepared transaction '%s'",
transactionName->data),
errhint("Run \"%s\" on %s:%s",
command->data, nodeName, nodePort)));
}
}
ereport(DEBUG2, (errmsg("sent COMMIT PREPARED over connection %ld",
connectionId)));
}
else
{
/* we shouldn't be committing if any transactions are not open */
Assert(transactionConnection->transactionState == TRANSACTION_STATE_OPEN);
/*
* Try to commit, if it fails and stopOnFailure is false then
* the user might lose data.
*/
result = PQexec(connection, "COMMIT");
if (PQresultStatus(result) != PGRES_COMMAND_OK)
{
char *nodeName = ConnectionGetOptionValue(connection, "host");
char *nodePort = ConnectionGetOptionValue(connection, "port");
if (stopOnFailure)
{
ereport(ERROR, (errmsg("failed to commit transaction on %s:%s",
nodeName, nodePort)));
}
else
{
ereport(WARNING, (errmsg("failed to commit transaction on %s:%s",
nodeName, nodePort)));
}
}
ereport(DEBUG2, (errmsg("sent COMMIT over connection %ld", connectionId)));
}
PQclear(result);
transactionConnection->transactionState = TRANSACTION_STATE_CLOSED;
}
}
/*
* BuildTransactionName constructs a transaction name that ensures there are no
* collisions with concurrent transactions by the same master node, subsequent
* transactions by the same backend, or transactions on a different shard.
*
* Collisions may occur over time if transactions fail to commit or abort and
* are left to linger. This would cause a PREPARE failure for the second
* transaction, which causes it to be rolled back. In general, the user
* should ensure that prepared transactions do not linger.
*/
static StringInfo
BuildTransactionName(int connectionId)
{
StringInfo commandString = makeStringInfo();
appendStringInfo(commandString, "citus_%d_%u_%d", MyProcPid,
DistributedTransactionId, connectionId);
return commandString;
}

View File

@ -1,342 +0,0 @@
/*-------------------------------------------------------------------------
*
* multi_shard_transaction.c
* This file contains functions for managing 1PC or 2PC transactions
* across many shard placements.
*
* Copyright (c) 2016, Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "libpq-fe.h"
#include "postgres.h"
#include "distributed/commit_protocol.h"
#include "distributed/connection_cache.h"
#include "distributed/master_metadata_utility.h"
#include "distributed/multi_shard_transaction.h"
#include "nodes/pg_list.h"
#include "storage/ipc.h"
#include "utils/memutils.h"
#define INITIAL_CONNECTION_CACHE_SIZE 1001
/* Global variables used in commit handler */
static HTAB *shardConnectionHash = NULL;
static bool subXactAbortAttempted = false;
/* functions needed by callbacks and hooks */
static void CompleteShardPlacementTransactions(XactEvent event, void *arg);
static void MultiShardSubXactCallback(SubXactEvent event, SubTransactionId subId,
SubTransactionId parentSubid, void *arg);
/*
* OpenTransactionsToAllShardPlacements opens connections to all placements
* using the provided shard identifier list. Connections accumulate in a global
* shardConnectionHash variable for use (and re-use) within this transaction.
*/
void
OpenTransactionsToAllShardPlacements(List *shardIntervalList, char *userName)
{
ListCell *shardIntervalCell = NULL;
if (shardConnectionHash == NULL)
{
shardConnectionHash = CreateShardConnectionHash(TopTransactionContext);
}
foreach(shardIntervalCell, shardIntervalList)
{
ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell);
uint64 shardId = shardInterval->shardId;
BeginTransactionOnShardPlacements(shardId, userName);
}
}
/*
* CreateShardConnectionHash constructs a hash table which maps from shard
* identifier to connection lists, passing the provided MemoryContext to
* hash_create for hash allocations.
*/
HTAB *
CreateShardConnectionHash(MemoryContext memoryContext)
{
HTAB *shardConnectionsHash = NULL;
int hashFlags = 0;
HASHCTL info;
memset(&info, 0, sizeof(info));
info.keysize = sizeof(int64);
info.entrysize = sizeof(ShardConnections);
info.hcxt = memoryContext;
hashFlags = (HASH_ELEM | HASH_CONTEXT | HASH_BLOBS);
shardConnectionsHash = hash_create("Shard Connections Hash",
INITIAL_CONNECTION_CACHE_SIZE, &info,
hashFlags);
return shardConnectionsHash;
}
/*
* BeginTransactionOnShardPlacements opens new connections (if necessary) to
* all placements of a shard (specified by shard identifier). After sending a
* BEGIN command on all connections, they are added to shardConnectionHash for
* use within this transaction. Exits early if connections already exist for
* the specified shard, and errors if no placements can be found, a connection
* cannot be made, or if the BEGIN command fails.
*/
void
BeginTransactionOnShardPlacements(uint64 shardId, char *userName)
{
List *shardPlacementList = NIL;
ListCell *placementCell = NULL;
ShardConnections *shardConnections = NULL;
bool shardConnectionsFound = false;
MemoryContext oldContext = NULL;
shardPlacementList = FinalizedShardPlacementList(shardId);
if (shardPlacementList == NIL)
{
/* going to have to have some placements to do any work */
ereport(ERROR, (errmsg("could not find any shard placements for the shard "
UINT64_FORMAT, shardId)));
}
/* get existing connections to the shard placements, if any */
shardConnections = GetShardConnections(shardId, &shardConnectionsFound);
if (shardConnectionsFound)
{
/* exit early if we've already established shard transactions */
return;
}
foreach(placementCell, shardPlacementList)
{
ShardPlacement *shardPlacement = (ShardPlacement *) lfirst(placementCell);
PGconn *connection = NULL;
TransactionConnection *transactionConnection = NULL;
PGresult *result = NULL;
connection = ConnectToNode(shardPlacement->nodeName, shardPlacement->nodePort,
userName);
if (connection == NULL)
{
ereport(ERROR, (errmsg("could not establish a connection to all "
"placements of shard %lu", shardId)));
}
/* entries must last through the whole top-level transaction */
oldContext = MemoryContextSwitchTo(TopTransactionContext);
transactionConnection = palloc0(sizeof(TransactionConnection));
transactionConnection->connectionId = shardConnections->shardId;
transactionConnection->transactionState = TRANSACTION_STATE_INVALID;
transactionConnection->connection = connection;
shardConnections->connectionList = lappend(shardConnections->connectionList,
transactionConnection);
MemoryContextSwitchTo(oldContext);
/* now that connection is tracked, issue BEGIN */
result = PQexec(connection, "BEGIN");
if (PQresultStatus(result) != PGRES_COMMAND_OK)
{
ReraiseRemoteError(connection, result);
}
}
}
/*
* GetShardConnections finds existing connections for a shard in the global
* connection hash. If not found, then a ShardConnections structure with empty
* connectionList is returned and the shardConnectionsFound output parameter
* will be set to false.
*/
ShardConnections *
GetShardConnections(int64 shardId, bool *shardConnectionsFound)
{
return GetShardHashConnections(shardConnectionHash, shardId, shardConnectionsFound);
}
/*
* GetShardHashConnections finds existing connections for a shard in the
* provided hash. If not found, then a ShardConnections structure with empty
* connectionList is returned.
*/
ShardConnections *
GetShardHashConnections(HTAB *connectionHash, int64 shardId, bool *connectionsFound)
{
ShardConnections *shardConnections = NULL;
shardConnections = (ShardConnections *) hash_search(connectionHash, &shardId,
HASH_ENTER, connectionsFound);
if (!*connectionsFound)
{
shardConnections->shardId = shardId;
shardConnections->connectionList = NIL;
}
return shardConnections;
}
/*
* ConnectionList flattens the connection hash to a list of placement connections.
*/
List *
ConnectionList(HTAB *connectionHash)
{
List *connectionList = NIL;
HASH_SEQ_STATUS status;
ShardConnections *shardConnections = NULL;
if (connectionHash == NULL)
{
return NIL;
}
hash_seq_init(&status, connectionHash);
shardConnections = (ShardConnections *) hash_seq_search(&status);
while (shardConnections != NULL)
{
List *shardConnectionsList = list_copy(shardConnections->connectionList);
connectionList = list_concat(connectionList, shardConnectionsList);
shardConnections = (ShardConnections *) hash_seq_search(&status);
}
return connectionList;
}
/*
* RegisterShardPlacementXactCallbacks registers transaction callbacks needed
* for multi-shard transactions.
*/
void
RegisterShardPlacementXactCallbacks(void)
{
RegisterXactCallback(CompleteShardPlacementTransactions, NULL);
RegisterSubXactCallback(MultiShardSubXactCallback, NULL);
}
/*
* CompleteShardPlacementTransactions commits or aborts pending shard placement
* transactions when the local transaction commits or aborts.
*/
static void
CompleteShardPlacementTransactions(XactEvent event, void *arg)
{
List *connectionList = ConnectionList(shardConnectionHash);
if (shardConnectionHash == NULL)
{
/* nothing to do */
return;
}
if (event == XACT_EVENT_PRE_COMMIT)
{
if (subXactAbortAttempted)
{
subXactAbortAttempted = false;
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot ROLLBACK TO SAVEPOINT in transactions "
"which modify distributed tables")));
}
/*
* Any failure here will cause local changes to be rolled back,
* and remote changes to either roll back (1PC) or, in case of
* connection or node failure, leave a prepared transaction
* (2PC).
*/
if (MultiShardCommitProtocol == COMMIT_PROTOCOL_2PC)
{
PrepareRemoteTransactions(connectionList);
}
return;
}
else if (event == XACT_EVENT_COMMIT)
{
/*
* A failure here will cause some remote changes to either
* roll back (1PC) or, in case of connection or node failure,
* leave a prepared transaction (2PC). However, the local
* changes have already been committed.
*/
CommitRemoteTransactions(connectionList, false);
}
else if (event == XACT_EVENT_ABORT)
{
/*
* A failure here will cause some remote changes to either
* roll back (1PC) or, in case of connection or node failure,
* leave a prepared transaction (2PC). The local changes have
* already been rolled back.
*/
AbortRemoteTransactions(connectionList);
}
else
{
return;
}
CloseConnections(connectionList);
shardConnectionHash = NULL;
XactModificationLevel = XACT_MODIFICATION_NONE;
subXactAbortAttempted = false;
}
static void
MultiShardSubXactCallback(SubXactEvent event, SubTransactionId subId,
SubTransactionId parentSubid, void *arg)
{
if ((shardConnectionHash != NULL) && (event == SUBXACT_EVENT_ABORT_SUB))
{
subXactAbortAttempted = true;
}
}
/*
* CloseConnections closes all connections in connectionList.
*/
void
CloseConnections(List *connectionList)
{
ListCell *connectionCell = NULL;
foreach(connectionCell, connectionList)
{
TransactionConnection *transactionConnection =
(TransactionConnection *) lfirst(connectionCell);
PGconn *connection = transactionConnection->connection;
PQfinish(connection);
}
}

View File

@ -0,0 +1,546 @@
/*-------------------------------------------------------------------------
*
* remote_transaction.c
* Management of transaction spanning more than one node.
*
* Copyright (c) 2016, Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "libpq-fe.h"
#include "miscadmin.h"
#include "access/xact.h"
#include "distributed/connection_management.h"
#include "distributed/transaction_management.h"
#include "distributed/remote_commands.h"
#include "distributed/remote_transaction.h"
#include "utils/hsearch.h"
static void Assign2PCIdentifier(MultiConnection *connection);
static void WarnAboutLeakedPreparedTransaction(MultiConnection *connection, bool commit);
/*
* Begin, if necessary according to this session's coordinated transaction
* state, and the connection's state, an explicit transaction on all the
* connections. This is done in parallel, to lessen latency penalties.
*/
void
AdjustRemoteTransactionStates(List *connectionList)
{
ListCell *connectionCell = NULL;
if (!InCoordinatedTransaction())
{
return;
}
/* issue BEGIN to all connections needing it */
foreach(connectionCell, connectionList)
{
MultiConnection *connection = (MultiConnection *) lfirst(connectionCell);
RemoteTransaction *transaction = &connection->remoteTransaction;
Assert(PQtransactionStatus(connection->conn) != PQTRANS_ACTIVE);
if (transaction->transactionFailed ||
transaction->transactionState != REMOTE_TRANS_INVALID)
{
continue;
}
if (PQtransactionStatus(connection->conn) != PQTRANS_INTRANS)
{
/*
* Check whether we're right now allowed to start new client
* transaction. FIXME: This likely can be removed soon.
*/
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")));
}
/*
* Explicitly specify READ COMMITTED, the default on the remote
* side might have been changed, and that would cause problematic
* behaviour.
*/
if (!SendRemoteCommand(connection, "BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;"))
{
ReportConnectionError(connection, WARNING);
MarkRemoteTransactionFailed(connection, true);
}
else
{
transaction->transactionState = REMOTE_TRANS_STARTING;
}
}
}
/* get result of all the BEGINs */
foreach(connectionCell, connectionList)
{
MultiConnection *connection = (MultiConnection *) lfirst(connectionCell);
RemoteTransaction *transaction = &connection->remoteTransaction;
PGresult *result = NULL;
if (transaction->transactionFailed)
{
continue;
}
if (!(transaction->transactionState == REMOTE_TRANS_STARTING))
{
continue;
}
result = PQgetResult(connection->conn);
if (!IsResponseOK(result))
{
ReportResultError(connection, result, WARNING);
MarkRemoteTransactionFailed(connection, true);
}
else
{
transaction->transactionState = REMOTE_TRANS_STARTED;
}
PQclear(result);
ForgetResults(connection);
Assert(PQtransactionStatus(connection->conn) == PQTRANS_INTRANS);
}
}
/*
* Begin, if necessary according to this session's coordinated transaction
* state, and the connection's state, an explicit transaction on the
* connections.
*/
void
AdjustRemoteTransactionState(MultiConnection *connection)
{
/* just delegate */
if (InCoordinatedTransaction())
{
List *connectionList = list_make1(connection);
AdjustRemoteTransactionStates(connectionList);
list_free(connectionList);
}
}
/*
* Record a connection as being failed. That'll, if a coordinated transaction
* is in progress, mean coordinated transactions will take appropriate action
* to handle with the failure.
*
* If the connection is marked as critical, and allowErrorPromotion is true,
* this routine will ERROR out. The allowErrorPromotion case is primarily
* required for the transaction management code itself. Usually it is helpful
* to fail as soon as possible.
*/
void
MarkRemoteTransactionFailed(MultiConnection *connection, bool allowErrorPromotion)
{
RemoteTransaction *transaction = &connection->remoteTransaction;
transaction->transactionFailed = true;
/*
* If the connection is marked as critical, fail the entire coordinated
* transaction. If allowed.
*/
if (transaction->criticalTransaction && allowErrorPromotion)
{
ereport(ERROR, (errmsg("failure on connection marked as essential: %s:%d",
connection->hostname, connection->port)));
}
}
/*
* Perform distributed transactions handling at commit time. This will be
* called at XACT_EVENT_PRE_COMMIT if 1PC commits are used - so shards can
* still be invalidated - and at XACT_EVENT_COMMIT if 2PC is being used.
*
* Note that this routine has to issue rollbacks for failed transactions. In
* that case affected placements will be marked as invalid (via
* CheckForFailedPlacements()).
*/
void
CoordinatedRemoteTransactionsCommit(void)
{
HASH_SEQ_STATUS status;
ConnectionHashEntry *entry;
ListCell *connectionCell;
/*
* Issue appropriate transaction commands to remote nodes. If everything
* went well that's going to be COMMIT or COMMIT PREPARED, if individual
* connections had errors, some or all of them might require a ROLLBACK.
*/
hash_seq_init(&status, ConnectionHash);
while ((entry = (ConnectionHashEntry *) hash_seq_search(&status)) != 0)
{
foreach(connectionCell, entry->connections)
{
MultiConnection *connection = (MultiConnection *) lfirst(connectionCell);
RemoteTransaction *transaction = &connection->remoteTransaction;
/* nothing to do if no transaction active */
if (transaction->transactionState == REMOTE_TRANS_INVALID)
{
continue;
}
if (transaction->transactionFailed)
{
/*
* Try sending an ROLLBACK; Depending on the state that won't
* succeed, but let's try. Have to clear previous results
* first.
*/
ForgetResults(connection); /* try to clear pending stuff */
if (!SendRemoteCommand(connection, "ROLLBACK;"))
{
/* no point in reporting a likely redundant message */
MarkRemoteTransactionFailed(connection, false);
}
else
{
transaction->transactionState = REMOTE_TRANS_1PC_ABORTING;
}
}
else if (transaction->transactionState == REMOTE_TRANS_PREPARED)
{
StringInfoData command;
initStringInfo(&command);
appendStringInfo(&command, "COMMIT PREPARED '%s';",
transaction->preparedName);
transaction->transactionState = REMOTE_TRANS_2PC_COMMITTING;
if (!SendRemoteCommand(connection, command.data))
{
ReportConnectionError(connection, WARNING);
MarkRemoteTransactionFailed(connection, false);
WarnAboutLeakedPreparedTransaction(connection, true);
}
}
else
{
transaction->transactionState = REMOTE_TRANS_1PC_COMMITTING;
if (!SendRemoteCommand(connection, "COMMIT;"))
{
/* for a moment there I thought we were in trouble */
ReportConnectionError(connection, WARNING);
MarkRemoteTransactionFailed(connection, false);
}
}
}
}
/* Wait for result */
hash_seq_init(&status, ConnectionHash);
while ((entry = (ConnectionHashEntry *) hash_seq_search(&status)) != 0)
{
foreach(connectionCell, entry->connections)
{
MultiConnection *connection = (MultiConnection *) lfirst(connectionCell);
RemoteTransaction *transaction = &connection->remoteTransaction;
PGresult *result = NULL;
if (transaction->transactionState != REMOTE_TRANS_1PC_COMMITTING &&
transaction->transactionState != REMOTE_TRANS_2PC_COMMITTING &&
transaction->transactionState != REMOTE_TRANS_1PC_ABORTING &&
transaction->transactionState != REMOTE_TRANS_2PC_ABORTING)
{
continue;
}
result = PQgetResult(connection->conn);
if (!IsResponseOK(result))
{
ReportResultError(connection, result, WARNING);
MarkRemoteTransactionFailed(connection, false);
if (transaction->transactionState == REMOTE_TRANS_2PC_COMMITTING)
{
WarnAboutLeakedPreparedTransaction(connection, true);
}
}
else if (transaction->transactionState == REMOTE_TRANS_1PC_ABORTING ||
transaction->transactionState == REMOTE_TRANS_2PC_ABORTING)
{
transaction->transactionState = REMOTE_TRANS_ABORTED;
}
else
{
transaction->transactionState = REMOTE_TRANS_COMMITTED;
}
PQclear(result);
ForgetResults(connection);
}
}
}
/*
* Perform distributed transactions handling at abort time. This issues
* ROLLBACKS and ROLLBACK PREPARED depending on whether the remote transaction
* has been prepared or not.
*/
void
CoordinatedRemoteTransactionsAbort(void)
{
HASH_SEQ_STATUS status;
ConnectionHashEntry *entry;
ListCell *connectionCell;
/* issue ROLLBACK; to all relevant remote nodes */
hash_seq_init(&status, ConnectionHash);
while ((entry = (ConnectionHashEntry *) hash_seq_search(&status)) != 0)
{
foreach(connectionCell, entry->connections)
{
MultiConnection *connection = (MultiConnection *) lfirst(connectionCell);
RemoteTransaction *transaction = &connection->remoteTransaction;
if (transaction->transactionState == REMOTE_TRANS_INVALID)
{
continue;
}
if (transaction->transactionState == REMOTE_TRANS_PREPARED)
{
StringInfoData command;
initStringInfo(&command);
appendStringInfo(&command, "ROLLBACK PREPARED '%s';",
transaction->preparedName);
if (!SendRemoteCommand(connection, command.data))
{
ReportConnectionError(connection, WARNING);
MarkRemoteTransactionFailed(connection, false);
WarnAboutLeakedPreparedTransaction(connection, false);
}
else
{
transaction->transactionState = REMOTE_TRANS_2PC_ABORTING;
}
}
else
{
/*
* Try sending an ROLLBACK; Depending on the state
* that won't have success, but let's try. Have
* to clear previous results first.
*/
ForgetResults(connection);
if (!SendRemoteCommand(connection, "ROLLBACK;"))
{
/* no point in reporting a likely redundant message */
MarkRemoteTransactionFailed(connection, false);
}
else
{
transaction->transactionState = REMOTE_TRANS_1PC_ABORTING;
}
}
}
}
/* Wait for result */
hash_seq_init(&status, ConnectionHash);
while ((entry = (ConnectionHashEntry *) hash_seq_search(&status)) != 0)
{
foreach(connectionCell, entry->connections)
{
MultiConnection *connection = (MultiConnection *) lfirst(connectionCell);
RemoteTransaction *transaction = &connection->remoteTransaction;
PGresult *result = NULL;
if (transaction->transactionState != REMOTE_TRANS_1PC_ABORTING &&
transaction->transactionState != REMOTE_TRANS_2PC_ABORTING)
{
continue;
}
result = PQgetResult(connection->conn);
if (!IsResponseOK(result))
{
ReportResultError(connection, result, WARNING);
MarkRemoteTransactionFailed(connection, false);
if (transaction->transactionState == REMOTE_TRANS_1PC_ABORTING)
{
ereport(WARNING,
(errmsg("failed to abort 2PC transaction \"%s\" on %s:%d",
transaction->preparedName, connection->hostname,
connection->port)));
}
else
{
WarnAboutLeakedPreparedTransaction(connection, false);
}
}
PQclear(result);
result = PQgetResult(connection->conn);
Assert(!result);
transaction->transactionState = REMOTE_TRANS_ABORTED;
}
}
}
/*
* Perform 2PC prepare on all non-failed transactions participating in the
* coordinated transaction.
*/
void
CoordinatedRemoteTransactionsPrepare(void)
{
HASH_SEQ_STATUS status;
ConnectionHashEntry *entry;
ListCell *connectionCell;
/* issue PREPARE TRANSACTION; to all relevant remote nodes */
/* TODO: skip connections that haven't done any DML/DDL */
hash_seq_init(&status, ConnectionHash);
while ((entry = (ConnectionHashEntry *) hash_seq_search(&status)) != 0)
{
foreach(connectionCell, entry->connections)
{
MultiConnection *connection = (MultiConnection *) lfirst(connectionCell);
RemoteTransaction *transaction = &connection->remoteTransaction;
if (transaction->transactionState == REMOTE_TRANS_INVALID)
{
continue;
}
if (!(transaction->transactionFailed))
{
StringInfoData command;
initStringInfo(&command);
Assign2PCIdentifier(connection);
appendStringInfo(&command, "PREPARE TRANSACTION '%s'",
transaction->preparedName);
if (!SendRemoteCommand(connection, command.data))
{
ReportConnectionError(connection, WARNING);
MarkRemoteTransactionFailed(connection, false);
}
else
{
transaction->transactionState = REMOTE_TRANS_PREPARING;
}
}
}
}
/* Wait for result */
hash_seq_init(&status, ConnectionHash);
while ((entry = (ConnectionHashEntry *) hash_seq_search(&status)) != 0)
{
foreach(connectionCell, entry->connections)
{
MultiConnection *connection = (MultiConnection *) lfirst(connectionCell);
RemoteTransaction *transaction = &connection->remoteTransaction;
PGresult *result = NULL;
if (transaction->transactionState != REMOTE_TRANS_PREPARING)
{
continue;
}
result = PQgetResult(connection->conn);
if (!IsResponseOK(result))
{
ReportResultError(connection, result, WARNING);
MarkRemoteTransactionFailed(connection, false);
}
else
{
transaction->transactionState = REMOTE_TRANS_PREPARED;
}
result = PQgetResult(connection->conn);
Assert(!result);
}
}
CurrentCoordinatedTransactionState = COORD_TRANS_PREPARED;
}
/*
* Compute the 2PC transaction name to use. Every 2PC transaction should get a
* new name, i.e. this function will need to be called again.
*
* NB: we rely on the fact that we don't need to do full escaping on the names
* generated here.
*/
static void
Assign2PCIdentifier(MultiConnection *connection)
{
static uint64 sequence = 0;
snprintf(connection->remoteTransaction.preparedName, NAMEDATALEN,
"citus_%d_"UINT64_FORMAT,
MyProcPid, sequence++);
}
static void
WarnAboutLeakedPreparedTransaction(MultiConnection *connection, bool commit)
{
StringInfoData command;
RemoteTransaction *transaction = &connection->remoteTransaction;
initStringInfo(&command);
if (commit)
appendStringInfo(&command, "COMMIT PREPARED '%s';",
transaction->preparedName);
else
appendStringInfo(&command, "ROLLBACK PREPARED '%s';",
transaction->preparedName);
/* log a warning so the user may abort the transaction later */
ereport(WARNING, (errmsg("failed to roll back prepared transaction '%s'",
transaction->preparedName),
errhint("Run \"%s\" on %s:%u",
command.data, connection->hostname, connection->port)));
}

View File

@ -0,0 +1,292 @@
/*-------------------------------------------------------------------------
*
* transaction_management.c
*
* Transaction management for Citus. Most of the work is delegated to other
* subsystems, this files, and especially CoordinatedTransactionCallback,
* coordinates the work between them.
*
*
* The standard pattern to perform work spanning this and remote nodes, is to:
*
* 1) Call BeginOrContinueCoordinatedTransaction(). This signals that work
* on remote nodes should be done inside explicit transactions. If that's
* not desired, e.g. inside router executor, this step should be skipped.
*
* 2) Acquire a connection to either the remote node (using
* GetNodeConnection() or similar) or one associated with a placement
* (using GetPlacementConnection() or similar). Always use the latter
* when performing work associated with a placement. Use the
* FOR_DML/FOR_DDL flags if appropriate.
*
* 3) Call AdjustRemoteTransactionState() or AdjustRemoteTransactionStates()
* on all connections used. The latter should be used if multiple
* connections are in use, since it is considerably faster.
*
* 4) Perform work on the connection, either using MultiConnection->conn
* directly via libpq, or using some of the remote_command.h helpers.
*
* 5) Done. If the local transaction commits/aborts, the remote
* transaction(s) are going to be committed/aborted as well. If a
* placement has been modified (DML or DDL flag to
* GetPlacementConnnection()) and the remote transaction failed,
* placements will be marked as invalid, or the entire transaction will
* be aborted, as appropriate.
*
*
* This subsystem delegates work to several subsystems:
* - connection lifecycle management is handled in connection_management.[ch]
* - transaction on remote nodes are managed via remote_transaction.[ch]
* - per-placement visibility, locking and invalidation resides in
* placement_connection.[ch]
* - simple and complex commands on other nodes can be executed via
* remote_commands.[ch]
*
*
* Copyright (c) 2016, Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "libpq-fe.h"
#include "miscadmin.h"
#include "access/xact.h"
#include "distributed/connection_management.h"
#include "distributed/hash_helpers.h"
#include "distributed/transaction_management.h"
#include "distributed/placement_connection.h"
#include "utils/hsearch.h"
CoordinatedTransactionState CurrentCoordinatedTransactionState = COORD_TRANS_NONE;
/* GUC, the commit protocol to use for commands affecting more than one connection */
int MultiShardCommitProtocol = COMMIT_PROTOCOL_1PC;
/* state needed to keep track of operations used during a transaction */
XactModificationType XactModificationLevel = XACT_MODIFICATION_NONE;
static bool subXactAbortAttempted = false;
/* transaction management functions */
static void CoordinatedTransactionCallback(XactEvent event, void *arg);
static void CoordinatedSubTransactionCallback(SubXactEvent event, SubTransactionId subId,
SubTransactionId parentSubid, void *arg);
/*
* BeginCoordinatedTransaction begins a coordinated transaction. No
* pre-existing coordinated transaction may be in progress.
*/
void
BeginCoordinatedTransaction(void)
{
if (CurrentCoordinatedTransactionState != COORD_TRANS_NONE &&
CurrentCoordinatedTransactionState != COORD_TRANS_IDLE)
{
ereport(ERROR, (errmsg("starting transaction in wrong state")));
}
CurrentCoordinatedTransactionState = COORD_TRANS_STARTED;
}
/*
* BeginOrContinueCoordinatedTransaction starts a coordinated transaction,
* unless one already is in progress.
*/
void
BeginOrContinueCoordinatedTransaction(void)
{
if (CurrentCoordinatedTransactionState == COORD_TRANS_STARTED)
{
return;
}
BeginCoordinatedTransaction();
}
/*
* InCoordinatedTransaction returns whether a coordinated transaction has been
* started.
*/
bool
InCoordinatedTransaction(void)
{
return CurrentCoordinatedTransactionState != COORD_TRANS_NONE &&
CurrentCoordinatedTransactionState != COORD_TRANS_IDLE;
}
void
InitializeTransactionManagement(void)
{
/* hook into transaction machinery */
RegisterXactCallback(CoordinatedTransactionCallback, NULL);
RegisterSubXactCallback(CoordinatedSubTransactionCallback, NULL);
}
/*
* Transaction management callback, handling coordinated transaction, and
* transaction independent connection management.
*
* NB: There should only ever be a single transaction callback in citus, the
* ordering between the callbacks and thee actions within those callbacks
* otherwise becomes too undeterministic / hard to reason about.
*/
static void
CoordinatedTransactionCallback(XactEvent event, void *arg)
{
switch (event)
{
case XACT_EVENT_COMMIT:
{
if (CurrentCoordinatedTransactionState == COORD_TRANS_PREPARED)
{
/* handles both already prepared and open transactions */
CoordinatedRemoteTransactionsCommit();
}
/* close connections etc. */
if (CurrentCoordinatedTransactionState != COORD_TRANS_NONE)
{
ResetPlacementConnectionManagement();
AtEOXact_Connections(true);
}
Assert(!subXactAbortAttempted);
CurrentCoordinatedTransactionState = COORD_TRANS_NONE;
XactModificationLevel = XACT_MODIFICATION_NONE;
}
break;
case XACT_EVENT_ABORT:
{
/*
* FIXME: Add warning for the COORD_TRANS_COMMITTED case. That
* can be reached if this backend fails after the
* XACT_EVENT_PRE_COMMIT state.
*/
/* handles both already prepared and open transactions */
if (CurrentCoordinatedTransactionState > COORD_TRANS_IDLE)
{
CoordinatedRemoteTransactionsAbort();
}
/* close connections etc. */
if (CurrentCoordinatedTransactionState != COORD_TRANS_NONE)
{
ResetPlacementConnectionManagement();
AtEOXact_Connections(false);
}
CurrentCoordinatedTransactionState = COORD_TRANS_NONE;
XactModificationLevel = XACT_MODIFICATION_NONE;
subXactAbortAttempted = false;
}
break;
case XACT_EVENT_PARALLEL_COMMIT:
case XACT_EVENT_PARALLEL_ABORT:
case XACT_EVENT_PREPARE:
{ }
break;
case XACT_EVENT_PRE_COMMIT:
{
bool using2PC = MultiShardCommitProtocol == COMMIT_PROTOCOL_2PC;
if (subXactAbortAttempted)
{
subXactAbortAttempted = false;
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot ROLLBACK TO SAVEPOINT in transactions "
"which modify distributed tables")));
}
/* nothing further to do if there's no managed remote xacts */
if (CurrentCoordinatedTransactionState == COORD_TRANS_NONE)
{
break;
}
/*
* TODO: It's probably a good idea to force constraints and
* such to 'immediate' here. Deferred triggers might try to
* send stuff to the remote side, which'd not be good. Doing
* so remotely would also catch a class of errors where
* committing fails, which can lead to divergence when not
* using 2PC.
*/
/*
* Check whether the coordinated transaction is in a state we want
* to persist, or whether we want to error out. This handles the
* case that iteratively executed commands marked all placements
* as invalid.
*/
CheckForFailedPlacements(true, using2PC);
if (MultiShardCommitProtocol == COMMIT_PROTOCOL_2PC)
{
CoordinatedRemoteTransactionsPrepare();
CurrentCoordinatedTransactionState = COORD_TRANS_PREPARED;
}
else
{
/*
* Have to commit remote transactions in PRE_COMMIT, to allow
* us to mark failed placements as invalid. Better don't use
* this for anything important (i.e. DDL/metadata).
*/
CoordinatedRemoteTransactionsCommit();
CurrentCoordinatedTransactionState = COORD_TRANS_COMMITTED;
}
/*
* Check again whether shards/placement successfully
* committed. This handles failure at COMMIT/PREPARE time.
*/
CheckForFailedPlacements(false, using2PC);
}
break;
case XACT_EVENT_PARALLEL_PRE_COMMIT:
case XACT_EVENT_PRE_PREPARE:
{
/*
* FIXME: do we want to support this? Or error out? Might be
* annoying to error out as it could prevent experimentation. If
* we error out, we should only do so if a coordinated transaction
* has been started, so independent 2PC usage doesn't cause
* errors.
*/
}
break;
}
}
/*
* Subtransaction callback - currently only used to remember whether a
* savepoint has been rolled back, as we don't support that.
*/
static void
CoordinatedSubTransactionCallback(SubXactEvent event, SubTransactionId subId,
SubTransactionId parentSubid, void *arg)
{
if (event == SUBXACT_EVENT_ABORT_SUB)
{
subXactAbortAttempted = true;
}
}

View File

@ -29,10 +29,6 @@
#include "utils/memutils.h"
#include "utils/palloc.h"
/* state needed to keep track of operations used during a transaction */
XactModificationType XactModificationLevel = XACT_MODIFICATION_NONE;
/*
* NodeConnectionHash is the connection hash itself. It begins uninitialized.
* The first call to GetOrEstablishConnection triggers hash creation.
@ -209,37 +205,6 @@ PurgeConnectionByKey(NodeConnectionKey *nodeConnectionKey)
}
/*
* SqlStateMatchesCategory returns true if the given sql state (which may be
* NULL if unknown) is in the given error category. Note that we use
* ERRCODE_TO_CATEGORY macro to determine error category of the sql state and
* expect the caller to use the same macro for the error category.
*/
bool
SqlStateMatchesCategory(char *sqlStateString, int category)
{
bool sqlStateMatchesCategory = false;
int sqlState = 0;
int sqlStateCategory = 0;
if (sqlStateString == NULL)
{
return false;
}
sqlState = MAKE_SQLSTATE(sqlStateString[0], sqlStateString[1], sqlStateString[2],
sqlStateString[3], sqlStateString[4]);
sqlStateCategory = ERRCODE_TO_CATEGORY(sqlState);
if (sqlStateCategory == category)
{
sqlStateMatchesCategory = true;
}
return sqlStateMatchesCategory;
}
/*
* WarnRemoteError retrieves error fields from a remote result and produces an
* error report at the WARNING level after amending the error with a CONTEXT

View File

@ -0,0 +1,543 @@
/*-------------------------------------------------------------------------
*
* connection_management.c
* Central management of connections and their life-cycle
*
* Copyright (c) 2016, Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#ifdef HAVE_POLL_H
#include <poll.h>
#endif
#include "libpq-fe.h"
#include "miscadmin.h"
#include "access/hash.h"
#include "commands/dbcommands.h"
#include "distributed/connection_management.h"
#include "distributed/metadata_cache.h"
#include "distributed/hash_helpers.h"
#include "mb/pg_wchar.h"
#include "utils/hsearch.h"
#include "utils/memutils.h"
HTAB *ConnectionHash = NULL;
MemoryContext ConnectionContext = NULL;
static uint32 ConnectionHashHash(const void *key, Size keysize);
static int ConnectionHashCompare(const void *a, const void *b, Size keysize);
static MultiConnection * StartConnectionEstablishment(ConnectionHashKey *key);
/*
* Initialize per-backend connection management infrastructure.
*/
void
InitializeConnectionManagement(void)
{
HASHCTL info;
uint32 hashFlags = 0;
/*
* Create a single context for connection and transaction related memory
* management. Doing so, instead of allocating in TopMemoryContext, makes
* it easier to associate used memory.
*/
ConnectionContext = AllocSetContextCreate(TopMemoryContext, "Connection Context",
ALLOCSET_DEFAULT_MINSIZE,
ALLOCSET_DEFAULT_INITSIZE,
ALLOCSET_DEFAULT_MAXSIZE);
/* create (host,port,user,database) -> [connection] hash */
memset(&info, 0, sizeof(info));
info.keysize = sizeof(ConnectionHashKey);
info.entrysize = sizeof(ConnectionHashEntry);
info.hash = ConnectionHashHash;
info.match = ConnectionHashCompare;
info.hcxt = ConnectionContext;
hashFlags = (HASH_ELEM | HASH_FUNCTION | HASH_CONTEXT | HASH_COMPARE);
ConnectionHash = hash_create("citus connection cache (host,port,user,database)",
64, &info, hashFlags);
}
/*
* Perform connection management activity after the end of a transaction. Both
* COMMIT and ABORT paths are handled here.
*
* This is called by Citus' global transaction callback.
*/
void
AtEOXact_Connections(bool isCommit)
{
HASH_SEQ_STATUS status;
ConnectionHashEntry *entry;
/*
* Close all remote connections if necessary anymore (i.e. not session
* lifetime), or if in a failed state.
*/
hash_seq_init(&status, ConnectionHash);
while ((entry = (ConnectionHashEntry *) hash_seq_search(&status)) != 0)
{
ListCell *previousCell = NULL;
ListCell *nextCell = NULL;
ListCell *connectionCell = NULL;
/*
* Have to iterate "manually", to be able to delete connections in the
* middle of the list.
*/
for (connectionCell = list_head(entry->connections);
connectionCell != NULL;
connectionCell = nextCell)
{
MultiConnection *connection = (MultiConnection *) lfirst(connectionCell);
nextCell = lnext(connectionCell);
/*
* To avoid code leaking connections we warn if connections are
* still claimed exclusively. We can only do so if the transaction
* committed, as it's normal that code didn't have chance to clean
* up after errors.
*/
if (isCommit && connection->claimedExclusively)
{
ereport(WARNING,
(errmsg("connection claimed exclusively at transaction commit")));
}
/*
* Only let a connection life longer than a single transaction if
* instructed to do so by the caller. We also skip doing so if
* it's in a state that wouldn't allow us to run queries again.
*/
if (!connection->sessionLifespan ||
PQstatus(connection->conn) != CONNECTION_OK ||
PQtransactionStatus(connection->conn) != PQTRANS_IDLE)
{
PQfinish(connection->conn);
connection->conn = NULL;
entry->connections =
list_delete_cell(entry->connections, connectionCell, previousCell);
pfree(connection);
}
else
{
/* reset per-transaction state */
connection->activeInTransaction = false;
connection->remoteTransaction.transactionState = REMOTE_TRANS_INVALID;
connection->remoteTransaction.transactionFailed = false;
UnclaimConnection(connection);
previousCell = connectionCell;
}
}
/*
* NB: We leave the hash entry in place, even if there's no individual
* connections in it anymore. There seems no benefit in deleting it,
* and it'll save a bit of work in the next transaction.
*/
}
}
/*
* GetNodeConnection() establishes a connection to remote node, using default
* user and database.
*
* See StartNodeUserDatabaseConnection for details.
*/
MultiConnection *
GetNodeConnection(uint32 flags, const char *hostname, int32 port)
{
return GetNodeUserDatabaseConnection(flags, hostname, port, NULL, NULL);
}
/*
* StartNodeConnection initiate a connection to remote node, using default
* user and database.
*
* See StartNodeUserDatabaseConnection for details.
*/
MultiConnection *
StartNodeConnection(uint32 flags, const char *hostname, int32 port)
{
return StartNodeUserDatabaseConnection(flags, hostname, port, NULL, NULL);
}
/*
* GetNodeUserDatabaseConnection establishes connection to remote node.
*
* See StartNodeUserDatabaseConnection for details.
*/
MultiConnection *
GetNodeUserDatabaseConnection(uint32 flags, const char *hostname, int32 port, const
char *user, const char *database)
{
MultiConnection *connection;
connection = StartNodeUserDatabaseConnection(flags, hostname, port, user, database);
FinishConnectionEstablishment(connection);
return connection;
}
/*
* StartNodeUserDatabaseConnection() initiates a connection to a remote node.
*
* If user or database are NULL, the current session's defaults are used. The
* following flags influence connection establishment behaviour:
* - NEW_CONNECTION - it is permitted to establish a new connection
* - CACHED_CONNECTION - it is permitted to re-use an established connection
* - SESSION_LIFESPAN - the connection should persist after transaction end
* - FOR_DML - only meaningful for placement associated connections
* - FOR_DDL - only meaningful for placement associated connections
* - CRITICAL_CONNECTION - transaction failures on this connection fail the entire
* coordinated transaction
*
* The returned connection has only been initiated, not fully
* established. That's useful to allow parallel connection establishment. If
* that's not desired use the Get* variant.
*/
MultiConnection *
StartNodeUserDatabaseConnection(uint32 flags, const char *hostname, int32 port, const
char *user, const char *database)
{
ConnectionHashKey key;
ConnectionHashEntry *entry = NULL;
MultiConnection *connection;
MemoryContext oldContext;
bool found;
strlcpy(key.hostname, hostname, MAX_NODE_LENGTH);
key.port = port;
if (user)
{
strlcpy(key.user, user, NAMEDATALEN);
}
else
{
strlcpy(key.user, CurrentUserName(), NAMEDATALEN);
}
if (database)
{
strlcpy(key.database, database, NAMEDATALEN);
}
else
{
strlcpy(key.database, get_database_name(MyDatabaseId), NAMEDATALEN);
}
if (CurrentCoordinatedTransactionState == COORD_TRANS_NONE)
{
CurrentCoordinatedTransactionState = COORD_TRANS_IDLE;
}
/*
* Lookup relevant hash entry. We always enter. If only a cached
* connection is desired, and there's none, we'll simply leave the
* connection list empty.
*/
entry = hash_search(ConnectionHash, &key, HASH_ENTER, &found);
if (!found)
{
entry->connections = NIL;
}
if (flags & CACHED_CONNECTION)
{
ListCell *connectionCell = NULL;
/* check connection cache for a connection that's not already in use */
foreach(connectionCell, entry->connections)
{
connection = (MultiConnection *) lfirst(connectionCell);
/* don't return claimed connections */
if (!connection->claimedExclusively)
{
if (flags & SESSION_LIFESPAN)
{
connection->sessionLifespan = true;
}
connection->activeInTransaction = true;
/*
* Check whether we're right now allowed to open new
* connections. A cached connection counts as new if it hasn't
* been used in this transaction.
*
* FIXME: This should be removed soon, once all connections go
* through this API.
*/
if (!connection->activeInTransaction &&
XactModificationLevel > XACT_MODIFICATION_DATA)
{
ereport(ERROR, (errcode(ERRCODE_ACTIVE_SQL_TRANSACTION),
errmsg("cannot open new connections after the first "
"modification command within a transaction")));
}
return connection;
}
/*
* One could argue for erroring out when the connection is in a
* failed state. But that'd be a bad idea for two reasons:
*
* 1) Generally starting a connection might fail, after calling
* this function, so calling code needs to handle that anyway.
* 2) This might be used in code that transparently handles
* connection failure.
*/
}
/* no connection available, done if a new connection isn't desirable */
if (!(flags & NEW_CONNECTION))
{
return NULL;
}
}
/*
* Check whether we're right now allowed to open new connections.
*
* FIXME: This should be removed soon, once all connections go through
* this API.
*/
if (XactModificationLevel > XACT_MODIFICATION_DATA)
{
ereport(ERROR, (errcode(ERRCODE_ACTIVE_SQL_TRANSACTION),
errmsg("cannot open new connections after the first modification "
"command within a transaction")));
}
/*
* Either no caching desired, or no pre-established, non-claimed,
* connection present. Initiate connection establishment.
*/
connection = StartConnectionEstablishment(&key);
oldContext = MemoryContextSwitchTo(ConnectionContext);
entry->connections = lappend(entry->connections, connection);
MemoryContextSwitchTo(oldContext);
if (flags & SESSION_LIFESPAN)
{
connection->sessionLifespan = true;
}
connection->activeInTransaction = true;
return connection;
}
/*
* Synchronously finish connection establishment of an individual connection.
*
* TODO: Replace with variant waiting for multiple connections.
*/
void
FinishConnectionEstablishment(MultiConnection *connection)
{
/*
* Loop until connection is established, or failed (possibly just timed
* out).
*/
while (true)
{
ConnStatusType status = PQstatus(connection->conn);
PostgresPollingStatusType pollmode;
if (status == CONNECTION_OK)
{
return;
}
/* FIXME: retries? */
if (status == CONNECTION_BAD)
{
return;
}
pollmode = PQconnectPoll(connection->conn);
/*
* FIXME: Do we want to add transparent retry support here?
*/
if (pollmode == PGRES_POLLING_FAILED)
{
return;
}
else if (pollmode == PGRES_POLLING_OK)
{
return;
}
else
{
Assert(pollmode == PGRES_POLLING_WRITING ||
pollmode == PGRES_POLLING_READING);
}
/* Loop, to handle poll() being interrupted by signals (EINTR) */
while (true)
{
struct pollfd pollFileDescriptor;
int pollResult = 0;
pollFileDescriptor.fd = PQsocket(connection->conn);
if (pollmode == PGRES_POLLING_READING)
{
pollFileDescriptor.events = POLLIN;
}
else
{
pollFileDescriptor.events = POLLOUT;
}
pollFileDescriptor.revents = 0;
pollResult = poll(&pollFileDescriptor, 1, CLIENT_CONNECT_TIMEOUT_SECONDS_INT);
if (pollResult == 0)
{
/* timeout exceeded */
}
else if (pollResult > 0)
{
/* IO possible, continue connection establishment */
break;
}
else if (pollResult != EINTR)
{
/* retrying, signal */
}
else
{
/*
* We ERROR here, instead of just returning a failed
* connection, because this shouldn't happen, and indicates a
* programming error somewhere, not a network etc. issue.
*/
ereport(ERROR, (errcode_for_socket_access(),
errmsg("poll() failed: %m")));
}
}
}
}
/*
* ClaimConnectionExclusively signals that this connection is actively being
* used. That means it'll not be, again, returned by
* StartNodeUserDatabaseConnection() et al until releases with
* UnclaimConnection().
*/
void
ClaimConnectionExclusively(MultiConnection *connection)
{
Assert(!connection->claimedExclusively);
connection->claimedExclusively = true;
}
/*
* UnclaimConnection signals that this connection is not being used
* anymore. That means it again may be returned by returned by
* StartNodeUserDatabaseConnection() et al.
*/
void
UnclaimConnection(MultiConnection *connection)
{
connection->claimedExclusively = false;
}
static uint32
ConnectionHashHash(const void *key, Size keysize)
{
ConnectionHashKey *entry = (ConnectionHashKey *) key;
uint32 hash = 0;
hash = string_hash(entry->hostname, NAMEDATALEN);
hash = hash_combine(hash, hash_uint32(entry->port));
hash = hash_combine(hash, string_hash(entry->user, NAMEDATALEN));
hash = hash_combine(hash, string_hash(entry->database, NAMEDATALEN));
return hash;
}
static int
ConnectionHashCompare(const void *a, const void *b, Size keysize)
{
ConnectionHashKey *ca = (ConnectionHashKey *) a;
ConnectionHashKey *cb = (ConnectionHashKey *) b;
if (strncmp(ca->hostname, cb->hostname, NAMEDATALEN) != 0 ||
ca->port != cb->port ||
strncmp(ca->user, cb->user, NAMEDATALEN) != 0 ||
strncmp(ca->database, cb->database, NAMEDATALEN) != 0)
{
return 1;
}
else
{
return 0;
}
}
/*
* Asynchronously establish connection to a remote node, but don't wait for
* that to finish. DNS lookups etc. are performed synchronously though.
*/
static MultiConnection *
StartConnectionEstablishment(ConnectionHashKey *key)
{
char nodePortString[12];
const char *clientEncoding = GetDatabaseEncodingName();
MultiConnection *connection = NULL;
const char *keywords[] = {
"host", "port", "dbname", "user",
"client_encoding", "fallback_application_name",
NULL
};
const char *values[] = {
key->hostname, nodePortString, key->database, key->user,
clientEncoding, "citus", NULL
};
connection = MemoryContextAllocZero(ConnectionContext, sizeof(MultiConnection));
sprintf(nodePortString, "%d", key->port);
strlcpy(connection->hostname, key->hostname, MAX_NODE_LENGTH);
connection->port = key->port;
strlcpy(connection->database, key->database, NAMEDATALEN);
strlcpy(connection->user, key->user, NAMEDATALEN);
connection->conn = PQconnectStartParams(keywords, values, false);
return connection;
}

View File

@ -0,0 +1,34 @@
/*-------------------------------------------------------------------------
*
* hash_helpers.c
* Helpers for dynahash.c style hash tables.
*
* Copyright (c) 2016, Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "distributed/hash_helpers.h"
#include "utils/hsearch.h"
/*
* Empty a hash, without destroying the hash table itself.
*/
void
hash_delete_all(HTAB *htab)
{
HASH_SEQ_STATUS status;
void *entry = NULL;
hash_seq_init(&status, htab);
while ((entry = hash_seq_search(&status)) != 0)
{
bool found = false;
hash_search(htab, entry, HASH_REMOVE, &found);
Assert(found);
}
}

View File

@ -0,0 +1,361 @@
/*-------------------------------------------------------------------------
*
* placement_connection.c
* Per-Placement connection & transaction handling
*
* Copyright (c) 2016, Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "libpq-fe.h"
#include "miscadmin.h"
#include "access/hash.h"
#include "distributed/connection_management.h"
#include "distributed/placement_connection.h"
#include "distributed/metadata_cache.h"
#include "distributed/hash_helpers.h"
#include "utils/hsearch.h"
#include "utils/memutils.h"
HTAB *ConnectionPlacementHash = NULL;
HTAB *ConnectionShardHash = NULL;
static void AssociatePlacementWithShard(ConnectionPlacementHashEntry *placementEntry,
ShardPlacement *placement);
/*
* GetPlacementConnection establishes a connection for a placement.
*
* See StartPlacementConnection for details.
*/
MultiConnection *
GetPlacementConnection(uint32 flags, ShardPlacement *placement)
{
MultiConnection *connection = StartPlacementConnection(flags, placement);
FinishConnectionEstablishment(connection);
return connection;
}
/*
* StartPlacementConnection() initiates a connection to a remote node,
* associated with the placement and transaction.
*
* The connection is established as the current user & database.
*
* See StartNodeUserDatabaseConnection for details.
*
* Flags have the corresponding meaning from StartNodeUserDatabaseConnection,
* except that two additional flags have an effect:
* - FOR_DML - signal that connection is going to be used for DML (modifications)
* - FOR_DDL - signal that connection is going to be used for DDL
*
* Only one connection associated with the placement may have FOR_DML or
* FOR_DDL set. This restriction prevents deadlocks and wrong results due to
* in-progress transactions.
*/
MultiConnection *
StartPlacementConnection(uint32 flags, ShardPlacement *placement)
{
ConnectionPlacementHashKey key;
ConnectionPlacementHashEntry *placementEntry = NULL;
MemoryContext oldContext = NULL;
bool found = false;
ConnectionReference *returnConnectionReference = NULL;
ListCell *referenceCell = NULL;
key.placementid = placement->placementId;
/* FIXME: not implemented */
Assert(flags & NEW_CONNECTION);
/*
* Lookup relevant hash entry. We always enter. If only a cached
* connection is desired, and there's none, we'll simply leave the
* connection list empty.
*/
placementEntry = hash_search(ConnectionPlacementHash, &key, HASH_ENTER, &found);
if (!found)
{
placementEntry->connectionReferences = NIL;
placementEntry->failed = false;
}
/*
* Check whether any of the connections already associated with the
* placement can be reused, or violates FOR_DML/FOR_DDL constraints.
*/
foreach(referenceCell, placementEntry->connectionReferences)
{
ConnectionReference *connectionReference = NULL;
bool useConnection = false;
MultiConnection *connection = NULL;
connectionReference = (ConnectionReference *) lfirst(referenceCell);
connection = connectionReference->connection;
/* use the connection, unless in a state that's not useful for us */
if (connection->claimedExclusively ||
!((flags & CACHED_CONNECTION)) ||
returnConnectionReference != NULL)
{
useConnection = false;
}
else
{
useConnection = true;
}
/*
* If not using the connection, verify that FOR_DML/DDL flags are
* compatible.
*/
if (useConnection)
{
returnConnectionReference = connectionReference;
}
else if (connectionReference->hadDDL)
{
/* XXX: errcode & errmsg */
ereport(ERROR, (errcode(ERRCODE_ACTIVE_SQL_TRANSACTION),
errmsg("cannot establish new placement connection when other "
"placement executed DDL")));
}
else if (connectionReference->hadDML)
{
/* XXX: errcode & errmsg */
ereport(ERROR, (errcode(ERRCODE_ACTIVE_SQL_TRANSACTION),
errmsg("cannot establish new placement connection when other "
"placement executed DML")));
}
}
/* no connection available, done if a new connection isn't desirable */
if (!(flags & NEW_CONNECTION))
{
return NULL;
}
/*
* Either no caching desired, or no connection present. Start connection
* establishment.
*/
if (returnConnectionReference == NULL)
{
MultiConnection *connection = StartNodeConnection(flags, placement->nodeName,
placement->nodePort);
oldContext = MemoryContextSwitchTo(ConnectionContext);
returnConnectionReference =
(ConnectionReference *) palloc(sizeof(ConnectionReference));
returnConnectionReference->connection = connection;
returnConnectionReference->hadDDL = false;
returnConnectionReference->hadDML = false;
placementEntry->connectionReferences =
lappend(placementEntry->connectionReferences, returnConnectionReference);
MemoryContextSwitchTo(oldContext);
AssociatePlacementWithShard(placementEntry, placement);
}
if (flags & FOR_DDL)
{
returnConnectionReference->hadDDL = true;
}
if (flags & FOR_DML)
{
returnConnectionReference->hadDML = true;
}
if (flags & CRITICAL_CONNECTION)
{
RemoteTransaction *transaction =
&returnConnectionReference->connection->remoteTransaction;
transaction->criticalTransaction = true;
}
return returnConnectionReference->connection;
}
void
InitPlacementConnectionManagement(void)
{
HASHCTL info;
uint32 hashFlags = 0;
/* create (placementid) -> [connection] hash */
memset(&info, 0, sizeof(info));
info.keysize = sizeof(ConnectionPlacementHashKey);
info.entrysize = sizeof(ConnectionPlacementHashEntry);
info.hash = tag_hash;
info.hcxt = ConnectionContext;
hashFlags = (HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
ConnectionPlacementHash = hash_create("citus connection cache (placementid)",
64, &info, hashFlags);
/* create (shardId) -> [ConnectionShardHashEntry] hash */
memset(&info, 0, sizeof(info));
info.keysize = sizeof(ConnectionShardHashKey);
info.entrysize = sizeof(ConnectionShardHashEntry);
info.hash = tag_hash;
info.hcxt = ConnectionContext;
hashFlags = (HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
ConnectionShardHash = hash_create("citus connection cache (shardid)",
64, &info, hashFlags);
}
/*
* Disassociate connections from placements and shards. This will be called at
* the end of XACT_EVENT_COMMIT and XACT_EVENT_ABORT.
*/
void
ResetPlacementConnectionManagement(void)
{
/* Simply delete all entries*/
hash_delete_all(ConnectionPlacementHash);
hash_delete_all(ConnectionShardHash);
}
/*
* Check which placements have to be marked as invalid, and/or whether
* sufficiently many placements have failed to abort the entire coordinated
* transaction.
*
* This will usually be called twice. Once before the remote commit is done,
* and once after. This is so we can abort before executing remote commits,
* and so we can handle remote transactions that failed during commit.
*
* When preCommit or using2PC is true, failures on transactions marked as
* critical will abort the entire coordinated transaction. Otherwise we can't
* anymore, because some remote transactions might have already committed.
*/
void
CheckForFailedPlacements(bool preCommit, bool using2PC)
{
HASH_SEQ_STATUS status;
ConnectionShardHashEntry *shardEntry = NULL;
hash_seq_init(&status, ConnectionShardHash);
while ((shardEntry = (ConnectionShardHashEntry *) hash_seq_search(&status)) != 0)
{
ListCell *placementCell = NULL;
int failures = 0;
int successes = 0;
foreach(placementCell, shardEntry->placementConnections)
{
ConnectionPlacementHashEntry *placementEntry =
(ConnectionPlacementHashEntry *) lfirst(placementCell);
ListCell *referenceCell = NULL;
foreach(referenceCell, placementEntry->connectionReferences)
{
ConnectionReference *reference =
(ConnectionReference *) lfirst(referenceCell);
MultiConnection *connection = reference->connection;
/*
* If neither DDL nor DML were executed, there's no need for
* invalidation.
*/
if (!reference->hadDDL && !reference->hadDML)
{
continue;
}
if (connection->remoteTransaction.transactionFailed)
{
placementEntry->failed = true;
/*
* Raise an error if failure was on a required connection,
* unless we're post-commit and not using 2PC. In that
* case escalating failures here could leave inconsistent
* shards in place, which are not marked as invalid.
*
* XXX: should we warn?
*/
if (preCommit || using2PC)
{
/* to raise ERROR if a required connection */
MarkRemoteTransactionFailed(connection, true);
}
}
}
if (placementEntry->failed)
{
failures++;
}
else
{
successes++;
}
}
if (failures > 0 && successes == 0)
{
/*
* FIXME: arguably we should only error out here if we're
* pre-commit or using 2PC. Otherwise we can end up with a state
* where parts of the transaction is committed and others aren't,
* without correspondingly marking things as invalid (which we
* can't, as we would have already committed).
*/
/* FIXME: better message */
ereport(ERROR, (errmsg("could not commit transaction on any active nodes")));
}
foreach(placementCell, shardEntry->placementConnections)
{
ConnectionPlacementHashEntry *placementEntry =
(ConnectionPlacementHashEntry *) lfirst(placementCell);
if (placementEntry->failed)
{
UpdateShardPlacementState(placementEntry->key.placementid, FILE_INACTIVE);
}
}
}
}
/* Record shard->placement relation */
static void
AssociatePlacementWithShard(ConnectionPlacementHashEntry *placementEntry,
ShardPlacement *placement)
{
ConnectionShardHashKey shardKey;
ConnectionShardHashEntry *shardEntry = NULL;
bool found = false;
MemoryContext oldContext = NULL;
shardKey.shardId = placement->shardId;
shardEntry = hash_search(ConnectionShardHash, &shardKey, HASH_ENTER, &found);
if (!found)
{
shardEntry->placementConnections = NIL;
}
oldContext = MemoryContextSwitchTo(ConnectionContext);
shardEntry->placementConnections =
list_append_unique_ptr(shardEntry->placementConnections, placementEntry);
MemoryContextSwitchTo(oldContext);
}

View File

@ -0,0 +1,703 @@
/*-------------------------------------------------------------------------
*
* remote_commands.c
* Helpers to make it easier to execute command on remote nodes.
*
* Copyright (c) 2016, Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "libpq-fe.h"
#include "distributed/connection_management.h"
#include "distributed/citus_ruleutils.h"
#include "distributed/master_protocol.h"
#include "distributed/metadata_cache.h"
#include "distributed/remote_commands.h"
#include "distributed/resource_lock.h"
#include "distributed/transaction_management.h"
#include "utils/builtins.h"
#include "utils/int8.h"
#include "utils/lsyscache.h"
/* GUC, determining whether statements sent to remote nodes are logged */
bool LogRemoteCommands = false;
static BatchCommand ** BatchCommandListToArray(List *batchCommandList);
static int CompareBatchCommands(const void *leftElement, const void *rightElement);
static void HandlePlacementFailures(List *goodPlacements, List *failedPlacements);
/* simple helpers */
/*
* IsResponseOK checks whether the result is a successful one.
*/
bool
IsResponseOK(PGresult *result)
{
ExecStatusType resultStatus = PQresultStatus(result);
if (resultStatus == PGRES_SINGLE_TUPLE || resultStatus == PGRES_TUPLES_OK ||
resultStatus == PGRES_COMMAND_OK)
{
return true;
}
return false;
}
/*
* Clear connection from current activity.
*
* FIXME: This probably should use PQcancel() if results would require network
* IO.
*/
void
ForgetResults(MultiConnection *connection)
{
while (true)
{
PGresult *result = NULL;
result = PQgetResult(connection->conn);
if (result == NULL)
{
break;
}
if (PQresultStatus(result) == PGRES_COPY_IN)
{
PQputCopyEnd(connection->conn, NULL);
/* FIXME: mark connection as failed? */
}
PQclear(result);
}
}
/*
* SqlStateMatchesCategory returns true if the given sql state (which may be
* NULL if unknown) is in the given error category. Note that we use
* ERRCODE_TO_CATEGORY macro to determine error category of the sql state and
* expect the caller to use the same macro for the error category.
*/
bool
SqlStateMatchesCategory(char *sqlStateString, int category)
{
bool sqlStateMatchesCategory = false;
int sqlState = 0;
int sqlStateCategory = 0;
if (sqlStateString == NULL)
{
return false;
}
sqlState = MAKE_SQLSTATE(sqlStateString[0], sqlStateString[1], sqlStateString[2],
sqlStateString[3], sqlStateString[4]);
sqlStateCategory = ERRCODE_TO_CATEGORY(sqlState);
if (sqlStateCategory == category)
{
sqlStateMatchesCategory = true;
}
return sqlStateMatchesCategory;
}
/* report errors & warnings */
/*
* Report libpq failure that's not associated with a result.
*/
void
ReportConnectionError(MultiConnection *connection, int elevel)
{
char *nodeName = connection->hostname;
int nodePort = connection->port;
ereport(elevel, (errmsg("connection error: %s:%d", nodeName, nodePort),
errdetail("%s", PQerrorMessage(connection->conn))));
}
/*
* Report libpq failure associated with a result.
*/
void
ReportResultError(MultiConnection *connection, PGresult *result, int elevel)
{
char *sqlStateString = PQresultErrorField(result, PG_DIAG_SQLSTATE);
char *messagePrimary = PQresultErrorField(result, PG_DIAG_MESSAGE_PRIMARY);
char *messageDetail = PQresultErrorField(result, PG_DIAG_MESSAGE_DETAIL);
char *messageHint = PQresultErrorField(result, PG_DIAG_MESSAGE_HINT);
char *messageContext = PQresultErrorField(result, PG_DIAG_CONTEXT);
char *nodeName = connection->hostname;
int nodePort = connection->port;
int sqlState = ERRCODE_INTERNAL_ERROR;
if (sqlStateString != NULL)
{
sqlState = MAKE_SQLSTATE(sqlStateString[0], sqlStateString[1], sqlStateString[2],
sqlStateString[3], sqlStateString[4]);
}
/*
* If the PGresult did not contain a message, the connection may provide a
* suitable top level one. At worst, this is an empty string.
*/
if (messagePrimary == NULL)
{
char *lastNewlineIndex = NULL;
messagePrimary = PQerrorMessage(connection->conn);
lastNewlineIndex = strrchr(messagePrimary, '\n');
/* trim trailing newline, if any */
if (lastNewlineIndex != NULL)
{
*lastNewlineIndex = '\0';
}
}
ereport(elevel, (errcode(sqlState), errmsg("%s", messagePrimary),
messageDetail ? errdetail("%s", messageDetail) : 0,
messageHint ? errhint("%s", messageHint) : 0,
messageContext ? errcontext("%s", messageContext) : 0,
errcontext("while executing command on %s:%d",
nodeName, nodePort)));
}
/*
* Log commands send to remote nodes if citus.log_remote_commands wants us to
* do so.
*/
void
LogRemoteCommand(MultiConnection *connection, const char *command)
{
if (!LogRemoteCommands)
{
return;
}
ereport(LOG, (errmsg("issuing %s", command),
errdetail("on server %s:%d", connection->hostname, connection->port)));
}
/* wrappers around libpq functions, with command logging support */
/*
* Tiny PQsendQuery wrapper that logs remote commands, and accepts a
* MultiConnection instead of a plain PGconn.
*/
int
SendRemoteCommand(MultiConnection *connection, const char *command)
{
LogRemoteCommand(connection, command);
return PQsendQuery(connection->conn, command);
}
/*
* Execute a statement over the connection. Basically equivalent to PQexec(),
* except for logging and error handling integration.
*
* NULL is returned upon errors, the query's results otherwise.
*/
PGresult *
ExecuteStatement(MultiConnection *connection, const char *statement)
{
return ExecuteStatementParams(connection, statement, 0, NULL, NULL);
}
/*
* Execute a statement over the connection. Basically equivalent to
* PQexecParams(), except for logging and error handling integration.
*
* NULL is returned upon errors, the query's results otherwise.
*/
PGresult *
ExecuteStatementParams(MultiConnection *connection, const char *statement,
int paramCount, const Oid *paramTypes,
const char *const *paramValues)
{
PGresult *result = NULL;
AdjustRemoteTransactionState(connection);
if (connection->remoteTransaction.transactionFailed)
{
return NULL;
}
LogRemoteCommand(connection, statement);
if (!PQsendQueryParams(connection->conn, statement, paramCount, paramTypes,
paramValues, NULL, NULL, 0))
{
ReportConnectionError(connection, WARNING);
MarkRemoteTransactionFailed(connection, true);
return NULL;
}
result = PQgetResult(connection->conn);
if (!IsResponseOK(result))
{
ReportResultError(connection, result, WARNING);
MarkRemoteTransactionFailed(connection, true);
PQclear(result);
result = PQgetResult(connection->conn);
Assert(result == NULL);
return NULL;
}
return result;
}
/*
* Execute a statement over the connection. Basically equivalent to PQexec(),
* except for logging and error handling integration.
*
* Returns true if the command succeeded, false otherwise.
*/
bool
ExecuteCheckStatement(MultiConnection *connection, const char *statement)
{
return ExecuteCheckStatementParams(connection, statement, 0, NULL, NULL);
}
/*
* Execute a statement over the connection. Basically equivalent to
* PQexecParams(), except for logging and error handling integration.
*
* Returns true if the command succeeded, false otherwise.
*/
bool
ExecuteCheckStatementParams(MultiConnection *connection, const char *statement,
int paramCount, const Oid *paramTypes,
const char *const *paramValues)
{
bool resultOk = false;
PGresult *result = ExecuteStatementParams(connection, statement, paramCount,
paramTypes, paramValues);
resultOk = result != NULL;
PQclear(result);
result = PQgetResult(connection->conn);
Assert(result == NULL);
return resultOk;
}
/* -------------------------------------------------------------------------
* Higher level command execution functions
* -------------------------------------------------------------------------
*/
/*
* Execute placement associated commands in parallel.
*
* TODO: Use less than one one connection per placement.
*/
void
ExecuteBatchCommands(List *batchCommandList)
{
List *connectionList = NIL;
int64 ncommands = list_length(batchCommandList);
BatchCommand **batchCommands = NULL;
int i = 0;
/* convert into usefully sorted array */
batchCommands = BatchCommandListToArray(batchCommandList);
/*
* Initiate connection establishment if necessary. All connections might
* be already existing and, possibly, fully established.
*/
for (i = 0; i < ncommands; i++)
{
BatchCommand *command = batchCommands[i];
ShardPlacement *placement = command->placement;
MultiConnection *connection = NULL;
/* asynchronously open connection to remote node */
connection =
StartPlacementConnection(command->connectionFlags,
placement);
/* couldn't work with that */
Assert(PQtransactionStatus(connection->conn) != PQTRANS_ACTIVE);
/* every command should get its own connection for now */
ClaimConnectionExclusively(connection);
command->connection = connection;
connectionList = lappend(connectionList, connection);
}
/* wait for connection establishment */
for (i = 0; i < ncommands; i++)
{
BatchCommand *command = batchCommands[i];
/*
* It'd better to wait for all connections at once. Especially when
* SSL (or complex authentication protocols), it's quite beneficial to
* do connection establishment fully in parallel using nonblocking
* IO. This way we'll currently do the initial connect() in parallel,
* but afterwards block in SSL connection establishment, which often
* takes the bulk of the time.
*/
FinishConnectionEstablishment(command->connection);
}
/* BEGIN transaction if necessary */
AdjustRemoteTransactionStates(connectionList);
/* Finally send commands to all connections in parallel */
for (i = 0; i < ncommands; i++)
{
BatchCommand *command = batchCommands[i];
MultiConnection *connection = command->connection;
if (connection->remoteTransaction.transactionFailed)
{
continue;
}
if (!SendRemoteCommand(connection, command->commandString))
{
ReportConnectionError(connection, WARNING);
MarkRemoteTransactionFailed(connection, true);
}
}
/*
* Wait for command results to come in.
*
* TODO: We should really wait asynchronously, using nonblocking IO, on
* all these connections. As long as they all only tranfer miniscule
* amounts of data, it doesn't matter much, but as soon that that's not
* the case...
*/
for (i = 0; i < ncommands; i++)
{
BatchCommand *command = batchCommands[i];
MultiConnection *connection = command->connection;
PGresult *result = NULL;
result = PQgetResult(connection->conn);
if (!IsResponseOK(result))
{
connection->remoteTransaction.transactionFailed = true;
command->failed = true;
ReportResultError(connection, result, WARNING);
MarkRemoteTransactionFailed(connection, true);
}
else
{
char *affectedTuples = PQcmdTuples(result);
if (strlen(affectedTuples) > 0)
{
scanint8(affectedTuples, false, &command->tuples);
}
command->failed = false;
}
/* XXX: allow for result processing? */
PQclear(result);
/* clear NULL result(s) */
ForgetResults(connection);
/* allow connection to be used again */
UnclaimConnection(connection);
}
}
/*
* Deparse and execute query on all finalized placements for the shards in
* shardIntervalList.
*
* Failed placements are marked as invalid, unless all placements for a shard
* fail.
*
* Returns the number of modified tuples.
*/
int64
ExecuteQueryOnPlacements(Query *query, List *shardIntervalList, Oid relationId)
{
List *commandList = NIL;
ListCell *intervalCell = NULL;
ListCell *commandCell = NULL;
int64 ntuples = 0;
int64 lastSuccessfulShardId = INVALID_SHARD_ID;
foreach(intervalCell, shardIntervalList)
{
ShardInterval *shardInterval = (ShardInterval *) lfirst(intervalCell);
List *shardPlacementList = FinalizedShardPlacementList(shardInterval->shardId);
ListCell *placementCell = NULL;
StringInfoData shardQueryString;
initStringInfo(&shardQueryString);
deparse_shard_query(query, relationId, shardInterval->shardId, &shardQueryString);
foreach(placementCell, shardPlacementList)
{
ShardPlacement *placement = (ShardPlacement *) lfirst(placementCell);
BatchCommand *command = (BatchCommand *) palloc0(sizeof(BatchCommand));
command->placement = placement;
command->connectionFlags = NEW_CONNECTION | CACHED_CONNECTION | FOR_DML;
command->commandString = shardQueryString.data;
commandList = lappend(commandList, command);
}
}
ExecuteBatchCommands(commandList);
InvalidateFailedPlacements(commandList);
foreach(commandCell, commandList)
{
BatchCommand *command = (BatchCommand *) lfirst(commandCell);
ShardPlacement *placement = command->placement;
if (!command->failed)
{
if (lastSuccessfulShardId != placement->shardId)
{
ntuples += command->tuples;
}
lastSuccessfulShardId = placement->shardId;
}
}
return ntuples;
}
/*
* Execute DDL on all finalized placements. All errors abort the command,
* i.e. shards are not marked as invalid (to avoid schema divergence).
*/
void
ExecuteDDLOnRelationPlacements(Oid relationId, const char *command)
{
/* FIXME: for correct locking we need to acquire metadata locks before */
List *shardIntervalList = LoadShardIntervalList(relationId);
Oid schemaId = get_rel_namespace(relationId);
char *schemaName = get_namespace_name(schemaId);
char *escapedSchemaName = quote_literal_cstr(schemaName);
char *escapedCommandString = quote_literal_cstr(command);
List *commandList = NIL;
StringInfo applyCommand = makeStringInfo();
ListCell *intervalCell = NULL;
BeginOrContinueCoordinatedTransaction();
LockShards(shardIntervalList, ShareLock);
foreach(intervalCell, shardIntervalList)
{
ShardInterval *shardInterval = (ShardInterval *) lfirst(intervalCell);
List *placementList = FinalizedShardPlacementList(shardInterval->shardId);
uint64 shardId = shardInterval->shardId;
ListCell *placementCell = NULL;
/* build the shard ddl command -- perhaps add parametrized variant instead? */
appendStringInfo(applyCommand, WORKER_APPLY_SHARD_DDL_COMMAND, shardId,
escapedSchemaName, escapedCommandString);
foreach(placementCell, placementList)
{
ShardPlacement *placement = (ShardPlacement *) lfirst(placementCell);
BatchCommand *command = (BatchCommand *) palloc0(sizeof(BatchCommand));
command->placement = placement;
command->connectionFlags = NEW_CONNECTION | CACHED_CONNECTION |
FOR_DDL | CRITICAL_CONNECTION;
command->commandString = pstrdup(applyCommand->data);
commandList = lappend(commandList, command);
}
resetStringInfo(applyCommand);
}
ExecuteBatchCommands(commandList);
}
/*
* Mark placements that failed in ExecuteBatchCommands as invalid, unless all
* placements in a shard failed.
*/
void
InvalidateFailedPlacements(List *batchCommandList)
{
BatchCommand **batchCommands = NULL;
int i = 0;
int64 lastShardId = INVALID_SHARD_ID;
List *failedPlacements = NIL;
List *goodPlacements = NIL;
int64 ncommands = list_length(batchCommandList);
/* convert into usefully sorted array */
batchCommands = BatchCommandListToArray(batchCommandList);
for (i = 0; i < ncommands; i++)
{
BatchCommand *command = batchCommands[i];
ShardPlacement *placement = command->placement;
/*
* If we're looking at the next shard, check whether some or all of
* the placements failed, and need to be marked as invalid.
*/
if (lastShardId != INVALID_SHARD_ID && lastShardId != placement->shardId)
{
HandlePlacementFailures(goodPlacements, failedPlacements);
failedPlacements = NIL;
goodPlacements = NIL;
}
if (command->failed)
{
failedPlacements = lappend(failedPlacements, placement);
}
else
{
goodPlacements = lappend(goodPlacements, placement);
}
}
HandlePlacementFailures(goodPlacements, failedPlacements);
}
/*
* Convert list of BatchCommands to a sorted array of BatchCommand*s.
*/
static BatchCommand **
BatchCommandListToArray(List *batchCommandList)
{
int64 ncommands = list_length(batchCommandList);
ListCell *commandCell = NULL;
BatchCommand **commands = NULL;
int off = 0;
commands = (BatchCommand **) palloc(sizeof(BatchCommand *) * ncommands);
foreach(commandCell, batchCommandList)
{
commands[off++] = (BatchCommand *) lfirst(commandCell);
}
qsort(commands, ncommands, sizeof(BatchCommand *),
CompareBatchCommands);
return commands;
}
/*
* Sorting helper for BatchCommand's. Sorts in a way that guarantees that all
* placements for a shard are consecutive.
*/
static int
CompareBatchCommands(const void *leftElement, const void *rightElement)
{
const BatchCommand *leftCommand = *((const BatchCommand **) leftElement);
const BatchCommand *rightCommand = *((const BatchCommand **) rightElement);
const ShardPlacement *leftPlacement = leftCommand->placement;
const ShardPlacement *rightPlacement = rightCommand->placement;
int compare = 0;
if (leftPlacement->shardId < rightPlacement->shardId)
{
return -1;
}
if (leftPlacement->shardId > rightPlacement->shardId)
{
return 1;
}
compare = strcmp(leftPlacement->nodeName, rightPlacement->nodeName);
if (compare != 0)
{
return compare;
}
if (leftPlacement->nodePort < rightPlacement->nodePort)
{
return -1;
}
if (leftPlacement->nodePort > rightPlacement->nodePort)
{
return 1;
}
if (leftPlacement->placementId < rightPlacement->placementId)
{
return -1;
}
if (leftPlacement->placementId > rightPlacement->placementId)
{
return 1;
}
/* other elements irrelevant for our purpose */
return 0;
}
/*
* Helper for InvalidateFailedPlacements.
*/
static void
HandlePlacementFailures(List *goodPlacements, List *failedPlacements)
{
if (list_length(failedPlacements) > 0 &&
list_length(goodPlacements) == 0)
{
elog(ERROR, "all placements failed");
}
else if (list_length(failedPlacements) > 0)
{
ListCell *placementCell = NULL;
elog(LOG, "some placements failed, marking as invalid");
foreach(placementCell, failedPlacements)
{
ShardPlacement *placement = (ShardPlacement *) lfirst(placementCell);
UpdateShardPlacementState(placement->placementId, FILE_INACTIVE);
}
}
}

View File

@ -1,61 +0,0 @@
/*-------------------------------------------------------------------------
*
* commit_protocol.h
* Type and function declarations used in performing transactions across
* shard placements.
*
* Copyright (c) 2016, Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#ifndef COMMIT_PROTOCOL_H
#define COMMIT_PROTOCOL_H
#include "access/xact.h"
#include "libpq-fe.h"
#include "lib/stringinfo.h"
#include "nodes/pg_list.h"
/* Enumeration that defines the different commit protocols available */
typedef enum
{
COMMIT_PROTOCOL_1PC = 0,
COMMIT_PROTOCOL_2PC = 1
} CommitProtocolType;
/* Enumeration that defines different remote transaction states */
typedef enum
{
TRANSACTION_STATE_INVALID = 0,
TRANSACTION_STATE_OPEN,
TRANSACTION_STATE_COPY_STARTED,
TRANSACTION_STATE_PREPARED,
TRANSACTION_STATE_CLOSED
} TransactionState;
/*
* TransactionConnection represents a connection to a remote node which is
* used to perform a transaction on shard placements.
*/
typedef struct TransactionConnection
{
int64 connectionId;
TransactionState transactionState;
PGconn *connection;
} TransactionConnection;
/* config variable managed via guc.c */
extern int MultiShardCommitProtocol;
/* Functions declarations for transaction and connection management */
extern void InitializeDistributedTransaction(void);
extern void PrepareRemoteTransactions(List *connectionList);
extern void AbortRemoteTransactions(List *connectionList);
extern void CommitRemoteTransactions(List *connectionList, bool stopOnFailure);
#endif /* COMMIT_PROTOCOL_H */

View File

@ -52,26 +52,11 @@ typedef struct NodeConnectionEntry
} NodeConnectionEntry;
/* describes what kind of modifications have occurred in the current transaction */
typedef enum
{
XACT_MODIFICATION_INVALID = 0, /* placeholder initial value */
XACT_MODIFICATION_NONE, /* no modifications have taken place */
XACT_MODIFICATION_DATA, /* data modifications (DML) have occurred */
XACT_MODIFICATION_SCHEMA /* schema modifications (DDL) have occurred */
} XactModificationType;
/* state needed to prevent new connections during modifying transactions */
extern XactModificationType XactModificationLevel;
/* function declarations for obtaining and using a connection */
extern PGconn * GetOrEstablishConnection(char *nodeName, int32 nodePort);
extern void PurgeConnection(PGconn *connection);
extern void BuildKeyForConnection(PGconn *connection, NodeConnectionKey *connectionKey);
extern PGconn * PurgeConnectionByKey(NodeConnectionKey *nodeConnectionKey);
extern bool SqlStateMatchesCategory(char *sqlStateString, int category);
extern void WarnRemoteError(PGconn *connection, PGresult *result);
extern void ReraiseRemoteError(PGconn *connection, PGresult *result);
extern PGconn * ConnectToNode(char *nodeName, int nodePort, char *nodeUser);

View File

@ -0,0 +1,137 @@
/*-------------------------------------------------------------------------
*
* connection_management.h
* Central management of connections and their life-cycle
*
* Copyright (c) 2016, Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#ifndef CONNECTION_MANAGMENT_H
#define CONNECTION_MANAGMENT_H
#include "distributed/transaction_management.h"
#include "distributed/remote_transaction.h"
#include "nodes/pg_list.h"
#include "utils/hsearch.h"
/* maximum (textual) lengths of hostname and port */
#define MAX_NODE_LENGTH 255 /* includes 0 byte */
#define CLIENT_CONNECT_TIMEOUT_SECONDS_INT 5
/* forward declare, to avoid forcing large headers on everyone */
struct pg_conn; /* target of the PGconn typedef */
struct MemoryContextData;
/*
* Flags determining connection establishment behaviour.
*/
enum MultiConnectionMode
{
/* allow establishment of new connections */
NEW_CONNECTION = 1 << 0,
/* allow use of pre-established connections */
CACHED_CONNECTION = 1 << 1,
/* mark returned connection having session lifespan */
SESSION_LIFESPAN = 1 << 2,
/* the connection will be used for DML */
FOR_DML = 1 << 3,
/* the connection will be used for DDL */
FOR_DDL = 1 << 4,
/* failures on this connection will fail entire coordinated transaction */
CRITICAL_CONNECTION = 1 << 5
};
/* declaring this directly above makes uncrustify go crazy */
typedef enum MultiConnectionMode MultiConnectionMode;
typedef struct MultiConnection
{
/* connection details, useful for error messages and such. */
char hostname[MAX_NODE_LENGTH];
int32 port;
char user[NAMEDATALEN];
char database[NAMEDATALEN];
/* underlying libpq connection */
struct pg_conn *conn;
/* is the connection intended to be kept after transaction end */
bool sessionLifespan;
/* is the connection currently in use, and shouldn't be used by anything else */
bool claimedExclusively;
/* has the connection been used in the current coordinated transaction? */
bool activeInTransaction;
/* information about the associated remote transaction */
RemoteTransaction remoteTransaction;
} MultiConnection;
/*
* Central connection management hash, mapping (host, port, user, database) to
* a list of connections.
*
* This hash is used to keep track of which connections are open to which
* node. Besides allowing connection reuse, that information is e.g. used to
* handle closing connections after the end of a transaction.
*/
/* hash key */
typedef struct ConnectionHashKey
{
char hostname[MAX_NODE_LENGTH];
int32 port;
char user[NAMEDATALEN];
char database[NAMEDATALEN];
} ConnectionHashKey;
/* hash entry */
typedef struct ConnectionHashEntry
{
ConnectionHashKey key;
List *connections;
} ConnectionHashEntry;
/* the hash table */
extern HTAB *ConnectionHash;
/* context for all connection and transaction related memory */
extern struct MemoryContextData *ConnectionContext;
extern void AtEOXact_Connections(bool isCommit);
extern void InitializeConnectionManagement(void);
/* Low-level connection establishment APIs */
extern MultiConnection * GetNodeConnection(uint32 flags, const char *hostname,
int32 port);
extern MultiConnection * StartNodeConnection(uint32 flags, const char *hostname,
int32 port);
extern MultiConnection * GetNodeUserDatabaseConnection(uint32 flags, const char *hostname,
int32 port, const char *user, const
char *database);
extern MultiConnection * StartNodeUserDatabaseConnection(uint32 flags,
const char *hostname,
int32 port,
const char *user,
const char *database);
/* dealing with a connection */
extern void FinishConnectionEstablishment(MultiConnection *connection);
extern void ClaimConnectionExclusively(MultiConnection *connection);
extern void UnclaimConnection(MultiConnection *connection);
#endif /* CONNECTION_MANAGMENT_H */

View File

@ -0,0 +1,30 @@
/*-------------------------------------------------------------------------
* hash_helpers.h
*
* Copyright (c) 2016, Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#ifndef HASH_HELPERS_H
#define HASH_HELPERS_H
#include "utils/hsearch.h"
/*
* Combine two hash values, resulting in another hash value, with decent bit
* mixing.
*
* Similar to boost's hash_combine().
*/
static inline uint32
hash_combine(uint32 a, uint32 b)
{
a ^= b + 0x9e3779b9 + (a << 6) + (a >> 2);
return a;
}
extern void hash_delete_all(HTAB *htab);
#endif

View File

@ -66,7 +66,7 @@
#define CREATE_SCHEMA_COMMAND "CREATE SCHEMA IF NOT EXISTS %s"
#define CREATE_EMPTY_SHARD_QUERY "SELECT master_create_empty_shard('%s')"
#define FINALIZED_SHARD_PLACEMENTS_QUERY \
"SELECT nodename, nodeport FROM pg_dist_shard_placement WHERE shardstate = 1 AND shardid = %ld"
"SELECT placementid, nodename, nodeport FROM pg_dist_shard_placement WHERE shardstate = 1 AND shardid = %ld"
#define UPDATE_SHARD_STATISTICS_QUERY \
"SELECT master_update_shard_statistics(%ld)"
#define PARTITION_METHOD_QUERY "SELECT part_method FROM master_get_table_metadata('%s');"

View File

@ -17,18 +17,6 @@
#include "nodes/pg_list.h"
/*
* XactShardConnSet keeps track of the mapping from shard to the set of nodes
* involved in multi-statement transaction-wrapped modifications of that shard.
* This information is used to mark placements inactive at transaction close.
*/
typedef struct XactShardConnSet
{
uint64 shardId; /* identifier of the shard that was modified */
List *connectionEntryList; /* NodeConnectionEntry pointers to participating nodes */
} XactShardConnSet;
/* Config variables managed via guc.c */
extern bool AllModificationsCommutative;
@ -37,6 +25,5 @@ extern void RouterExecutorStart(QueryDesc *queryDesc, int eflags, Task *task);
extern void RouterExecutorRun(QueryDesc *queryDesc, ScanDirection direction, long count);
extern void RouterExecutorFinish(QueryDesc *queryDesc);
extern void RouterExecutorEnd(QueryDesc *queryDesc);
extern void RegisterRouterExecutorXactCallbacks(void);
#endif /* MULTI_ROUTER_EXECUTOR_H_ */

View File

@ -1,39 +0,0 @@
/*-------------------------------------------------------------------------
*
* multi_shard_transaction.h
* Type and function declarations used in performing transactions across
* shard placements.
*
* Copyright (c) 2016, Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#ifndef MULTI_SHARD_TRANSACTION_H
#define MULTI_SHARD_TRANSACTION_H
#include "utils/hsearch.h"
#include "nodes/pg_list.h"
/* ShardConnections represents a set of connections for each placement of a shard */
typedef struct ShardConnections
{
int64 shardId;
List *connectionList;
} ShardConnections;
extern void OpenTransactionsToAllShardPlacements(List *shardIdList, char *relationOwner);
extern HTAB * CreateShardConnectionHash(MemoryContext memoryContext);
extern void BeginTransactionOnShardPlacements(uint64 shardId, char *nodeUser);
extern ShardConnections * GetShardConnections(int64 shardId, bool *shardConnectionsFound);
extern ShardConnections * GetShardHashConnections(HTAB *connectionHash, int64 shardId,
bool *connectionsFound);
extern List * ConnectionList(HTAB *connectionHash);
extern void CloseConnections(List *connectionList);
extern void RegisterShardPlacementXactCallbacks(void);
#endif /* MULTI_SHARD_TRANSACTION_H */

View File

@ -0,0 +1,97 @@
/*-------------------------------------------------------------------------
* placement_connection.h
*
* Copyright (c) 2016, Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#ifndef PLACEMENT_CONNECTION_H
#define PLACEMENT_CONNECTION_H
#include "distributed/connection_management.h"
#include "utils/hsearch.h"
/* forward declare, to avoid dependency on ShardPlacement definition */
struct ShardPlacement;
/*
* Hash table mapping placements to a list of connections.
*
* This stores a list of connections for each placement, because multiple
* connections to the same placement may exist at the same time. E.g. a
* real-time executor query may reference the same placement in several
* sub-tasks.
*
* We keep track about a connection having executed DML or DDL, since we can
* only ever allow a single transaction to do either to prevent deadlocks and
* consistency violations (e.g. read-your-own-writes).
*/
/* hash key */
typedef struct ConnectionPlacementHashKey
{
uint32 placementid;
} ConnectionPlacementHashKey;
/* information about a connection reference to a table */
typedef struct ConnectionReference
{
MultiConnection *connection;
bool hadDML;
bool hadDDL;
} ConnectionReference;
/* hash entry */
typedef struct ConnectionPlacementHashEntry
{
ConnectionPlacementHashKey key;
bool failed;
List *connectionReferences;
} ConnectionPlacementHashEntry;
/* hash table */
extern HTAB *ConnectionPlacementHash;
/*
* Hash table mapping shard ids to placements.
*
* This is used to track whether placements of a shard have to be marked
* invalid after a failure, or whether a coordinated transaction has to be
* aborted, to avoid all placements of a shard to be marked invalid.
*/
/* hash key */
typedef struct ConnectionShardHashKey
{
uint64 shardId;
} ConnectionShardHashKey;
/* hash entry */
typedef struct ConnectionShardHashEntry
{
ConnectionShardHashKey *key;
List *placementConnections;
} ConnectionShardHashEntry;
/* hash table itself */
extern HTAB *ConnectionShardHash;
/* Higher level connection handling API. */
extern MultiConnection * GetPlacementConnection(uint32 flags,
struct ShardPlacement *placement);
extern MultiConnection * StartPlacementConnection(uint32 flags,
struct ShardPlacement *placement);
extern void CheckForFailedPlacements(bool preCommit, bool using2PC);
extern void InitPlacementConnectionManagement(void);
extern void ResetPlacementConnectionManagement(void);
#endif /* PLACEMENT_CONNECTION_H */

View File

@ -0,0 +1,79 @@
/*-------------------------------------------------------------------------
*
* remote_commands.h
* Helpers to execute commands on remote nodes, over libpq.
*
* Copyright (c) 2016, Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#ifndef REMOTE_COMMAND_H
#define REMOTE_COMMAND_H
#include "distributed/connection_management.h"
#include "distributed/placement_connection.h"
struct pg_result; /* target of the PGresult typedef */
struct Query;
typedef struct BatchCommand
{
/* user input fields */
struct ShardPlacement *placement;
uint32 connectionFlags;
void *userData;
const char *commandString;
/* user output fields */
bool failed;
int64 tuples;
/* internal fields */
MultiConnection *connection;
} BatchCommand;
/* GUC, determining whether statements sent to remote nodes are logged */
extern bool LogRemoteCommands;
/* simple helpers */
extern bool IsResponseOK(struct pg_result *result);
extern void ForgetResults(MultiConnection *connection);
extern bool SqlStateMatchesCategory(char *sqlStateString, int category);
/* report errors & warnings */
extern void ReportConnectionError(MultiConnection *connection, int elevel);
extern void ReportResultError(MultiConnection *connection, struct pg_result *result,
int elevel);
extern void LogRemoteCommand(MultiConnection *connection, const char *command);
/* wrappers around libpq functions, with command logging support */
extern int SendRemoteCommand(MultiConnection *connection, const char *command);
/* libpq helpers */
extern struct pg_result * ExecuteStatement(MultiConnection *connection, const
char *statement);
extern struct pg_result * ExecuteStatementParams(MultiConnection *connection,
const char *statement,
int paramCount, const Oid *paramTypes,
const char *const *paramValues);
extern bool ExecuteCheckStatement(MultiConnection *connection, const char *statement);
extern bool ExecuteCheckStatementParams(MultiConnection *connection,
const char *statement,
int paramCount, const Oid *paramTypes,
const char *const *paramValues);
/* higher level command execution helpers */
extern void ExecuteBatchCommands(List *batchCommandList);
extern int64 ExecuteQueryOnPlacements(struct Query *query, List *shardPlacementList,
Oid relationId);
extern void ExecuteDDLOnRelationPlacements(Oid relationId, const char *command);
extern void InvalidateFailedPlacements(List *batchCommandList);
#endif /* REMOTE_COMMAND_H */

View File

@ -0,0 +1,78 @@
/*-------------------------------------------------------------------------
* remote_transaction.h
*
* Copyright (c) 2016, Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#ifndef REMOTE_TRANSACTION_H
#define REMOTE_TRANSACTION_H
#include "nodes/pg_list.h"
/* forward declare, to avoid recursive includes */
struct MultiConnection;
/*
* Enum that defines different remote transaction states, of a single remote
* transaction.
*/
typedef enum
{
/* no transaction active */
REMOTE_TRANS_INVALID = 0,
/* transaction start */
REMOTE_TRANS_STARTING,
REMOTE_TRANS_STARTED,
/* 2pc prepare */
REMOTE_TRANS_PREPARING,
REMOTE_TRANS_PREPARED,
/* transaction abort */
REMOTE_TRANS_1PC_ABORTING,
REMOTE_TRANS_2PC_ABORTING,
REMOTE_TRANS_ABORTED,
/* transaction commit */
REMOTE_TRANS_1PC_COMMITTING,
REMOTE_TRANS_2PC_COMMITTING,
REMOTE_TRANS_COMMITTED
} RemoteTransactionState;
/*
* Transaction state associated associated with a single MultiConnection.
*/
typedef struct RemoteTransaction
{
/* what state is the remote side transaction in */
RemoteTransactionState transactionState;
/* failures on this connection should abort entire coordinated transaction */
bool criticalTransaction;
/* failed in current transaction */
bool transactionFailed;
/* 2PC transaction name currently associated with connection */
char preparedName[NAMEDATALEN];
} RemoteTransaction;
/* change remote transaction state */
extern void AdjustRemoteTransactionState(struct MultiConnection *connection);
extern void AdjustRemoteTransactionStates(List *connectionList);
extern void MarkRemoteTransactionFailed(struct MultiConnection *connection,
bool allowErrorPromotion);
extern void CoordinatedRemoteTransactionsCommit(void);
extern void CoordinatedRemoteTransactionsAbort(void);
extern void CoordinatedRemoteTransactionsPrepare(void);
#endif /* REMOTE_TRANSACTION_H */

View File

@ -0,0 +1,75 @@
/*-------------------------------------------------------------------------
* transaction_management.h
*
* Copyright (c) 2016, Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#ifndef TRANSACTION_MANAGMENT_H
#define TRANSACTION_MANAGMENT_H
/* describes what kind of modifications have occurred in the current transaction */
typedef enum
{
XACT_MODIFICATION_INVALID = 0, /* placeholder initial value */
XACT_MODIFICATION_NONE, /* no modifications have taken place */
XACT_MODIFICATION_DATA, /* data modifications (DML) have occurred */
XACT_MODIFICATION_SCHEMA /* schema modifications (DDL) have occurred */
} XactModificationType;
/*
* Enum defining the state of a coordinated (i.e. a transaction potentially
* spanning several nodes).
*/
typedef enum CoordinatedTransactionState
{
/* no coordinated transaction in progress, no connections established */
COORD_TRANS_NONE,
/* no coordinated transaction in progress, but connections established */
COORD_TRANS_IDLE,
/* coordinated transaction in progress */
COORD_TRANS_STARTED,
/* coordinated transaction prepared on all workers */
COORD_TRANS_PREPARED,
/* coordinated transaction committed */
COORD_TRANS_COMMITTED
} CoordinatedTransactionState;
/* Enumeration that defines the different commit protocols available */
typedef enum
{
COMMIT_PROTOCOL_1PC = 0,
COMMIT_PROTOCOL_2PC = 1
} CommitProtocolType;
/* config variable managed via guc.c */
extern int MultiShardCommitProtocol;
/* state needed to prevent new connections during modifying transactions */
extern XactModificationType XactModificationLevel;
extern CoordinatedTransactionState CurrentCoordinatedTransactionState;
/*
* Coordinated transaction management.
*/
extern void BeginCoordinatedTransaction(void);
extern void BeginOrContinueCoordinatedTransaction(void);
extern bool InCoordinatedTransaction(void);
/*
* Initialization.
*/
extern void InitializeTransactionManagement(void);
#endif /* TRANSACTION_MANAGMENT_H */

View File

@ -155,12 +155,12 @@ NOTICE: using one-phase commit for distributed DDL commands
HINT: You can enable two-phase commit for extra safety with: SET citus.multi_shard_commit_protocol TO '2pc'
WARNING: data type bigint has no default operator class for access method "gist"
HINT: You must specify an operator class for the index or define a default operator class for the data type.
CONTEXT: while executing command on localhost:57638
ERROR: could not execute DDL command on worker node shards
CONTEXT: while executing command on localhost:57637
ERROR: failure on connection marked as essential: localhost:57637
CREATE INDEX try_index ON lineitem (non_existent_column);
WARNING: column "non_existent_column" does not exist
CONTEXT: while executing command on localhost:57638
ERROR: could not execute DDL command on worker node shards
CONTEXT: while executing command on localhost:57637
ERROR: failure on connection marked as essential: localhost:57637
CREATE INDEX ON lineitem (l_orderkey);
ERROR: creating index without a name on a distributed table is currently unsupported
-- Verify that none of failed indexes got created on the master node

View File

@ -44,11 +44,7 @@ SELECT master_create_worker_shards('lineitem_hash', 2, 1);
CREATE INDEX lineitem_hash_time_index ON lineitem_hash (l_shipdate);
NOTICE: using one-phase commit for distributed DDL commands
HINT: You can enable two-phase commit for extra safety with: SET citus.multi_shard_commit_protocol TO '2pc'
DEBUG: applied command on shard 650000 on node localhost:57637
DEBUG: applied command on shard 650001 on node localhost:57638
DEBUG: building index "lineitem_hash_time_index" on table "lineitem_hash"
DEBUG: sent COMMIT over connection 650000
DEBUG: sent COMMIT over connection 650001
CREATE TABLE orders_hash (
o_orderkey bigint not null,
o_custkey integer not null,

View File

@ -135,8 +135,7 @@ SELECT * FROM researchers, labs WHERE labs.id = researchers.lab_id;
BEGIN;
INSERT INTO labs VALUES (6, 'Bell Labs');
INSERT INTO researchers VALUES (9, 6, 'Leslie Lamport');
ERROR: no transaction participant matches localhost:57638
DETAIL: Transactions which modify distributed tables may only target nodes affected by the modification command which began the transaction.
ERROR: cannot open new connections after the first modification command within a transaction
COMMIT;
-- this logic even applies to router SELECTs occurring after a modification:
-- selecting from the modified node is fine...
@ -159,58 +158,29 @@ AND sp.nodeport = :worker_1_port
AND s.logicalrelid = 'researchers'::regclass;
INSERT INTO labs VALUES (6, 'Bell Labs');
SELECT count(*) FROM researchers WHERE lab_id = 6;
ERROR: no transaction participant matches localhost:57638
DETAIL: Transactions which modify distributed tables may only target nodes affected by the modification command which began the transaction.
ERROR: cannot open new connections after the first modification command within a transaction
ABORT;
-- applies to DDL, too
BEGIN;
INSERT INTO labs VALUES (6, 'Bell Labs');
ALTER TABLE labs ADD COLUMN motto text;
ERROR: distributed DDL commands must not appear within transaction blocks containing data modifications
COMMIT;
-- whether it occurs first or second
BEGIN;
ALTER TABLE labs ADD COLUMN motto text;
INSERT INTO labs VALUES (6, 'Bell Labs');
ERROR: distributed data modifications must not appear in transaction blocks which contain distributed DDL commands
COMMIT;
-- but the DDL should correctly roll back
\d labs
Table "public.labs"
Column | Type | Modifiers
--------+--------+-----------
id | bigint | not null
name | text | not null
SELECT * FROM labs WHERE id = 6;
id | name
----+------
(0 rows)
-- COPY can't happen second,
-- Check COPY can happen after INSERT
BEGIN;
INSERT INTO labs VALUES (6, 'Bell Labs');
\copy labs from stdin delimiter ','
ERROR: distributed copy operations must not appear in transaction blocks containing other distributed modifications
CONTEXT: COPY labs, line 1: "10,Weyland-Yutani"
COMMIT;
-- though it will work if before any modifications
-- Check COPY can happen before INSERT
BEGIN;
\copy labs from stdin delimiter ','
SELECT name FROM labs WHERE id = 10;
name
----------------
Weyland-Yutani
(1 row)
Weyland-Yutani
(2 rows)
INSERT INTO labs VALUES (6, 'Bell Labs');
COMMIT;
-- but a double-copy isn't allowed (the first will persist)
-- Two COPYs are also ok
BEGIN;
\copy labs from stdin delimiter ','
\copy labs from stdin delimiter ','
ERROR: distributed copy operations must not appear in transaction blocks containing other distributed modifications
CONTEXT: COPY labs, line 1: "12,fsociety"
COMMIT;
SELECT name FROM labs WHERE id = 11;
name
@ -218,13 +188,12 @@ SELECT name FROM labs WHERE id = 11;
Planet Express
(1 row)
-- finally, ALTER and copy aren't compatible
-- finally, check ALTER and copy are compatible
BEGIN;
ALTER TABLE labs ADD COLUMN motto text;
ALTER TABLE labs ADD COLUMN motto2 text;
\copy labs from stdin delimiter ','
ERROR: distributed copy operations must not appear in transaction blocks containing other distributed modifications
CONTEXT: COPY labs, line 1: "12,fsociety,lol"
COMMIT;
ALTER TABLE labs DROP COLUMN motto2;
-- but the DDL should correctly roll back
\d labs
Table "public.labs"
@ -233,30 +202,33 @@ COMMIT;
id | bigint | not null
name | text | not null
SELECT * FROM labs WHERE id = 12;
id | name
----+------
(0 rows)
-- and if the copy is before the ALTER...
BEGIN;
\copy labs from stdin delimiter ','
ALTER TABLE labs ADD COLUMN motto text;
ERROR: distributed DDL commands must not appear within transaction blocks containing data modifications
COMMIT;
-- the DDL fails, but copy persists
\d labs
Table "public.labs"
Column | Type | Modifiers
--------+--------+-----------
id | bigint | not null
name | text | not null
SELECT * FROM labs WHERE id = 12;
id | name
----+----------
12 | fsociety
(1 row)
12 | fsociety
(2 rows)
-- and if the copy is before the ALTER...
BEGIN;
\copy labs from stdin delimiter ','
ALTER TABLE labs ADD COLUMN motto3 text;
ERROR: distributed DDL commands must not appear within transaction blocks containing data modifications
COMMIT;
-- the DDL fails, and copy does not persist
\d labs
Table "public.labs"
Column | Type | Modifiers
--------+--------+-----------
id | bigint | not null
name | text | not null
SELECT * FROM labs WHERE id = 12;
id | name
----+----------
12 | fsociety
12 | fsociety
(2 rows)
-- now, for some special failures...
CREATE TABLE objects (
@ -512,6 +484,7 @@ INSERT INTO labs VALUES (9, 'BAD');
COMMIT;
WARNING: illegal value
CONTEXT: while executing command on localhost:57637
ERROR: could not commit transaction on any active nodes
-- data to objects should be persisted, but labs should not...
SELECT * FROM objects WHERE id = 1;
id | name
@ -536,9 +509,8 @@ ORDER BY s.logicalrelid, sp.shardstate;
logicalrelid | shardstate | count
--------------+------------+-------
labs | 1 | 1
objects | 1 | 1
objects | 3 | 1
(3 rows)
objects | 1 | 2
(2 rows)
-- some append-partitioned tests for good measure
CREATE TABLE append_researchers ( LIKE researchers );

View File

@ -103,10 +103,6 @@ SELECT master_modify_multiple_shards('DELETE FROM multi_shard_modify_test WHERE
DEBUG: predicate pruning for shardId 350001
DEBUG: predicate pruning for shardId 350002
DEBUG: predicate pruning for shardId 350003
DEBUG: sent PREPARE TRANSACTION over connection 350000
DEBUG: sent PREPARE TRANSACTION over connection 350000
DEBUG: sent COMMIT PREPARED over connection 350000
DEBUG: sent COMMIT PREPARED over connection 350000
master_modify_multiple_shards
-------------------------------
1

View File

@ -22,15 +22,22 @@ ERROR: cannot execute ALTER TABLE command involving partition column
-- verify that the distribution column can't be dropped
ALTER TABLE testtableddl DROP COLUMN distributecol;
ERROR: cannot execute ALTER TABLE command involving partition column
-- verify that the table cannot be dropped in a transaction block
\set VERBOSITY terse
-- verify that the table can be dropped in a transaction block
BEGIN;
SELECT 1;
?column?
----------
1
(1 row)
DROP TABLE testtableddl;
ERROR: DROP distributed table cannot run inside a transaction block
ROLLBACK;
\set VERBOSITY default
-- verify that the table can be dropped
DROP TABLE testtableddl;
SELECT 1;
?column?
----------
1
(1 row)
COMMIT;
-- verify that the table can dropped even if shards exist
CREATE TABLE testtableddl(somecol int, distributecol text NOT NULL);
SELECT master_create_distributed_table('testtableddl', 'distributecol', 'append');

View File

@ -68,11 +68,8 @@ SELECT shardid FROM pg_dist_shard where logicalrelid = 'test_truncate_append'::r
---------
(0 rows)
-- command can not be run inside transaction
-- verify command can be run inside transaction
BEGIN; TRUNCATE TABLE test_truncate_append; COMMIT;
ERROR: DROP distributed table cannot run inside a transaction block
CONTEXT: SQL statement "SELECT master_drop_all_shards(TG_RELID, TG_TABLE_SCHEMA, TG_TABLE_NAME)"
PL/pgSQL function citus_truncate_trigger() line 13 at PERFORM
DROP TABLE test_truncate_append;
--
-- truncate for range distribution

View File

@ -230,7 +230,9 @@ COMMIT;
-- Nothing from the block should have committed
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter';
SET client_min_messages = 'ERROR';
ALTER TABLE lineitem_alter DROP COLUMN first;
RESET client_min_messages;
-- Create single-shard table (to avoid deadlocks in the upcoming test hackery)
CREATE TABLE single_shard_items (id integer, name text);
SELECT master_create_distributed_table('single_shard_items', 'id', 'hash');
@ -264,14 +266,13 @@ COMMIT;
-- Nothing from the block should have committed
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'single_shard_items';
-- Now try with 2pc off
-- Now try with 2pc off, that'll result in invalid shards
RESET citus.multi_shard_commit_protocol;
BEGIN;
CREATE INDEX single_index_2 ON single_shard_items(id);
CREATE INDEX single_index_3 ON single_shard_items(name);
COMMIT;
-- The block should have committed with a warning
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'single_shard_items';
\c - - - :worker_2_port

View File

@ -262,8 +262,8 @@ NOTICE: relation "non_existent_table" does not exist, skipping
ALTER TABLE IF EXISTS lineitem_alter ALTER COLUMN int_column2 SET DATA TYPE INTEGER;
ALTER TABLE lineitem_alter DROP COLUMN non_existent_column;
WARNING: column "non_existent_column" of relation "lineitem_alter_220000" does not exist
CONTEXT: while executing command on localhost:57638
ERROR: could not execute DDL command on worker node shards
CONTEXT: while executing command on localhost:57637
ERROR: failure on connection marked as essential: localhost:57637
ALTER TABLE lineitem_alter DROP COLUMN IF EXISTS non_existent_column;
NOTICE: column "non_existent_column" of relation "lineitem_alter" does not exist, skipping
ALTER TABLE lineitem_alter DROP COLUMN IF EXISTS int_column2;
@ -362,16 +362,16 @@ DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT and TYPE subc
-- types
ALTER TABLE lineitem_alter ADD COLUMN new_column non_existent_type;
WARNING: type "non_existent_type" does not exist
CONTEXT: while executing command on localhost:57638
ERROR: could not execute DDL command on worker node shards
CONTEXT: while executing command on localhost:57637
ERROR: failure on connection marked as essential: localhost:57637
ALTER TABLE lineitem_alter ALTER COLUMN null_column SET NOT NULL;
WARNING: column "null_column" contains null values
CONTEXT: while executing command on localhost:57638
ERROR: could not execute DDL command on worker node shards
CONTEXT: while executing command on localhost:57637
ERROR: failure on connection marked as essential: localhost:57637
ALTER TABLE lineitem_alter ALTER COLUMN l_partkey SET DEFAULT 'a';
WARNING: invalid input syntax for integer: "a"
CONTEXT: while executing command on localhost:57638
ERROR: could not execute DDL command on worker node shards
CONTEXT: while executing command on localhost:57637
ERROR: failure on connection marked as essential: localhost:57637
-- Verify that we error out on statements involving RENAME
ALTER TABLE lineitem_alter RENAME TO lineitem_renamed;
ERROR: renaming distributed tables or their objects is currently unsupported
@ -520,7 +520,7 @@ HINT: You can enable two-phase commit for extra safety with: SET citus.multi_sh
ALTER TABLE lineitem_alter ADD COLUMN first integer;
WARNING: column "first" of relation "lineitem_alter_220000" already exists
CONTEXT: while executing command on localhost:57638
ERROR: could not execute DDL command on worker node shards
ERROR: failure on connection marked as essential: localhost:57638
COMMIT;
-- Nothing from the block should have committed
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter';
@ -528,6 +528,10 @@ SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'lineitem_alter';
-----------+-----------
(0 rows)
SET client_min_messages = 'ERROR';
ALTER TABLE lineitem_alter DROP COLUMN first;
ERROR: failure on connection marked as essential: localhost:57637
RESET client_min_messages;
-- Create single-shard table (to avoid deadlocks in the upcoming test hackery)
CREATE TABLE single_shard_items (id integer, name text);
SELECT master_create_distributed_table('single_shard_items', 'id', 'hash');
@ -565,14 +569,14 @@ COMMIT;
WARNING: duplicate key value violates unique constraint "ddl_commands_command_key"
DETAIL: Key (command)=(CREATE INDEX) already exists.
CONTEXT: while executing command on localhost:57638
ERROR: failed to prepare transaction
ERROR: failure on connection marked as essential: localhost:57638
-- Nothing from the block should have committed
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'single_shard_items';
indexname | tablename
-----------+-----------
(0 rows)
-- Now try with 2pc off
-- Now try with 2pc off, that'll result in invalid shards
RESET citus.multi_shard_commit_protocol;
BEGIN;
CREATE INDEX single_index_2 ON single_shard_items(id);
@ -580,8 +584,9 @@ NOTICE: using one-phase commit for distributed DDL commands
HINT: You can enable two-phase commit for extra safety with: SET citus.multi_shard_commit_protocol TO '2pc'
CREATE INDEX single_index_3 ON single_shard_items(name);
COMMIT;
WARNING: failed to commit transaction on localhost:57638
-- The block should have committed with a warning
WARNING: duplicate key value violates unique constraint "ddl_commands_command_key"
DETAIL: Key (command)=(CREATE INDEX) already exists.
CONTEXT: while executing command on localhost:57638
SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'single_shard_items';
indexname | tablename
----------------+--------------------
@ -653,8 +658,8 @@ INSERT INTO test_ab VALUES (2, 11);
CREATE UNIQUE INDEX temp_unique_index_1 ON test_ab(a);
WARNING: could not create unique index "temp_unique_index_1_220022"
DETAIL: Key (a)=(2) is duplicated.
CONTEXT: while executing command on localhost:57638
ERROR: could not execute DDL command on worker node shards
CONTEXT: while executing command on localhost:57637
ERROR: failure on connection marked as essential: localhost:57637
SELECT shardid FROM pg_dist_shard_placement NATURAL JOIN pg_dist_shard
WHERE logicalrelid='test_ab'::regclass AND shardstate=3;
shardid

View File

@ -702,8 +702,4 @@ SELECT master_create_distributed_table('composite_partition_column_table', 'comp
WARNING: function min(number_pack) does not exist
HINT: No function matches the given name and argument types. You might need to add explicit type casts.
CONTEXT: while executing command on localhost:57637
WARNING: function min(number_pack) does not exist
HINT: No function matches the given name and argument types. You might need to add explicit type casts.
CONTEXT: while executing command on localhost:57638
WARNING: could not get statistics for shard public.composite_partition_column_table_560164
DETAIL: Setting shard statistics to NULL
ERROR: failure on connection marked as essential: localhost:57637

View File

@ -131,23 +131,7 @@ INSERT INTO labs VALUES (6, 'Bell Labs');
SELECT count(*) FROM researchers WHERE lab_id = 6;
ABORT;
-- applies to DDL, too
BEGIN;
INSERT INTO labs VALUES (6, 'Bell Labs');
ALTER TABLE labs ADD COLUMN motto text;
COMMIT;
-- whether it occurs first or second
BEGIN;
ALTER TABLE labs ADD COLUMN motto text;
INSERT INTO labs VALUES (6, 'Bell Labs');
COMMIT;
-- but the DDL should correctly roll back
\d labs
SELECT * FROM labs WHERE id = 6;
-- COPY can't happen second,
-- Check COPY can happen after INSERT
BEGIN;
INSERT INTO labs VALUES (6, 'Bell Labs');
\copy labs from stdin delimiter ','
@ -155,7 +139,7 @@ INSERT INTO labs VALUES (6, 'Bell Labs');
\.
COMMIT;
-- though it will work if before any modifications
-- Check COPY can happen before INSERT
BEGIN;
\copy labs from stdin delimiter ','
10,Weyland-Yutani
@ -164,7 +148,7 @@ SELECT name FROM labs WHERE id = 10;
INSERT INTO labs VALUES (6, 'Bell Labs');
COMMIT;
-- but a double-copy isn't allowed (the first will persist)
-- Two COPYs are also ok
BEGIN;
\copy labs from stdin delimiter ','
11,Planet Express
@ -176,13 +160,14 @@ COMMIT;
SELECT name FROM labs WHERE id = 11;
-- finally, ALTER and copy aren't compatible
-- finally, check ALTER and copy are compatible
BEGIN;
ALTER TABLE labs ADD COLUMN motto text;
ALTER TABLE labs ADD COLUMN motto2 text;
\copy labs from stdin delimiter ','
12,fsociety,lol
\.
COMMIT;
ALTER TABLE labs DROP COLUMN motto2;
-- but the DDL should correctly roll back
\d labs
@ -193,10 +178,10 @@ BEGIN;
\copy labs from stdin delimiter ','
12,fsociety
\.
ALTER TABLE labs ADD COLUMN motto text;
ALTER TABLE labs ADD COLUMN motto3 text;
COMMIT;
-- the DDL fails, but copy persists
-- the DDL fails, and copy does not persist
\d labs
SELECT * FROM labs WHERE id = 12;

View File

@ -20,15 +20,12 @@ ALTER TABLE testtableddl ALTER COLUMN distributecol TYPE text;
-- verify that the distribution column can't be dropped
ALTER TABLE testtableddl DROP COLUMN distributecol;
-- verify that the table cannot be dropped in a transaction block
\set VERBOSITY terse
-- verify that the table can be dropped in a transaction block
BEGIN;
SELECT 1;
DROP TABLE testtableddl;
ROLLBACK;
\set VERBOSITY default
-- verify that the table can be dropped
DROP TABLE testtableddl;
SELECT 1;
COMMIT;
-- verify that the table can dropped even if shards exist
CREATE TABLE testtableddl(somecol int, distributecol text NOT NULL);

View File

@ -41,7 +41,7 @@ SELECT count(*) FROM test_truncate_append;
-- verify no shard exists anymore
SELECT shardid FROM pg_dist_shard where logicalrelid = 'test_truncate_append'::regclass;
-- command can not be run inside transaction
-- verify command can be run inside transaction
BEGIN; TRUNCATE TABLE test_truncate_append; COMMIT;
DROP TABLE test_truncate_append;
@ -173,4 +173,4 @@ TRUNCATE TABLE "a b append";
-- verify all shards are dropped
SELECT shardid FROM pg_dist_shard where logicalrelid = '"a b append"'::regclass;
DROP TABLE "a b append";
DROP TABLE "a b append";