diff --git a/src/backend/distributed/commands/multi_copy.c b/src/backend/distributed/commands/multi_copy.c index 43299d922..6a18bf533 100644 --- a/src/backend/distributed/commands/multi_copy.c +++ b/src/backend/distributed/commands/multi_copy.c @@ -71,8 +71,6 @@ #include "commands/copy.h" #include "commands/defrem.h" #include "distributed/citus_ruleutils.h" -#include "distributed/commit_protocol.h" -#include "distributed/connection_cache.h" #include "distributed/listutils.h" #include "distributed/metadata_cache.h" #include "distributed/master_metadata_utility.h" @@ -80,8 +78,9 @@ #include "distributed/metadata_cache.h" #include "distributed/multi_copy.h" #include "distributed/multi_physical_planner.h" -#include "distributed/multi_shard_transaction.h" #include "distributed/pg_dist_partition.h" +#include "distributed/placement_connection.h" +#include "distributed/remote_commands.h" #include "distributed/resource_lock.h" #include "distributed/shardinterval_utils.h" #include "distributed/worker_protocol.h" @@ -127,12 +126,18 @@ #include "utils/tuplestore.h" #include "utils/memutils.h" +/* ShardConnections represents a set of connections for each placement of a shard */ +typedef struct ShardConnections +{ + int64 shardId; + List *connectionList; +} ShardConnections; /* constant used in binary protocol */ static const char BinarySignature[11] = "PGCOPY\n\377\r\n\0"; /* use a global connection to the master node in order to skip passing it around */ -static PGconn *masterConnection = NULL; +static MultiConnection *masterConnection = NULL; /* Local functions forward declarations */ @@ -141,29 +146,34 @@ static void CopyToExistingShards(CopyStmt *copyStatement, char *completionTag); static void CopyToNewShards(CopyStmt *copyStatement, char *completionTag, Oid relationId); static char MasterPartitionMethod(RangeVar *relation); static void RemoveMasterOptions(CopyStmt *copyStatement); -static void OpenCopyTransactions(CopyStmt *copyStatement, - ShardConnections *shardConnections, bool stopOnFailure, - bool useBinaryCopyFormat); +static HTAB * CreateShardConnectionHash(void); +static ShardConnections * GetShardConnections(HTAB *shardConnectionHash, int64 shardId, + bool *shardConnectionsFound); +static void OpenCopyConnections(CopyStmt *copyStatement, + ShardConnections *shardConnections, bool stopOnFailure, + bool useBinaryCopyFormat); static bool CanUseBinaryCopyFormat(TupleDesc tupleDescription, CopyOutState rowOutputState); static List * MasterShardPlacementList(uint64 shardId); static List * RemoteFinalizedShardPlacementList(uint64 shardId); -static void SendCopyBinaryHeaders(CopyOutState copyOutState, List *connectionList); -static void SendCopyBinaryFooters(CopyOutState copyOutState, List *connectionList); +static void SendCopyBinaryHeaders(CopyOutState copyOutState, int64 shardId, + List *connectionList); +static void SendCopyBinaryFooters(CopyOutState copyOutState, int64 shardId, + List *connectionList); static StringInfo ConstructCopyStatement(CopyStmt *copyStatement, int64 shardId, bool useBinaryCopyFormat); -static void SendCopyDataToAll(StringInfo dataBuffer, List *connectionList); -static void SendCopyDataToPlacement(StringInfo dataBuffer, PGconn *connection, - int64 shardId); -static void EndRemoteCopy(List *connectionList, bool stopOnFailure); -static void ReportCopyError(PGconn *connection, PGresult *result); +static void SendCopyDataToAll(StringInfo dataBuffer, int64 shardId, List *connectionList); +static void SendCopyDataToPlacement(StringInfo dataBuffer, int64 shardId, + MultiConnection *connection); +static void EndRemoteCopy(int64 shardId, List *connectionList, bool stopOnFailure); +static void ReportCopyError(MultiConnection *connection, PGresult *result); static uint32 AvailableColumnCount(TupleDesc tupleDescriptor); -static void StartCopyToNewShard(ShardConnections *shardConnections, - CopyStmt *copyStatement, bool useBinaryCopyFormat); +static int64 StartCopyToNewShard(ShardConnections *shardConnections, + CopyStmt *copyStatement, bool useBinaryCopyFormat); static int64 MasterCreateEmptyShard(char *relationName); static int64 CreateEmptyShard(char *relationName); static int64 RemoteCreateEmptyShard(char *relationName); -static void FinalizeCopyToNewShard(ShardConnections *shardConnections); +static void FinalizeCopyToNewShard(int64 shardId, ShardConnections *shardConnections); static void MasterUpdateShardStatistics(uint64 shardId); static void RemoteUpdateShardStatistics(uint64 shardId); @@ -187,6 +197,8 @@ CitusCopyFrom(CopyStmt *copyStatement, char *completionTag) { bool isCopyFromWorker = false; + BeginOrContinueCoordinatedTransaction(); + /* disallow COPY to/from file or program except for superusers */ if (copyStatement->filename != NULL && !superuser()) { @@ -208,6 +220,7 @@ CitusCopyFrom(CopyStmt *copyStatement, char *completionTag) } } + masterConnection = NULL; /* reset, might still be set after error */ isCopyFromWorker = IsCopyFromWorker(copyStatement); if (isCopyFromWorker) { @@ -268,79 +281,42 @@ CopyFromWorkerNode(CopyStmt *copyStatement, char *completionTag) NodeAddress *masterNodeAddress = MasterNodeAddress(copyStatement); char *nodeName = masterNodeAddress->nodeName; int32 nodePort = masterNodeAddress->nodePort; - char *nodeUser = CurrentUserName(); + Oid relationId = InvalidOid; + char partitionMethod = 0; + char *schemaName = NULL; - if (XactModificationLevel > XACT_MODIFICATION_NONE) + masterConnection = GetNodeConnection(NEW_CONNECTION | CACHED_CONNECTION, + nodeName, nodePort); + ClaimConnectionExclusively(masterConnection); + + /* run all metadata commands in a transaction */ + AdjustRemoteTransactionState(masterConnection); + + /* strip schema name for local reference */ + schemaName = copyStatement->relation->schemaname; + copyStatement->relation->schemaname = NULL; + + relationId = RangeVarGetRelid(copyStatement->relation, NoLock, false); + + /* put schema name back */ + copyStatement->relation->schemaname = schemaName; + + partitionMethod = MasterPartitionMethod(copyStatement->relation); + if (partitionMethod != DISTRIBUTE_BY_APPEND) { - ereport(ERROR, (errcode(ERRCODE_ACTIVE_SQL_TRANSACTION), - errmsg("distributed copy operations must not appear in " - "transaction blocks containing other distributed " - "modifications"))); + ereport(ERROR, (errmsg("copy from worker nodes is only supported " + "for append-partitioned tables"))); } - masterConnection = ConnectToNode(nodeName, nodePort, nodeUser); + /* + * Remove master node options from the copy statement because they are not + * recognized by PostgreSQL machinery. + */ + RemoveMasterOptions(copyStatement); - PG_TRY(); - { - PGresult *queryResult = NULL; - Oid relationId = InvalidOid; - char partitionMethod = 0; + CopyToNewShards(copyStatement, completionTag, relationId); - /* strip schema name for local reference */ - char *schemaName = copyStatement->relation->schemaname; - copyStatement->relation->schemaname = NULL; - - relationId = RangeVarGetRelid(copyStatement->relation, NoLock, false); - - /* put schema name back */ - copyStatement->relation->schemaname = schemaName; - - partitionMethod = MasterPartitionMethod(copyStatement->relation); - if (partitionMethod != DISTRIBUTE_BY_APPEND) - { - ereport(ERROR, (errmsg("copy from worker nodes is only supported " - "for append-partitioned tables"))); - } - - /* run all metadata commands in a transaction */ - queryResult = PQexec(masterConnection, "BEGIN"); - if (PQresultStatus(queryResult) != PGRES_COMMAND_OK) - { - ereport(ERROR, (errmsg("could not start to update master node metadata"))); - } - - PQclear(queryResult); - - /* - * Remove master node options from the copy statement because they are not - * recognized by PostgreSQL machinery. - */ - RemoveMasterOptions(copyStatement); - - CopyToNewShards(copyStatement, completionTag, relationId); - - /* commit metadata transactions */ - queryResult = PQexec(masterConnection, "COMMIT"); - if (PQresultStatus(queryResult) != PGRES_COMMAND_OK) - { - ereport(ERROR, (errmsg("could not commit master node metadata changes"))); - } - - PQclear(queryResult); - - /* close the connection */ - PQfinish(masterConnection); - masterConnection = NULL; - } - PG_CATCH(); - { - /* close the connection */ - PQfinish(masterConnection); - masterConnection = NULL; - - PG_RE_THROW(); - } - PG_END_TRY(); + UnclaimConnection(masterConnection); } @@ -371,9 +347,9 @@ CopyToExistingShards(CopyStmt *copyStatement, char *completionTag) ShardInterval **shardIntervalCache = NULL; bool useBinarySearch = false; - HTAB *copyConnectionHash = NULL; + HTAB *shardConnectionHash = NULL; + HASH_SEQ_STATUS shardConnectionIterator; ShardConnections *shardConnections = NULL; - List *connectionList = NIL; EState *executorState = NULL; MemoryContext executorTupleContext = NULL; @@ -387,6 +363,8 @@ CopyToExistingShards(CopyStmt *copyStatement, char *completionTag) Var *partitionColumn = PartitionColumn(tableId, 0); char partitionMethod = PartitionMethod(tableId); + ErrorContextCallback errorCallback; + /* get hash function for partition column */ hashFunction = cacheEntry->hashFunction; @@ -467,152 +445,120 @@ CopyToExistingShards(CopyStmt *copyStatement, char *completionTag) columnOutputFunctions = ColumnOutputFunctions(tupleDescriptor, copyOutState->binary); - /* - * Create a mapping of shard id to a connection for each of its placements. - * The hash should be initialized before the PG_TRY, since it is used in - * PG_CATCH. Otherwise, it may be undefined in the PG_CATCH (see sigsetjmp - * documentation). - */ - copyConnectionHash = CreateShardConnectionHash(TopTransactionContext); + /* create a mapping of shard id to a connection for each of its placements */ + shardConnectionHash = CreateShardConnectionHash(); - /* we use a PG_TRY block to roll back on errors (e.g. in NextCopyFrom) */ - PG_TRY(); + /* set up callback to identify error line number */ + errorCallback.callback = CopyFromErrorCallback; + errorCallback.arg = (void *) copyState; + errorCallback.previous = error_context_stack; + error_context_stack = &errorCallback; + + while (true) { - ErrorContextCallback errorCallback; + bool nextRowFound = false; + Datum partitionColumnValue = 0; + ShardInterval *shardInterval = NULL; + int64 shardId = 0; + bool shardConnectionsFound = false; + MemoryContext oldContext = NULL; - /* set up callback to identify error line number */ - errorCallback.callback = CopyFromErrorCallback; - errorCallback.arg = (void *) copyState; - errorCallback.previous = error_context_stack; - error_context_stack = &errorCallback; + ResetPerTupleExprContext(executorState); - /* ensure transactions have unique names on worker nodes */ - InitializeDistributedTransaction(); + oldContext = MemoryContextSwitchTo(executorTupleContext); - while (true) + /* parse a row from the input */ + nextRowFound = NextCopyFrom(copyState, executorExpressionContext, + columnValues, columnNulls, NULL); + + if (!nextRowFound) { - bool nextRowFound = false; - Datum partitionColumnValue = 0; - ShardInterval *shardInterval = NULL; - int64 shardId = 0; - bool shardConnectionsFound = false; - MemoryContext oldContext = NULL; - - ResetPerTupleExprContext(executorState); - - oldContext = MemoryContextSwitchTo(executorTupleContext); - - /* parse a row from the input */ - nextRowFound = NextCopyFrom(copyState, executorExpressionContext, - columnValues, columnNulls, NULL); - - if (!nextRowFound) - { - MemoryContextSwitchTo(oldContext); - break; - } - - CHECK_FOR_INTERRUPTS(); - - /* find the partition column value */ - - if (columnNulls[partitionColumn->varattno - 1]) - { - ereport(ERROR, (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED), - errmsg("cannot copy row with NULL value " - "in partition column"))); - } - - partitionColumnValue = columnValues[partitionColumn->varattno - 1]; - - /* find the shard interval and id for the partition column value */ - shardInterval = FindShardInterval(partitionColumnValue, shardIntervalCache, - shardCount, partitionMethod, - compareFunction, hashFunction, - useBinarySearch); - if (shardInterval == NULL) - { - ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE), - errmsg("could not find shard for partition column " - "value"))); - } - - shardId = shardInterval->shardId; - MemoryContextSwitchTo(oldContext); - - /* get existing connections to the shard placements, if any */ - shardConnections = GetShardHashConnections(copyConnectionHash, shardId, - &shardConnectionsFound); - if (!shardConnectionsFound) - { - /* open connections and initiate COPY on shard placements */ - OpenCopyTransactions(copyStatement, shardConnections, false, - copyOutState->binary); - - /* send copy binary headers to shard placements */ - if (copyOutState->binary) - { - SendCopyBinaryHeaders(copyOutState, - shardConnections->connectionList); - } - } - - /* replicate row to shard placements */ - resetStringInfo(copyOutState->fe_msgbuf); - AppendCopyRowData(columnValues, columnNulls, tupleDescriptor, - copyOutState, columnOutputFunctions); - SendCopyDataToAll(copyOutState->fe_msgbuf, shardConnections->connectionList); - - processedRowCount += 1; + break; } - connectionList = ConnectionList(copyConnectionHash); + CHECK_FOR_INTERRUPTS(); + /* find the partition column value */ + + if (columnNulls[partitionColumn->varattno - 1]) + { + ereport(ERROR, (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED), + errmsg("cannot copy row with NULL value " + "in partition column"))); + } + + partitionColumnValue = columnValues[partitionColumn->varattno - 1]; + + /* find the shard interval and id for the partition column value */ + shardInterval = FindShardInterval(partitionColumnValue, shardIntervalCache, + shardCount, partitionMethod, + compareFunction, hashFunction, + useBinarySearch); + if (shardInterval == NULL) + { + ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE), + errmsg("could not find shard for partition column " + "value"))); + } + + shardId = shardInterval->shardId; + + MemoryContextSwitchTo(oldContext); + + /* get existing connections to the shard placements, if any */ + shardConnections = GetShardConnections(shardConnectionHash, + shardId, + &shardConnectionsFound); + if (!shardConnectionsFound) + { + /* open connections and initiate COPY on shard placements */ + OpenCopyConnections(copyStatement, shardConnections, false, + copyOutState->binary); + + /* send copy binary headers to shard placements */ + if (copyOutState->binary) + { + SendCopyBinaryHeaders(copyOutState, shardId, + shardConnections->connectionList); + } + } + + /* replicate row to shard placements */ + resetStringInfo(copyOutState->fe_msgbuf); + AppendCopyRowData(columnValues, columnNulls, tupleDescriptor, + copyOutState, columnOutputFunctions); + SendCopyDataToAll(copyOutState->fe_msgbuf, shardId, + shardConnections->connectionList); + + processedRowCount += 1; + } + + /* all lines have been copied, stop showing line number in errors */ + error_context_stack = errorCallback.previous; + + CHECK_FOR_INTERRUPTS(); + + /* finish copy on all open connections */ + hash_seq_init(&shardConnectionIterator, shardConnectionHash); + shardConnections = (ShardConnections *) hash_seq_search(&shardConnectionIterator); + while (shardConnections != NULL) + { /* send copy binary footers to all shard placements */ if (copyOutState->binary) { - SendCopyBinaryFooters(copyOutState, connectionList); + SendCopyBinaryFooters(copyOutState, shardConnections->shardId, + shardConnections->connectionList); } - /* all lines have been copied, stop showing line number in errors */ - error_context_stack = errorCallback.previous; - /* close the COPY input on all shard placements */ - EndRemoteCopy(connectionList, true); + EndRemoteCopy(shardConnections->shardId, shardConnections->connectionList, true); - if (MultiShardCommitProtocol == COMMIT_PROTOCOL_2PC) - { - PrepareRemoteTransactions(connectionList); - } - - EndCopyFrom(copyState); - heap_close(distributedRelation, NoLock); - - /* check for cancellation one last time before committing */ - CHECK_FOR_INTERRUPTS(); + shardConnections = (ShardConnections *) hash_seq_search(&shardConnectionIterator); } - PG_CATCH(); - { - List *abortConnectionList = NIL; - /* roll back all transactions */ - abortConnectionList = ConnectionList(copyConnectionHash); - EndRemoteCopy(abortConnectionList, false); - AbortRemoteTransactions(abortConnectionList); - CloseConnections(abortConnectionList); - - PG_RE_THROW(); - } - PG_END_TRY(); - - /* - * Ready to commit the transaction, this code is below the PG_TRY block because - * we do not want any of the transactions rolled back if a failure occurs. Instead, - * they should be rolled forward. - */ - CommitRemoteTransactions(connectionList, false); - CloseConnections(connectionList); + EndCopyFrom(copyState); + heap_close(distributedRelation, NoLock); if (completionTag != NULL) { @@ -645,11 +591,13 @@ CopyToNewShards(CopyStmt *copyStatement, char *completionTag, Oid relationId) const char *delimiterCharacter = "\t"; const char *nullPrintCharacter = "\\N"; - /* - * Shard connections should be initialized before the PG_TRY, since it is - * used in PG_CATCH. Otherwise, it may be undefined in the PG_CATCH - * (see sigsetjmp documentation). - */ + ErrorContextCallback errorCallback; + + int64 currentShardId = INVALID_SHARD_ID; + uint64 shardMaxSizeInBytes = (int64) ShardMaxSize * 1024L; + uint64 copiedDataSizeInBytes = 0; + uint64 processedRowCount = 0; + ShardConnections *shardConnections = (ShardConnections *) palloc0(sizeof(ShardConnections)); @@ -670,140 +618,124 @@ CopyToNewShards(CopyStmt *copyStatement, char *completionTag, Oid relationId) columnOutputFunctions = ColumnOutputFunctions(tupleDescriptor, copyOutState->binary); - /* we use a PG_TRY block to close connections on errors (e.g. in NextCopyFrom) */ - PG_TRY(); + /* set up callback to identify error line number */ + errorCallback.callback = CopyFromErrorCallback; + errorCallback.arg = (void *) copyState; + errorCallback.previous = error_context_stack; + + while (true) { - uint64 shardMaxSizeInBytes = (int64) ShardMaxSize * 1024L; - uint64 copiedDataSizeInBytes = 0; - uint64 processedRowCount = 0; + bool nextRowFound = false; + MemoryContext oldContext = NULL; + uint64 messageBufferSize = 0; - /* set up callback to identify error line number */ - ErrorContextCallback errorCallback; + ResetPerTupleExprContext(executorState); - errorCallback.callback = CopyFromErrorCallback; - errorCallback.arg = (void *) copyState; - errorCallback.previous = error_context_stack; + /* switch to tuple memory context and start showing line number in errors */ + error_context_stack = &errorCallback; + oldContext = MemoryContextSwitchTo(executorTupleContext); - while (true) + /* parse a row from the input */ + nextRowFound = NextCopyFrom(copyState, executorExpressionContext, + columnValues, columnNulls, NULL); + + if (!nextRowFound) { - bool nextRowFound = false; - MemoryContext oldContext = NULL; - uint64 messageBufferSize = 0; - - ResetPerTupleExprContext(executorState); - - /* switch to tuple memory context and start showing line number in errors */ - error_context_stack = &errorCallback; - oldContext = MemoryContextSwitchTo(executorTupleContext); - - /* parse a row from the input */ - nextRowFound = NextCopyFrom(copyState, executorExpressionContext, - columnValues, columnNulls, NULL); - - if (!nextRowFound) - { - /* switch to regular memory context and stop showing line number in errors */ - MemoryContextSwitchTo(oldContext); - error_context_stack = errorCallback.previous; - break; - } - - CHECK_FOR_INTERRUPTS(); - /* switch to regular memory context and stop showing line number in errors */ MemoryContextSwitchTo(oldContext); error_context_stack = errorCallback.previous; - - /* - * If copied data size is zero, this means either this is the first - * line in the copy or we just filled the previous shard up to its - * capacity. Either way, we need to create a new shard and - * start copying new rows into it. - */ - if (copiedDataSizeInBytes == 0) - { - /* create shard and open connections to shard placements */ - StartCopyToNewShard(shardConnections, copyStatement, - copyOutState->binary); - - /* send copy binary headers to shard placements */ - if (copyOutState->binary) - { - SendCopyBinaryHeaders(copyOutState, - shardConnections->connectionList); - } - } - - /* replicate row to shard placements */ - resetStringInfo(copyOutState->fe_msgbuf); - AppendCopyRowData(columnValues, columnNulls, tupleDescriptor, - copyOutState, columnOutputFunctions); - SendCopyDataToAll(copyOutState->fe_msgbuf, shardConnections->connectionList); - - messageBufferSize = copyOutState->fe_msgbuf->len; - copiedDataSizeInBytes = copiedDataSizeInBytes + messageBufferSize; - - /* - * If we filled up this shard to its capacity, send copy binary footers - * to shard placements, commit copy transactions, close connections - * and finally update shard statistics. - * - * */ - if (copiedDataSizeInBytes > shardMaxSizeInBytes) - { - if (copyOutState->binary) - { - SendCopyBinaryFooters(copyOutState, - shardConnections->connectionList); - } - FinalizeCopyToNewShard(shardConnections); - MasterUpdateShardStatistics(shardConnections->shardId); - - copiedDataSizeInBytes = 0; - } - - processedRowCount += 1; + break; } + CHECK_FOR_INTERRUPTS(); + + /* switch to regular memory context and stop showing line number in errors */ + MemoryContextSwitchTo(oldContext); + error_context_stack = errorCallback.previous; + /* - * For the last shard, send copy binary footers to shard placements, - * commit copy transactions, close connections and finally update shard - * statistics. If no row is send, there is no shard to finalize the - * copy command. + * If copied data size is zero, this means either this is the first + * line in the copy or we just filled the previous shard up to its + * capacity. Either way, we need to create a new shard and + * start copying new rows into it. */ - if (copiedDataSizeInBytes > 0) + if (copiedDataSizeInBytes == 0) + { + /* create shard and open connections to shard placements */ + currentShardId = StartCopyToNewShard(shardConnections, copyStatement, + copyOutState->binary); + + /* send copy binary headers to shard placements */ + if (copyOutState->binary) + { + SendCopyBinaryHeaders(copyOutState, currentShardId, + shardConnections->connectionList); + } + } + + /* replicate row to shard placements */ + resetStringInfo(copyOutState->fe_msgbuf); + AppendCopyRowData(columnValues, columnNulls, tupleDescriptor, + copyOutState, columnOutputFunctions); + SendCopyDataToAll(copyOutState->fe_msgbuf, currentShardId, + shardConnections->connectionList); + + messageBufferSize = copyOutState->fe_msgbuf->len; + copiedDataSizeInBytes = copiedDataSizeInBytes + messageBufferSize; + + /* + * If we filled up this shard to its capacity, send copy binary footers + * to shard placements, commit copy transactions, close connections + * and finally update shard statistics. + * + * */ + if (copiedDataSizeInBytes > shardMaxSizeInBytes) { if (copyOutState->binary) { - SendCopyBinaryFooters(copyOutState, + SendCopyBinaryFooters(copyOutState, currentShardId, shardConnections->connectionList); } - FinalizeCopyToNewShard(shardConnections); + FinalizeCopyToNewShard(currentShardId, shardConnections); MasterUpdateShardStatistics(shardConnections->shardId); + + copiedDataSizeInBytes = 0; + currentShardId = INVALID_SHARD_ID; } - EndCopyFrom(copyState); - heap_close(distributedRelation, NoLock); - - /* check for cancellation one last time before returning */ - CHECK_FOR_INTERRUPTS(); - - if (completionTag != NULL) - { - snprintf(completionTag, COMPLETION_TAG_BUFSIZE, - "COPY " UINT64_FORMAT, processedRowCount); - } + processedRowCount += 1; } - PG_CATCH(); + + /* + * For the last shard, send copy binary footers to shard placements, + * commit copy transactions, close connections and finally update shard + * statistics. If no row is send, there is no shard to finalize the + * copy command. + */ + if (copiedDataSizeInBytes > 0) { - /* roll back all transactions */ - EndRemoteCopy(shardConnections->connectionList, false); - AbortRemoteTransactions(shardConnections->connectionList); - CloseConnections(shardConnections->connectionList); + Assert(currentShardId != INVALID_SHARD_ID); - PG_RE_THROW(); + if (copyOutState->binary) + { + SendCopyBinaryFooters(copyOutState, currentShardId, + shardConnections->connectionList); + } + FinalizeCopyToNewShard(currentShardId, shardConnections); + MasterUpdateShardStatistics(shardConnections->shardId); + } + + EndCopyFrom(copyState); + heap_close(distributedRelation, NoLock); + + /* check for cancellation one last time before returning */ + CHECK_FOR_INTERRUPTS(); + + if (completionTag != NULL) + { + snprintf(completionTag, COMPLETION_TAG_BUFSIZE, + "COPY " UINT64_FORMAT, processedRowCount); } - PG_END_TRY(); } @@ -859,7 +791,7 @@ MasterPartitionMethod(RangeVar *relation) StringInfo partitionMethodCommand = makeStringInfo(); appendStringInfo(partitionMethodCommand, PARTITION_METHOD_QUERY, qualifiedName); - queryResult = PQexec(masterConnection, partitionMethodCommand->data); + queryResult = PQexec(masterConnection->conn, partitionMethodCommand->data); if (PQresultStatus(queryResult) == PGRES_TUPLES_OK) { char *partitionMethodString = PQgetvalue((PGresult *) queryResult, 0, 0); @@ -873,7 +805,7 @@ MasterPartitionMethod(RangeVar *relation) } else { - WarnRemoteError(masterConnection, queryResult); + ReportResultError(masterConnection, queryResult, WARNING); ereport(ERROR, (errmsg("could not get the partition method of the " "distributed table"))); } @@ -913,25 +845,65 @@ RemoveMasterOptions(CopyStmt *copyStatement) } +static HTAB * +CreateShardConnectionHash(void) +{ + HTAB *shardConnectionsHash = NULL; + int hashFlags = 0; + HASHCTL info; + memset(&info, 0, sizeof(info)); + info.keysize = sizeof(int64); + info.entrysize = sizeof(ShardConnections); + info.hash = tag_hash; + info.hcxt = TopTransactionContext; + + hashFlags = HASH_ELEM | HASH_FUNCTION | HASH_CONTEXT; + shardConnectionsHash = hash_create("Shard Connections Hash", + 256, &info, + hashFlags); + + return shardConnectionsHash; +} + + +static ShardConnections * +GetShardConnections(HTAB *shardConnectionHash, int64 shardId, + bool *shardConnectionsFound) +{ + ShardConnections *shardConnections = NULL; + + shardConnections = (ShardConnections *) hash_search(shardConnectionHash, + &shardId, + HASH_ENTER, + shardConnectionsFound); + if (!*shardConnectionsFound) + { + shardConnections->shardId = shardId; + shardConnections->connectionList = NIL; + } + + return shardConnections; +} + + /* - * OpenCopyTransactions opens a connection for each placement of a shard and - * starts a COPY transaction. If a connection cannot be opened, then the shard - * placement is marked as inactive and the COPY continues with the remaining - * shard placements. + * OpenCopyConnections opens a connection for each placement of a shard and + * starts COPY. If a connection cannot be opened, then the shard placement is + * marked as inactive and the COPY continues with the remaining shard + * placements. */ static void -OpenCopyTransactions(CopyStmt *copyStatement, ShardConnections *shardConnections, - bool stopOnFailure, bool useBinaryCopyFormat) +OpenCopyConnections(CopyStmt *copyStatement, ShardConnections *shardConnections, + bool stopOnFailure, bool useBinaryCopyFormat) { List *finalizedPlacementList = NIL; List *failedPlacementList = NIL; ListCell *placementCell = NULL; - ListCell *failedPlacementCell = NULL; List *connectionList = NULL; int64 shardId = shardConnections->shardId; MemoryContext localContext = AllocSetContextCreate(CurrentMemoryContext, - "OpenCopyTransactions", + "OpenCopyConnections", ALLOCSET_DEFAULT_MINSIZE, ALLOCSET_DEFAULT_INITSIZE, ALLOCSET_DEFAULT_MAXSIZE); @@ -943,28 +915,31 @@ OpenCopyTransactions(CopyStmt *copyStatement, ShardConnections *shardConnections MemoryContextSwitchTo(oldContext); - if (XactModificationLevel > XACT_MODIFICATION_NONE) - { - ereport(ERROR, (errcode(ERRCODE_ACTIVE_SQL_TRANSACTION), - errmsg("distributed copy operations must not appear in " - "transaction blocks containing other distributed " - "modifications"))); - } - foreach(placementCell, finalizedPlacementList) { ShardPlacement *placement = (ShardPlacement *) lfirst(placementCell); char *nodeName = placement->nodeName; int nodePort = placement->nodePort; - char *nodeUser = CurrentUserName(); - PGconn *connection = ConnectToNode(nodeName, nodePort, nodeUser); - - TransactionConnection *transactionConnection = NULL; + MultiConnection *connection = NULL; + uint32 connectionFlags = NEW_CONNECTION | CACHED_CONNECTION | FOR_DML; StringInfo copyCommand = NULL; PGresult *result = NULL; + if (stopOnFailure) + { + connectionFlags |= CRITICAL_CONNECTION; + } + + /* + * FIXME: should make connection establishment parallel, by using + * StartPlacementConnection etc. + */ + connection = GetPlacementConnection(connectionFlags, placement); + ClaimConnectionExclusively(connection); + if (connection == NULL) { + /* FIXME: remove or replace with working code */ if (stopOnFailure) { ereport(ERROR, (errmsg("could not open connection to %s:%d", @@ -975,39 +950,26 @@ OpenCopyTransactions(CopyStmt *copyStatement, ShardConnections *shardConnections continue; } - result = PQexec(connection, "BEGIN"); - if (PQresultStatus(result) != PGRES_COMMAND_OK) - { - WarnRemoteError(connection, result); - failedPlacementList = lappend(failedPlacementList, placement); + /* start transaction unless already done so */ + AdjustRemoteTransactionState(connection); - PQclear(result); - continue; - } - - PQclear(result); copyCommand = ConstructCopyStatement(copyStatement, shardConnections->shardId, useBinaryCopyFormat); - result = PQexec(connection, copyCommand->data); + result = PQexec(connection->conn, copyCommand->data); if (PQresultStatus(result) != PGRES_COPY_IN) { - WarnRemoteError(connection, result); - failedPlacementList = lappend(failedPlacementList, placement); - + ReportResultError(connection, result, WARNING); PQclear(result); + MarkRemoteTransactionFailed(connection, true); + + /* failed placements will be invalidated by transaction machinery */ continue; } PQclear(result); - transactionConnection = palloc0(sizeof(TransactionConnection)); - - transactionConnection->connectionId = shardConnections->shardId; - transactionConnection->transactionState = TRANSACTION_STATE_COPY_STARTED; - transactionConnection->connection = connection; - - connectionList = lappend(connectionList, transactionConnection); + connectionList = lappend(connectionList, connection); } /* if all placements failed, error out */ @@ -1016,20 +978,6 @@ OpenCopyTransactions(CopyStmt *copyStatement, ShardConnections *shardConnections ereport(ERROR, (errmsg("could not find any active placements"))); } - /* - * If stopOnFailure is true, we just error out and code execution should - * never reach to this point. This is the case for copy from worker nodes. - */ - Assert(!stopOnFailure || list_length(failedPlacementList) == 0); - - /* otherwise, mark failed placements as inactive: they're stale */ - foreach(failedPlacementCell, failedPlacementList) - { - ShardPlacement *failedPlacement = (ShardPlacement *) lfirst(failedPlacementCell); - - UpdateShardPlacementState(failedPlacement->placementId, FILE_INACTIVE); - } - shardConnections->connectionList = connectionList; MemoryContextReset(localContext); @@ -1113,7 +1061,7 @@ RemoteFinalizedShardPlacementList(uint64 shardId) StringInfo shardPlacementsCommand = makeStringInfo(); appendStringInfo(shardPlacementsCommand, FINALIZED_SHARD_PLACEMENTS_QUERY, shardId); - queryResult = PQexec(masterConnection, shardPlacementsCommand->data); + queryResult = PQexec(masterConnection->conn, shardPlacementsCommand->data); if (PQresultStatus(queryResult) == PGRES_TUPLES_OK) { int rowCount = PQntuples(queryResult); @@ -1121,14 +1069,17 @@ RemoteFinalizedShardPlacementList(uint64 shardId) for (rowIndex = 0; rowIndex < rowCount; rowIndex++) { - char *nodeName = PQgetvalue(queryResult, rowIndex, 0); - - char *nodePortString = PQgetvalue(queryResult, rowIndex, 1); + char *placementIdString = PQgetvalue(queryResult, rowIndex, 0); + char *nodeName = PQgetvalue(queryResult, rowIndex, 1); + char *nodePortString = PQgetvalue(queryResult, rowIndex, 2); + uint64 placementId = atol(placementIdString); uint32 nodePort = atoi(nodePortString); ShardPlacement *shardPlacement = (ShardPlacement *) palloc0(sizeof(ShardPlacement)); + shardPlacement->shardId = shardId; + shardPlacement->placementId = placementId; shardPlacement->nodeName = nodeName; shardPlacement->nodePort = nodePort; @@ -1146,21 +1097,21 @@ RemoteFinalizedShardPlacementList(uint64 shardId) /* Send copy binary headers to given connections */ static void -SendCopyBinaryHeaders(CopyOutState copyOutState, List *connectionList) +SendCopyBinaryHeaders(CopyOutState copyOutState, int64 shardId, List *connectionList) { resetStringInfo(copyOutState->fe_msgbuf); AppendCopyBinaryHeaders(copyOutState); - SendCopyDataToAll(copyOutState->fe_msgbuf, connectionList); + SendCopyDataToAll(copyOutState->fe_msgbuf, shardId, connectionList); } /* Send copy binary footers to given connections */ static void -SendCopyBinaryFooters(CopyOutState copyOutState, List *connectionList) +SendCopyBinaryFooters(CopyOutState copyOutState, int64 shardId, List *connectionList) { resetStringInfo(copyOutState->fe_msgbuf); AppendCopyBinaryFooters(copyOutState); - SendCopyDataToAll(copyOutState->fe_msgbuf, connectionList); + SendCopyDataToAll(copyOutState->fe_msgbuf, shardId, connectionList); } @@ -1203,17 +1154,15 @@ ConstructCopyStatement(CopyStmt *copyStatement, int64 shardId, bool useBinaryCop * SendCopyDataToAll sends copy data to all connections in a list. */ static void -SendCopyDataToAll(StringInfo dataBuffer, List *connectionList) +SendCopyDataToAll(StringInfo dataBuffer, int64 shardId, List *connectionList) { ListCell *connectionCell = NULL; foreach(connectionCell, connectionList) { - TransactionConnection *transactionConnection = - (TransactionConnection *) lfirst(connectionCell); - PGconn *connection = transactionConnection->connection; - int64 shardId = transactionConnection->connectionId; + MultiConnection *connection = + (MultiConnection *) lfirst(connectionCell); - SendCopyDataToPlacement(dataBuffer, connection, shardId); + SendCopyDataToPlacement(dataBuffer, shardId, connection); } } @@ -1223,73 +1172,71 @@ SendCopyDataToAll(StringInfo dataBuffer, List *connectionList) * over the given connection. */ static void -SendCopyDataToPlacement(StringInfo dataBuffer, PGconn *connection, int64 shardId) +SendCopyDataToPlacement(StringInfo dataBuffer, int64 shardId, MultiConnection *connection) { - int copyResult = PQputCopyData(connection, dataBuffer->data, dataBuffer->len); + int copyResult = PQputCopyData(connection->conn, dataBuffer->data, dataBuffer->len); if (copyResult != 1) { - char *nodeName = ConnectionGetOptionValue(connection, "host"); - char *nodePort = ConnectionGetOptionValue(connection, "port"); - ereport(ERROR, (errcode(ERRCODE_IO_ERROR), - errmsg("failed to COPY to shard %ld on %s:%s", - shardId, nodeName, nodePort))); + errmsg("failed to COPY to shard %ld on %s:%d", + shardId, connection->hostname, connection->port))); } } /* - * EndRemoteCopy ends the COPY input on all connections. If stopOnFailure - * is true, then EndRemoteCopy reports an error on failure, otherwise it - * reports a warning or continues. + * EndRemoteCopy ends the COPY input on all connections. If stopOnFailure is + * true, then EndRemoteCopy reports an error on failure, otherwise it reports + * a warning or continues. */ static void -EndRemoteCopy(List *connectionList, bool stopOnFailure) +EndRemoteCopy(int64 shardId, List *connectionList, bool stopOnFailure) { ListCell *connectionCell = NULL; foreach(connectionCell, connectionList) { - TransactionConnection *transactionConnection = - (TransactionConnection *) lfirst(connectionCell); - PGconn *connection = transactionConnection->connection; - int64 shardId = transactionConnection->connectionId; + MultiConnection *connection = (MultiConnection *) lfirst(connectionCell); int copyEndResult = 0; PGresult *result = NULL; +#if 0 + /* FIXME: Is this actually a relevant case? */ if (transactionConnection->transactionState != TRANSACTION_STATE_COPY_STARTED) { /* a failure occurred after having previously called EndRemoteCopy */ continue; } +#endif /* end the COPY input */ - copyEndResult = PQputCopyEnd(connection, NULL); - transactionConnection->transactionState = TRANSACTION_STATE_OPEN; + copyEndResult = PQputCopyEnd(connection->conn, NULL); if (copyEndResult != 1) { if (stopOnFailure) { - char *nodeName = ConnectionGetOptionValue(connection, "host"); - char *nodePort = ConnectionGetOptionValue(connection, "port"); - ereport(ERROR, (errcode(ERRCODE_IO_ERROR), - errmsg("failed to COPY to shard %ld on %s:%s", - shardId, nodeName, nodePort))); + errmsg("failed to COPY to shard %ld on %s:%d", + shardId, connection->hostname, connection->port))); } continue; } /* check whether there were any COPY errors */ - result = PQgetResult(connection); + result = PQgetResult(connection->conn); if (PQresultStatus(result) != PGRES_COMMAND_OK && stopOnFailure) { ReportCopyError(connection, result); } PQclear(result); + + result = PQgetResult(connection->conn); + Assert(!result); + + UnclaimConnection(connection); } } @@ -1299,7 +1246,7 @@ EndRemoteCopy(List *connectionList, bool stopOnFailure) * the remote COPY error messages. */ static void -ReportCopyError(PGconn *connection, PGresult *result) +ReportCopyError(MultiConnection *connection, PGresult *result) { char *remoteMessage = PQresultErrorField(result, PG_DIAG_MESSAGE_PRIMARY); @@ -1315,10 +1262,8 @@ ReportCopyError(PGconn *connection, PGresult *result) { /* probably a connection problem, get the message from the connection */ char *lastNewlineIndex = NULL; - char *nodeName = ConnectionGetOptionValue(connection, "host"); - char *nodePort = ConnectionGetOptionValue(connection, "port"); - remoteMessage = PQerrorMessage(connection); + remoteMessage = PQerrorMessage(connection->conn); lastNewlineIndex = strrchr(remoteMessage, '\n'); /* trim trailing newline, if any */ @@ -1328,7 +1273,8 @@ ReportCopyError(PGconn *connection, PGresult *result) } ereport(ERROR, (errcode(ERRCODE_IO_ERROR), - errmsg("failed to complete COPY on %s:%s", nodeName, nodePort), + errmsg("failed to complete COPY on %s:%u", + connection->hostname, connection->port), errdetail("%s", remoteMessage))); } } @@ -1531,7 +1477,7 @@ AppendCopyBinaryFooters(CopyOutState footerOutputState) * StartCopyToNewShard creates a new shard and related shard placements and * opens connections to shard placements. */ -static void +static int64 StartCopyToNewShard(ShardConnections *shardConnections, CopyStmt *copyStatement, bool useBinaryCopyFormat) { @@ -1543,11 +1489,12 @@ StartCopyToNewShard(ShardConnections *shardConnections, CopyStmt *copyStatement, shardConnections->shardId = shardId; - list_free_deep(shardConnections->connectionList); shardConnections->connectionList = NIL; /* connect to shards placements and start transactions */ - OpenCopyTransactions(copyStatement, shardConnections, true, useBinaryCopyFormat); + OpenCopyConnections(copyStatement, shardConnections, true, useBinaryCopyFormat); + + return shardId; } @@ -1604,7 +1551,7 @@ RemoteCreateEmptyShard(char *relationName) StringInfo createEmptyShardCommand = makeStringInfo(); appendStringInfo(createEmptyShardCommand, CREATE_EMPTY_SHARD_QUERY, relationName); - queryResult = PQexec(masterConnection, createEmptyShardCommand->data); + queryResult = PQexec(masterConnection->conn, createEmptyShardCommand->data); if (PQresultStatus(queryResult) == PGRES_TUPLES_OK) { char *shardIdString = PQgetvalue((PGresult *) queryResult, 0, 0); @@ -1613,7 +1560,7 @@ RemoteCreateEmptyShard(char *relationName) } else { - WarnRemoteError(masterConnection, queryResult); + ReportResultError(masterConnection, queryResult, WARNING); ereport(ERROR, (errmsg("could not create a new empty shard on the remote node"))); } @@ -1624,18 +1571,23 @@ RemoteCreateEmptyShard(char *relationName) /* - * FinalizeCopyToNewShard commits copy transaction and closes connections to - * shard placements. + * FinalizeCopyToNewShard ends the COPY and marks connection as inactive for + * all shard placements. */ static void -FinalizeCopyToNewShard(ShardConnections *shardConnections) +FinalizeCopyToNewShard(int64 shardId, ShardConnections *shardConnections) { - /* close the COPY input on all shard placements */ - EndRemoteCopy(shardConnections->connectionList, true); + ListCell *connectionCell = NULL; - /* commit transactions and close connections */ - CommitRemoteTransactions(shardConnections->connectionList, true); - CloseConnections(shardConnections->connectionList); + /* close the COPY input on all shard placements */ + EndRemoteCopy(shardId, shardConnections->connectionList, true); + + foreach(connectionCell, shardConnections->connectionList) + { + MultiConnection *connection = (MultiConnection *) lfirst(connectionCell); + + UnclaimConnection(connection); + } } @@ -1669,7 +1621,7 @@ RemoteUpdateShardStatistics(uint64 shardId) appendStringInfo(updateShardStatisticsCommand, UPDATE_SHARD_STATISTICS_QUERY, shardId); - queryResult = PQexec(masterConnection, updateShardStatisticsCommand->data); + queryResult = PQexec(masterConnection->conn, updateShardStatisticsCommand->data); if (PQresultStatus(queryResult) != PGRES_TUPLES_OK) { ereport(ERROR, (errmsg("could not update shard statistics"))); diff --git a/src/backend/distributed/executor/multi_client_executor.c b/src/backend/distributed/executor/multi_client_executor.c index da852d5c1..50b25a6cf 100644 --- a/src/backend/distributed/executor/multi_client_executor.c +++ b/src/backend/distributed/executor/multi_client_executor.c @@ -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" diff --git a/src/backend/distributed/executor/multi_router_executor.c b/src/backend/distributed/executor/multi_router_executor.c index 3056bd9f9..0f1462032 100644 --- a/src/backend/distributed/executor/multi_router_executor.c +++ b/src/backend/distributed/executor/multi_router_executor.c @@ -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, ¶meterTypes, ¶meterValues); - 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); - } - } - } -} diff --git a/src/backend/distributed/executor/multi_utility.c b/src/backend/distributed/executor/multi_utility.c index 8864a7b16..ee6f91e99 100644 --- a/src/backend/distributed/executor/multi_utility.c +++ b/src/backend/distributed/executor/multi_utility.c @@ -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(, )" 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); } } diff --git a/src/backend/distributed/master/master_create_shards.c b/src/backend/distributed/master/master_create_shards.c index a55ed2842..f4d3fb788 100644 --- a/src/backend/distributed/master/master_create_shards.c +++ b/src/backend/distributed/master/master_create_shards.c @@ -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" diff --git a/src/backend/distributed/master/master_delete_protocol.c b/src/backend/distributed/master/master_delete_protocol.c index 0ed584ed8..63e743f77 100644 --- a/src/backend/distributed/master/master_delete_protocol.c +++ b/src/backend/distributed/master/master_delete_protocol.c @@ -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); diff --git a/src/backend/distributed/master/master_modify_multiple_shards.c b/src/backend/distributed/master/master_modify_multiple_shards.c index d54407341..e5827098d 100644 --- a/src/backend/distributed/master/master_modify_multiple_shards.c +++ b/src/backend/distributed/master/master_modify_multiple_shards.c @@ -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; -} diff --git a/src/backend/distributed/master/master_stage_protocol.c b/src/backend/distributed/master/master_stage_protocol.c index fa916c10a..f77184496 100644 --- a/src/backend/distributed/master/master_stage_protocol.c +++ b/src/backend/distributed/master/master_stage_protocol.c @@ -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; } diff --git a/src/backend/distributed/shared_library_init.c b/src/backend/distributed/shared_library_init.c index 549d20958..e39a4b039 100644 --- a/src/backend/distributed/shared_library_init.c +++ b/src/backend/distributed/shared_library_init.c @@ -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."), diff --git a/src/backend/distributed/transaction/commit_protocol.c b/src/backend/distributed/transaction/commit_protocol.c deleted file mode 100644 index 728f672f4..000000000 --- a/src/backend/distributed/transaction/commit_protocol.c +++ /dev/null @@ -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; -} diff --git a/src/backend/distributed/transaction/multi_shard_transaction.c b/src/backend/distributed/transaction/multi_shard_transaction.c deleted file mode 100644 index 712beaa5e..000000000 --- a/src/backend/distributed/transaction/multi_shard_transaction.c +++ /dev/null @@ -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); - } -} diff --git a/src/backend/distributed/transaction/remote_transaction.c b/src/backend/distributed/transaction/remote_transaction.c new file mode 100644 index 000000000..ae9deab03 --- /dev/null +++ b/src/backend/distributed/transaction/remote_transaction.c @@ -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))); + +} diff --git a/src/backend/distributed/transaction/transaction_management.c b/src/backend/distributed/transaction/transaction_management.c new file mode 100644 index 000000000..9dfd29ec8 --- /dev/null +++ b/src/backend/distributed/transaction/transaction_management.c @@ -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; + } +} diff --git a/src/backend/distributed/utils/connection_cache.c b/src/backend/distributed/utils/connection_cache.c index 8f852fdd4..efdaf46c9 100644 --- a/src/backend/distributed/utils/connection_cache.c +++ b/src/backend/distributed/utils/connection_cache.c @@ -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 diff --git a/src/backend/distributed/utils/connection_management.c b/src/backend/distributed/utils/connection_management.c new file mode 100644 index 000000000..e4828021f --- /dev/null +++ b/src/backend/distributed/utils/connection_management.c @@ -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 +#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; +} diff --git a/src/backend/distributed/utils/hash_helpers.c b/src/backend/distributed/utils/hash_helpers.c new file mode 100644 index 000000000..e74616b37 --- /dev/null +++ b/src/backend/distributed/utils/hash_helpers.c @@ -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); + } +} diff --git a/src/backend/distributed/utils/placement_connection.c b/src/backend/distributed/utils/placement_connection.c new file mode 100644 index 000000000..3ef16a92b --- /dev/null +++ b/src/backend/distributed/utils/placement_connection.c @@ -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); +} diff --git a/src/backend/distributed/utils/remote_commands.c b/src/backend/distributed/utils/remote_commands.c new file mode 100644 index 000000000..a94f7468f --- /dev/null +++ b/src/backend/distributed/utils/remote_commands.c @@ -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); + } + } +} diff --git a/src/include/distributed/commit_protocol.h b/src/include/distributed/commit_protocol.h deleted file mode 100644 index df590bd48..000000000 --- a/src/include/distributed/commit_protocol.h +++ /dev/null @@ -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 */ diff --git a/src/include/distributed/connection_cache.h b/src/include/distributed/connection_cache.h index 54fb97a22..3d9fa1260 100644 --- a/src/include/distributed/connection_cache.h +++ b/src/include/distributed/connection_cache.h @@ -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); diff --git a/src/include/distributed/connection_management.h b/src/include/distributed/connection_management.h new file mode 100644 index 000000000..0eba1b1e2 --- /dev/null +++ b/src/include/distributed/connection_management.h @@ -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 */ diff --git a/src/include/distributed/hash_helpers.h b/src/include/distributed/hash_helpers.h new file mode 100644 index 000000000..0fecf0c03 --- /dev/null +++ b/src/include/distributed/hash_helpers.h @@ -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 diff --git a/src/include/distributed/master_protocol.h b/src/include/distributed/master_protocol.h index 18c5451a4..4e19a2556 100644 --- a/src/include/distributed/master_protocol.h +++ b/src/include/distributed/master_protocol.h @@ -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');" diff --git a/src/include/distributed/multi_router_executor.h b/src/include/distributed/multi_router_executor.h index 236222f5c..14c4af8e4 100644 --- a/src/include/distributed/multi_router_executor.h +++ b/src/include/distributed/multi_router_executor.h @@ -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_ */ diff --git a/src/include/distributed/multi_shard_transaction.h b/src/include/distributed/multi_shard_transaction.h deleted file mode 100644 index 4b5ec354a..000000000 --- a/src/include/distributed/multi_shard_transaction.h +++ /dev/null @@ -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 */ diff --git a/src/include/distributed/placement_connection.h b/src/include/distributed/placement_connection.h new file mode 100644 index 000000000..eeb47b31f --- /dev/null +++ b/src/include/distributed/placement_connection.h @@ -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 */ diff --git a/src/include/distributed/remote_commands.h b/src/include/distributed/remote_commands.h new file mode 100644 index 000000000..1c9ea009d --- /dev/null +++ b/src/include/distributed/remote_commands.h @@ -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 */ diff --git a/src/include/distributed/remote_transaction.h b/src/include/distributed/remote_transaction.h new file mode 100644 index 000000000..e16e0d898 --- /dev/null +++ b/src/include/distributed/remote_transaction.h @@ -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 */ diff --git a/src/include/distributed/transaction_management.h b/src/include/distributed/transaction_management.h new file mode 100644 index 000000000..14d5cf970 --- /dev/null +++ b/src/include/distributed/transaction_management.h @@ -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 */ diff --git a/src/test/regress/expected/multi_index_statements.out b/src/test/regress/expected/multi_index_statements.out index 639d18ad0..1144d52fa 100644 --- a/src/test/regress/expected/multi_index_statements.out +++ b/src/test/regress/expected/multi_index_statements.out @@ -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 diff --git a/src/test/regress/expected/multi_join_order_additional.out b/src/test/regress/expected/multi_join_order_additional.out index 7254b56ba..7cebe1ec9 100644 --- a/src/test/regress/expected/multi_join_order_additional.out +++ b/src/test/regress/expected/multi_join_order_additional.out @@ -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, diff --git a/src/test/regress/expected/multi_modifying_xacts.out b/src/test/regress/expected/multi_modifying_xacts.out index b8b83a5e9..7f6f96cf5 100644 --- a/src/test/regress/expected/multi_modifying_xacts.out +++ b/src/test/regress/expected/multi_modifying_xacts.out @@ -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 ); diff --git a/src/test/regress/expected/multi_shard_modify.out b/src/test/regress/expected/multi_shard_modify.out index d722c3ad3..23a781f16 100644 --- a/src/test/regress/expected/multi_shard_modify.out +++ b/src/test/regress/expected/multi_shard_modify.out @@ -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 diff --git a/src/test/regress/expected/multi_table_ddl.out b/src/test/regress/expected/multi_table_ddl.out index b4d2c3704..a12ea20a6 100644 --- a/src/test/regress/expected/multi_table_ddl.out +++ b/src/test/regress/expected/multi_table_ddl.out @@ -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'); diff --git a/src/test/regress/expected/multi_truncate.out b/src/test/regress/expected/multi_truncate.out index 09fb65e06..795d5e74b 100644 --- a/src/test/regress/expected/multi_truncate.out +++ b/src/test/regress/expected/multi_truncate.out @@ -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 diff --git a/src/test/regress/input/multi_alter_table_statements.source b/src/test/regress/input/multi_alter_table_statements.source index 614327b0c..f14ee756b 100644 --- a/src/test/regress/input/multi_alter_table_statements.source +++ b/src/test/regress/input/multi_alter_table_statements.source @@ -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 diff --git a/src/test/regress/output/multi_alter_table_statements.source b/src/test/regress/output/multi_alter_table_statements.source index 2e74da86d..cacf6fef0 100644 --- a/src/test/regress/output/multi_alter_table_statements.source +++ b/src/test/regress/output/multi_alter_table_statements.source @@ -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 diff --git a/src/test/regress/output/multi_copy.source b/src/test/regress/output/multi_copy.source index 64880627b..6a01452cb 100644 --- a/src/test/regress/output/multi_copy.source +++ b/src/test/regress/output/multi_copy.source @@ -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 diff --git a/src/test/regress/sql/multi_modifying_xacts.sql b/src/test/regress/sql/multi_modifying_xacts.sql index 7c3395705..479ff8ca5 100644 --- a/src/test/regress/sql/multi_modifying_xacts.sql +++ b/src/test/regress/sql/multi_modifying_xacts.sql @@ -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; diff --git a/src/test/regress/sql/multi_table_ddl.sql b/src/test/regress/sql/multi_table_ddl.sql index 8cb2ddbf3..1aaf894d8 100644 --- a/src/test/regress/sql/multi_table_ddl.sql +++ b/src/test/regress/sql/multi_table_ddl.sql @@ -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); diff --git a/src/test/regress/sql/multi_truncate.sql b/src/test/regress/sql/multi_truncate.sql index 4abc9f27e..d0fdf3e9d 100644 --- a/src/test/regress/sql/multi_truncate.sql +++ b/src/test/regress/sql/multi_truncate.sql @@ -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"; \ No newline at end of file +DROP TABLE "a b append";