From 61eb3e4c5ee9180ec928e2884153aedd9367ea94 Mon Sep 17 00:00:00 2001 From: Andres Freund Date: Fri, 7 Oct 2016 17:35:20 -0700 Subject: [PATCH 01/14] Add some basic helpers to make use of dynahash hashtables easier. --- src/backend/distributed/utils/hash_helpers.c | 34 ++++++++++++++++++++ src/include/distributed/hash_helpers.h | 30 +++++++++++++++++ 2 files changed, 64 insertions(+) create mode 100644 src/backend/distributed/utils/hash_helpers.c create mode 100644 src/include/distributed/hash_helpers.h 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/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 From 90899e321050ef566673dd37ab73a894c025cc8d Mon Sep 17 00:00:00 2001 From: Andres Freund Date: Fri, 7 Oct 2016 14:48:25 -0700 Subject: [PATCH 02/14] Add beginning of new and central connection lifetime management. This is intended to eventually replace multi_client_executor.c and connection_cache.c, and to provide the basis of a centralized transaction management. The newly introduced transaction hook should, in the future, be the only one in citus, to allow for proper ordering between operations. For now this central handler is responsible for releasing connections and resetting XactModificationLevel after a transaction. --- .../executor/multi_client_executor.c | 1 + .../executor/multi_router_executor.c | 3 +- src/backend/distributed/shared_library_init.c | 7 + .../distributed/transaction/commit_protocol.c | 4 - .../transaction/multi_shard_transaction.c | 1 - .../transaction/transaction_management.c | 142 +++++ .../distributed/utils/connection_cache.c | 35 -- .../distributed/utils/connection_management.c | 541 ++++++++++++++++++ src/include/distributed/commit_protocol.h | 12 +- src/include/distributed/connection_cache.h | 15 - .../distributed/connection_management.h | 133 +++++ .../distributed/transaction_management.h | 59 ++ 12 files changed, 886 insertions(+), 67 deletions(-) create mode 100644 src/backend/distributed/transaction/transaction_management.c create mode 100644 src/backend/distributed/utils/connection_management.c create mode 100644 src/include/distributed/connection_management.h create mode 100644 src/include/distributed/transaction_management.h 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..caf4eeb2a 100644 --- a/src/backend/distributed/executor/multi_router_executor.c +++ b/src/backend/distributed/executor/multi_router_executor.c @@ -29,6 +29,7 @@ #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" @@ -36,6 +37,7 @@ #include "distributed/multi_planner.h" #include "distributed/multi_router_executor.h" #include "distributed/relay_utility.h" +#include "distributed/remote_commands.h" #include "distributed/resource_lock.h" #include "executor/execdesc.h" #include "executor/executor.h" @@ -1272,7 +1274,6 @@ RouterTransactionCallback(XactEvent event, void *arg) } /* reset transaction state */ - XactModificationLevel = XACT_MODIFICATION_NONE; xactParticipantHash = NULL; xactShardConnSetList = NIL; subXactAbortAttempted = false; diff --git a/src/backend/distributed/shared_library_init.c b/src/backend/distributed/shared_library_init.c index 549d20958..f2e75291a 100644 --- a/src/backend/distributed/shared_library_init.c +++ b/src/backend/distributed/shared_library_init.c @@ -19,6 +19,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" @@ -31,7 +32,9 @@ #include "distributed/multi_server_executor.h" #include "distributed/multi_shard_transaction.h" #include "distributed/multi_utility.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,6 +153,10 @@ _PG_init(void) /* organize that task tracker is started once server is up */ TaskTrackerRegister(); + /* initialize coordinated transaction management */ + InitializeTransactionManagement(); + InitializeConnectionManagement(); + /* initialize transaction callbacks */ RegisterRouterExecutorXactCallbacks(); RegisterShardPlacementXactCallbacks(); diff --git a/src/backend/distributed/transaction/commit_protocol.c b/src/backend/distributed/transaction/commit_protocol.c index 728f672f4..329512749 100644 --- a/src/backend/distributed/transaction/commit_protocol.c +++ b/src/backend/distributed/transaction/commit_protocol.c @@ -30,10 +30,6 @@ static uint32 DistributedTransactionId = 0; 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. diff --git a/src/backend/distributed/transaction/multi_shard_transaction.c b/src/backend/distributed/transaction/multi_shard_transaction.c index 712beaa5e..cfbb44ec3 100644 --- a/src/backend/distributed/transaction/multi_shard_transaction.c +++ b/src/backend/distributed/transaction/multi_shard_transaction.c @@ -307,7 +307,6 @@ CompleteShardPlacementTransactions(XactEvent event, void *arg) CloseConnections(connectionList); shardConnectionHash = NULL; - XactModificationLevel = XACT_MODIFICATION_NONE; subXactAbortAttempted = false; } diff --git a/src/backend/distributed/transaction/transaction_management.c b/src/backend/distributed/transaction/transaction_management.c new file mode 100644 index 000000000..e03f1476b --- /dev/null +++ b/src/backend/distributed/transaction/transaction_management.c @@ -0,0 +1,142 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + * 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 "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); + + +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: + { + /* close connections etc. */ + if (CurrentCoordinatedTransactionState != COORD_TRANS_NONE) + { + AtEOXact_Connections(true); + } + + Assert(!subXactAbortAttempted); + CurrentCoordinatedTransactionState = COORD_TRANS_NONE; + XactModificationLevel = XACT_MODIFICATION_NONE; + } + break; + + case XACT_EVENT_ABORT: + { + /* close connections etc. */ + if (CurrentCoordinatedTransactionState != COORD_TRANS_NONE) + { + 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: + { + if (subXactAbortAttempted) + { + subXactAbortAttempted = false; + + ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("cannot ROLLBACK TO SAVEPOINT in transactions " + "which modify distributed tables"))); + } + } + 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..462cc5061 --- /dev/null +++ b/src/backend/distributed/utils/connection_management.c @@ -0,0 +1,541 @@ +/*------------------------------------------------------------------------- + * + * 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; + + 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/include/distributed/commit_protocol.h b/src/include/distributed/commit_protocol.h index df590bd48..43fbb6328 100644 --- a/src/include/distributed/commit_protocol.h +++ b/src/include/distributed/commit_protocol.h @@ -14,18 +14,12 @@ #include "access/xact.h" +#include "distributed/connection_management.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 { @@ -48,10 +42,6 @@ typedef struct TransactionConnection } 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); 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..cc22b7794 --- /dev/null +++ b/src/include/distributed/connection_management.h @@ -0,0 +1,133 @@ +/*------------------------------------------------------------------------- + * + * 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 "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; +} 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/transaction_management.h b/src/include/distributed/transaction_management.h new file mode 100644 index 000000000..77f4cfa37 --- /dev/null +++ b/src/include/distributed/transaction_management.h @@ -0,0 +1,59 @@ +/*------------------------------------------------------------------------- + * 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 +} 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; + + +/* + * Initialization. + */ +extern void InitializeTransactionManagement(void); + + +#endif /* TRANSACTION_MANAGMENT_H */ From 3929e322b531fbeed3854d5d57735458ffb3a8e4 Mon Sep 17 00:00:00 2001 From: Andres Freund Date: Fri, 7 Oct 2016 17:39:11 -0700 Subject: [PATCH 03/14] Add initial helpers to make interactions with MultiConnection et al. easier. This includes basic infrastructure for logging of commands sent to remote/worker nodes. Note that this has no effect as of yet, since no callers are converted to the new infrastructure. --- src/backend/distributed/shared_library_init.c | 10 + .../distributed/utils/remote_commands.c | 195 ++++++++++++++++++ src/include/distributed/remote_commands.h | 36 ++++ 3 files changed, 241 insertions(+) create mode 100644 src/backend/distributed/utils/remote_commands.c create mode 100644 src/include/distributed/remote_commands.h diff --git a/src/backend/distributed/shared_library_init.c b/src/backend/distributed/shared_library_init.c index f2e75291a..6be2ba898 100644 --- a/src/backend/distributed/shared_library_init.c +++ b/src/backend/distributed/shared_library_init.c @@ -266,6 +266,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/utils/remote_commands.c b/src/backend/distributed/utils/remote_commands.c new file mode 100644 index 000000000..cdafaa161 --- /dev/null +++ b/src/backend/distributed/utils/remote_commands.c @@ -0,0 +1,195 @@ +/*------------------------------------------------------------------------- + * + * 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/remote_commands.h" + + +/* GUC, determining whether statements sent to remote nodes are logged */ +bool LogRemoteCommands = false; + + +/* 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); +} diff --git a/src/include/distributed/remote_commands.h b/src/include/distributed/remote_commands.h new file mode 100644 index 000000000..6ce25ccfd --- /dev/null +++ b/src/include/distributed/remote_commands.h @@ -0,0 +1,36 @@ +/*------------------------------------------------------------------------- + * + * 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" + + +/* 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); + + +#endif /* REMOTE_COMMAND_H */ From 6ffa0976cfe064d6e894363cdca162c7df496a47 Mon Sep 17 00:00:00 2001 From: Andres Freund Date: Fri, 7 Oct 2016 15:26:26 -0700 Subject: [PATCH 04/14] Coordinated remote transaction management. --- .../transaction/remote_transaction.c | 546 ++++++++++++++++++ .../transaction/transaction_management.c | 86 +++ .../distributed/utils/connection_management.c | 2 + .../distributed/connection_management.h | 4 + src/include/distributed/remote_transaction.h | 78 +++ .../distributed/transaction_management.h | 18 +- 6 files changed, 733 insertions(+), 1 deletion(-) create mode 100644 src/backend/distributed/transaction/remote_transaction.c create mode 100644 src/include/distributed/remote_transaction.h 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 index e03f1476b..269aebd57 100644 --- a/src/backend/distributed/transaction/transaction_management.c +++ b/src/backend/distributed/transaction/transaction_management.c @@ -42,6 +42,51 @@ static void CoordinatedSubTransactionCallback(SubXactEvent event, SubTransaction 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) { @@ -66,6 +111,12 @@ CoordinatedTransactionCallback(XactEvent event, void *arg) { 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) { @@ -80,6 +131,18 @@ CoordinatedTransactionCallback(XactEvent event, void *arg) 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) { @@ -108,6 +171,29 @@ CoordinatedTransactionCallback(XactEvent event, void *arg) 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; + } + + + 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; + } } break; diff --git a/src/backend/distributed/utils/connection_management.c b/src/backend/distributed/utils/connection_management.c index 462cc5061..e4828021f 100644 --- a/src/backend/distributed/utils/connection_management.c +++ b/src/backend/distributed/utils/connection_management.c @@ -138,6 +138,8 @@ AtEOXact_Connections(bool isCommit) { /* reset per-transaction state */ connection->activeInTransaction = false; + connection->remoteTransaction.transactionState = REMOTE_TRANS_INVALID; + connection->remoteTransaction.transactionFailed = false; UnclaimConnection(connection); diff --git a/src/include/distributed/connection_management.h b/src/include/distributed/connection_management.h index cc22b7794..0eba1b1e2 100644 --- a/src/include/distributed/connection_management.h +++ b/src/include/distributed/connection_management.h @@ -12,6 +12,7 @@ #define CONNECTION_MANAGMENT_H #include "distributed/transaction_management.h" +#include "distributed/remote_transaction.h" #include "nodes/pg_list.h" #include "utils/hsearch.h" @@ -71,6 +72,9 @@ typedef struct MultiConnection /* has the connection been used in the current coordinated transaction? */ bool activeInTransaction; + + /* information about the associated remote transaction */ + RemoteTransaction remoteTransaction; } MultiConnection; 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 index 77f4cfa37..14d5cf970 100644 --- a/src/include/distributed/transaction_management.h +++ b/src/include/distributed/transaction_management.h @@ -29,7 +29,16 @@ typedef enum CoordinatedTransactionState COORD_TRANS_NONE, /* no coordinated transaction in progress, but connections established */ - COORD_TRANS_IDLE + 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; @@ -50,6 +59,13 @@ extern XactModificationType XactModificationLevel; extern CoordinatedTransactionState CurrentCoordinatedTransactionState; +/* + * Coordinated transaction management. + */ +extern void BeginCoordinatedTransaction(void); +extern void BeginOrContinueCoordinatedTransaction(void); +extern bool InCoordinatedTransaction(void); + /* * Initialization. */ From 14f73e78b09f25fe42c88b41d99b62803fc3191a Mon Sep 17 00:00:00 2001 From: Andres Freund Date: Fri, 7 Oct 2016 15:29:47 -0700 Subject: [PATCH 05/14] Introduce association between placements and connections. --- src/backend/distributed/shared_library_init.c | 2 + .../transaction/transaction_management.c | 27 ++ .../distributed/utils/placement_connection.c | 361 ++++++++++++++++++ .../distributed/placement_connection.h | 97 +++++ 4 files changed, 487 insertions(+) create mode 100644 src/backend/distributed/utils/placement_connection.c create mode 100644 src/include/distributed/placement_connection.h diff --git a/src/backend/distributed/shared_library_init.c b/src/backend/distributed/shared_library_init.c index 6be2ba898..48a634c88 100644 --- a/src/backend/distributed/shared_library_init.c +++ b/src/backend/distributed/shared_library_init.c @@ -32,6 +32,7 @@ #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" @@ -156,6 +157,7 @@ _PG_init(void) /* initialize coordinated transaction management */ InitializeTransactionManagement(); InitializeConnectionManagement(); + InitPlacementConnectionManagement(); /* initialize transaction callbacks */ RegisterRouterExecutorXactCallbacks(); diff --git a/src/backend/distributed/transaction/transaction_management.c b/src/backend/distributed/transaction/transaction_management.c index 269aebd57..36d030fa3 100644 --- a/src/backend/distributed/transaction/transaction_management.c +++ b/src/backend/distributed/transaction/transaction_management.c @@ -21,6 +21,7 @@ #include "distributed/connection_management.h" #include "distributed/hash_helpers.h" #include "distributed/transaction_management.h" +#include "distributed/placement_connection.h" #include "utils/hsearch.h" @@ -120,6 +121,7 @@ CoordinatedTransactionCallback(XactEvent event, void *arg) /* close connections etc. */ if (CurrentCoordinatedTransactionState != COORD_TRANS_NONE) { + ResetPlacementConnectionManagement(); AtEOXact_Connections(true); } @@ -146,6 +148,7 @@ CoordinatedTransactionCallback(XactEvent event, void *arg) /* close connections etc. */ if (CurrentCoordinatedTransactionState != COORD_TRANS_NONE) { + ResetPlacementConnectionManagement(); AtEOXact_Connections(false); } @@ -163,6 +166,8 @@ CoordinatedTransactionCallback(XactEvent event, void *arg) case XACT_EVENT_PRE_COMMIT: { + bool using2PC = MultiShardCommitProtocol == COMMIT_PROTOCOL_2PC; + if (subXactAbortAttempted) { subXactAbortAttempted = false; @@ -178,6 +183,22 @@ CoordinatedTransactionCallback(XactEvent event, void *arg) 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) { @@ -194,6 +215,12 @@ CoordinatedTransactionCallback(XactEvent event, void *arg) CoordinatedRemoteTransactionsCommit(); CurrentCoordinatedTransactionState = COORD_TRANS_COMMITTED; } + + /* + * Check again whether shards/placement successfully + * committed. This handles failure at COMMIT/PREPARE time. + */ + CheckForFailedPlacements(false, using2PC); } break; 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/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 */ From 81d75901ad3862be9f7e80d989a22db2ab9d7048 Mon Sep 17 00:00:00 2001 From: Andres Freund Date: Fri, 7 Oct 2016 15:31:59 -0700 Subject: [PATCH 06/14] Add additional remote command helpers. --- .../distributed/utils/remote_commands.c | 508 ++++++++++++++++++ src/include/distributed/remote_commands.h | 43 ++ 2 files changed, 551 insertions(+) diff --git a/src/backend/distributed/utils/remote_commands.c b/src/backend/distributed/utils/remote_commands.c index cdafaa161..a94f7468f 100644 --- a/src/backend/distributed/utils/remote_commands.c +++ b/src/backend/distributed/utils/remote_commands.c @@ -13,13 +13,26 @@ #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 */ /* @@ -193,3 +206,498 @@ 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/remote_commands.h b/src/include/distributed/remote_commands.h index 6ce25ccfd..1c9ea009d 100644 --- a/src/include/distributed/remote_commands.h +++ b/src/include/distributed/remote_commands.h @@ -12,6 +12,27 @@ #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 */ @@ -33,4 +54,26 @@ extern void LogRemoteCommand(MultiConnection *connection, const char *command); 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 */ From 57f2b7c452f27d70df1195d89c802eeebd96a154 Mon Sep 17 00:00:00 2001 From: Andres Freund Date: Sun, 18 Sep 2016 18:31:25 -0700 Subject: [PATCH 07/14] Convert master_modify_multiple_shards() to new API. --- .../master/master_modify_multiple_shards.c | 132 +----------------- .../regress/expected/multi_shard_modify.out | 4 - 2 files changed, 6 insertions(+), 130 deletions(-) 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/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 From 8d42d18c81e320b4f007d500583ea5abf0cb8029 Mon Sep 17 00:00:00 2001 From: Andres Freund Date: Sun, 18 Sep 2016 19:23:06 -0700 Subject: [PATCH 08/14] Parallelize and transactionalize DDL. --- .../distributed/executor/multi_utility.c | 185 +++--------------- .../distributed/master/master_create_shards.c | 1 - .../expected/multi_index_statements.out | 8 +- .../expected/multi_join_order_additional.out | 4 - .../input/multi_alter_table_statements.source | 7 +- .../multi_alter_table_statements.source | 35 ++-- 6 files changed, 55 insertions(+), 185 deletions(-) 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/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/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 From 0850204bc58b4fa3c089a0c9afba7045aa22d45d Mon Sep 17 00:00:00 2001 From: Andres Freund Date: Fri, 2 Sep 2016 16:27:19 -0700 Subject: [PATCH 09/14] Move DropShards() and master_drop_sequences() to new framework. This includes parallelizing DropShards, and always using transactions, which should greatly increase DROP TABLE performance for tables with a lot of shards. --- .../master/master_delete_protocol.c | 89 ++++++++----------- src/test/regress/expected/multi_table_ddl.out | 21 +++-- src/test/regress/expected/multi_truncate.out | 5 +- src/test/regress/sql/multi_table_ddl.sql | 11 +-- src/test/regress/sql/multi_truncate.sql | 4 +- 5 files changed, 60 insertions(+), 70 deletions(-) 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/test/regress/expected/multi_table_ddl.out b/src/test/regress/expected/multi_table_ddl.out index 7da6ce247..2c4dd6ddd 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/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"; From 8a6a8fae9b4e767d6869d7c4b9574812892ee178 Mon Sep 17 00:00:00 2001 From: Andres Freund Date: Wed, 7 Sep 2016 16:45:36 -0700 Subject: [PATCH 10/14] Move WorkerShardStats() to new transaction & connection framework. That's a prerequisite for transactional COPY. Otherwise, for append partitioned tables, the WorkerShardStats() triggered during COPY can't see data. Which, in turn, prevents proper shard pruning. --- .../master/master_stage_protocol.c | 60 +++++++++---------- 1 file changed, 29 insertions(+), 31 deletions(-) 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; } From 360307f6d5cf751e556bb9ad5aee813fbe2b282c Mon Sep 17 00:00:00 2001 From: Andres Freund Date: Sun, 18 Sep 2016 19:33:07 -0700 Subject: [PATCH 11/14] Move router executor over to new connection & transaction framework. Note that this changes behaviour in a few edgecases (as demonstrated by the tests). There's also a few FIXMEs. --- .../executor/multi_router_executor.c | 584 ++---------------- src/backend/distributed/shared_library_init.c | 1 - .../distributed/multi_router_executor.h | 13 - .../expected/multi_modifying_xacts.out | 42 +- .../regress/sql/multi_modifying_xacts.sql | 20 +- 5 files changed, 56 insertions(+), 604 deletions(-) diff --git a/src/backend/distributed/executor/multi_router_executor.c b/src/backend/distributed/executor/multi_router_executor.c index caf4eeb2a..0f1462032 100644 --- a/src/backend/distributed/executor/multi_router_executor.c +++ b/src/backend/distributed/executor/multi_router_executor.c @@ -28,7 +28,6 @@ #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" @@ -36,9 +35,11 @@ #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" @@ -66,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, @@ -98,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); /* @@ -154,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(); } } @@ -186,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 @@ -311,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. */ @@ -422,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 */ @@ -489,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) { @@ -506,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. @@ -515,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) { @@ -527,7 +432,6 @@ ExecuteTaskAndStoreResults(QueryDesc *queryDesc, Task *task, queryOK = SendQueryInSingleRowMode(connection, queryString, paramListInfo); if (!queryOK) { - PurgeConnectionForPlacement(taskPlacement); failedPlacementList = lappend(failedPlacementList, taskPlacement); continue; } @@ -584,8 +488,6 @@ ExecuteTaskAndStoreResults(QueryDesc *queryDesc, Task *task, } else { - PurgeConnectionForPlacement(taskPlacement); - failedPlacementList = lappend(failedPlacementList, taskPlacement); continue; @@ -662,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; @@ -814,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; } @@ -904,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); @@ -939,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; @@ -962,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); @@ -1032,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; @@ -1046,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) @@ -1074,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); @@ -1117,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. */ @@ -1195,221 +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 */ - 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/shared_library_init.c b/src/backend/distributed/shared_library_init.c index 48a634c88..cde09697e 100644 --- a/src/backend/distributed/shared_library_init.c +++ b/src/backend/distributed/shared_library_init.c @@ -160,7 +160,6 @@ _PG_init(void) InitPlacementConnectionManagement(); /* initialize transaction callbacks */ - RegisterRouterExecutorXactCallbacks(); RegisterShardPlacementXactCallbacks(); } 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/test/regress/expected/multi_modifying_xacts.out b/src/test/regress/expected/multi_modifying_xacts.out index b8b83a5e9..e3ae86ee0 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,34 +158,8 @@ 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, BEGIN; INSERT INTO labs VALUES (6, 'Bell Labs'); @@ -204,6 +177,7 @@ SELECT name FROM labs WHERE id = 10; (1 row) INSERT INTO labs VALUES (6, 'Bell Labs'); +ERROR: cannot open new connections after the first modification command within a transaction COMMIT; -- but a double-copy isn't allowed (the first will persist) BEGIN; @@ -220,7 +194,7 @@ SELECT name FROM labs WHERE id = 11; -- finally, ALTER and copy aren't 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" @@ -241,7 +215,7 @@ SELECT * FROM labs WHERE id = 12; -- and if the copy is before the ALTER... BEGIN; \copy labs from stdin delimiter ',' -ALTER TABLE labs ADD COLUMN motto text; +ALTER TABLE labs ADD COLUMN motto3 text; ERROR: distributed DDL commands must not appear within transaction blocks containing data modifications COMMIT; -- the DDL fails, but copy persists @@ -512,6 +486,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 +511,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/sql/multi_modifying_xacts.sql b/src/test/regress/sql/multi_modifying_xacts.sql index 7c3395705..1cd72b6ae 100644 --- a/src/test/regress/sql/multi_modifying_xacts.sql +++ b/src/test/regress/sql/multi_modifying_xacts.sql @@ -131,22 +131,6 @@ 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, BEGIN; INSERT INTO labs VALUES (6, 'Bell Labs'); @@ -178,7 +162,7 @@ SELECT name FROM labs WHERE id = 11; -- finally, ALTER and copy aren't compatible BEGIN; -ALTER TABLE labs ADD COLUMN motto text; +ALTER TABLE labs ADD COLUMN motto2 text; \copy labs from stdin delimiter ',' 12,fsociety,lol \. @@ -193,7 +177,7 @@ 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 From a8f9e983a08f1c223e17a1bb68461480e5f02b40 Mon Sep 17 00:00:00 2001 From: Andres Freund Date: Sun, 18 Sep 2016 19:45:25 -0700 Subject: [PATCH 12/14] Migration COPY to new framework. This implies several behaviour changes: - COPY is now transactional - failure to compute stats for append partitioned tables is an error --- src/backend/distributed/commands/multi_copy.c | 834 +++++++++--------- src/include/distributed/master_protocol.h | 2 +- .../expected/multi_modifying_xacts.out | 62 +- src/test/regress/output/multi_copy.source | 6 +- .../regress/sql/multi_modifying_xacts.sql | 11 +- 5 files changed, 431 insertions(+), 484 deletions(-) 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/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/test/regress/expected/multi_modifying_xacts.out b/src/test/regress/expected/multi_modifying_xacts.out index e3ae86ee0..7f6f96cf5 100644 --- a/src/test/regress/expected/multi_modifying_xacts.out +++ b/src/test/regress/expected/multi_modifying_xacts.out @@ -160,31 +160,27 @@ INSERT INTO labs VALUES (6, 'Bell Labs'); SELECT count(*) FROM researchers WHERE lab_id = 6; ERROR: cannot open new connections after the first modification command within a transaction ABORT; --- 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'); -ERROR: cannot open new connections after the first modification command within a transaction 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 @@ -192,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 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" @@ -207,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 motto3 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 ( 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 1cd72b6ae..479ff8ca5 100644 --- a/src/test/regress/sql/multi_modifying_xacts.sql +++ b/src/test/regress/sql/multi_modifying_xacts.sql @@ -131,7 +131,7 @@ INSERT INTO labs VALUES (6, 'Bell Labs'); SELECT count(*) FROM researchers WHERE lab_id = 6; ABORT; --- COPY can't happen second, +-- Check COPY can happen after INSERT BEGIN; INSERT INTO labs VALUES (6, 'Bell Labs'); \copy labs from stdin delimiter ',' @@ -139,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 @@ -148,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 @@ -160,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 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 @@ -180,7 +181,7 @@ BEGIN; 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; From 3b9826e6e972d7f0afeec9698783792fd9d3f8ec Mon Sep 17 00:00:00 2001 From: Andres Freund Date: Fri, 7 Oct 2016 17:09:55 -0700 Subject: [PATCH 13/14] Remove obsoleted transaction infrastructure. --- src/backend/distributed/shared_library_init.c | 5 - .../distributed/transaction/commit_protocol.c | 260 ------------- .../transaction/multi_shard_transaction.c | 341 ------------------ src/include/distributed/commit_protocol.h | 51 --- .../distributed/multi_shard_transaction.h | 39 -- 5 files changed, 696 deletions(-) delete mode 100644 src/backend/distributed/transaction/commit_protocol.c delete mode 100644 src/backend/distributed/transaction/multi_shard_transaction.c delete mode 100644 src/include/distributed/commit_protocol.h delete mode 100644 src/include/distributed/multi_shard_transaction.h diff --git a/src/backend/distributed/shared_library_init.c b/src/backend/distributed/shared_library_init.c index cde09697e..e39a4b039 100644 --- a/src/backend/distributed/shared_library_init.c +++ b/src/backend/distributed/shared_library_init.c @@ -18,7 +18,6 @@ #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" @@ -30,7 +29,6 @@ #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" @@ -158,9 +156,6 @@ _PG_init(void) InitializeTransactionManagement(); InitializeConnectionManagement(); InitPlacementConnectionManagement(); - - /* initialize transaction callbacks */ - RegisterShardPlacementXactCallbacks(); } diff --git a/src/backend/distributed/transaction/commit_protocol.c b/src/backend/distributed/transaction/commit_protocol.c deleted file mode 100644 index 329512749..000000000 --- a/src/backend/distributed/transaction/commit_protocol.c +++ /dev/null @@ -1,260 +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); - - -/* - * 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 cfbb44ec3..000000000 --- a/src/backend/distributed/transaction/multi_shard_transaction.c +++ /dev/null @@ -1,341 +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; - 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/include/distributed/commit_protocol.h b/src/include/distributed/commit_protocol.h deleted file mode 100644 index 43fbb6328..000000000 --- a/src/include/distributed/commit_protocol.h +++ /dev/null @@ -1,51 +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 "distributed/connection_management.h" -#include "libpq-fe.h" -#include "lib/stringinfo.h" -#include "nodes/pg_list.h" - - -/* 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; - - -/* 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/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 */ From cba340ebe9a419e6d112bf84b584fd5350b425a6 Mon Sep 17 00:00:00 2001 From: Andres Freund Date: Fri, 7 Oct 2016 19:20:35 -0700 Subject: [PATCH 14/14] Add overview instructions to transaction_management.c's header. --- .../transaction/transaction_management.c | 37 +++++++++++++++++++ 1 file changed, 37 insertions(+) diff --git a/src/backend/distributed/transaction/transaction_management.c b/src/backend/distributed/transaction/transaction_management.c index 36d030fa3..9dfd29ec8 100644 --- a/src/backend/distributed/transaction/transaction_management.c +++ b/src/backend/distributed/transaction/transaction_management.c @@ -6,6 +6,43 @@ * 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. * *-------------------------------------------------------------------------