diff options
author | Pavan Deolasee | 2011-03-08 11:15:12 +0000 |
---|---|---|
committer | Pavan Deolasee | 2011-06-17 07:42:24 +0000 |
commit | 246072c6301bf3e38331ee49e4ff9bd4bd42b9a4 (patch) | |
tree | 780ed90ae933dec23ccfb94a82689910e2e80052 | |
parent | 88a19b42f3b599927b74e0aef2e19b9161b3a7eb (diff) |
First cut implementation of BARRIER for PITR and global consistent recovery
-rw-r--r-- | src/backend/access/transam/rmgr.c | 8 | ||||
-rw-r--r-- | src/backend/access/transam/xlog.c | 42 | ||||
-rw-r--r-- | src/backend/nodes/copyfuncs.c | 17 | ||||
-rw-r--r-- | src/backend/nodes/equalfuncs.c | 15 | ||||
-rw-r--r-- | src/backend/parser/gram.y | 34 | ||||
-rw-r--r-- | src/backend/pgxc/Makefile | 2 | ||||
-rw-r--r-- | src/backend/pgxc/barrier/Makefile | 19 | ||||
-rw-r--r-- | src/backend/pgxc/barrier/barrier.c | 493 | ||||
-rw-r--r-- | src/backend/pgxc/pool/execRemote.c | 11 | ||||
-rw-r--r-- | src/backend/tcop/postgres.c | 33 | ||||
-rw-r--r-- | src/backend/tcop/utility.c | 13 | ||||
-rw-r--r-- | src/include/access/rmgr.h | 5 | ||||
-rw-r--r-- | src/include/nodes/nodes.h | 3 | ||||
-rw-r--r-- | src/include/nodes/parsenodes.h | 13 | ||||
-rw-r--r-- | src/include/parser/kwlist.h | 3 | ||||
-rw-r--r-- | src/include/pgxc/barrier.h | 40 | ||||
-rw-r--r-- | src/include/pgxc/execRemote.h | 1 | ||||
-rw-r--r-- | src/include/pgxc/pgxcnode.h | 1 | ||||
-rw-r--r-- | src/include/storage/lwlock.h | 1 |
19 files changed, 749 insertions, 5 deletions
diff --git a/src/backend/access/transam/rmgr.c b/src/backend/access/transam/rmgr.c index 8038b25d1d..d989a59197 100644 --- a/src/backend/access/transam/rmgr.c +++ b/src/backend/access/transam/rmgr.c @@ -20,11 +20,13 @@ #include "commands/dbcommands.h" #include "commands/sequence.h" #include "commands/tablespace.h" +#ifdef PGXC +#include "pgxc/barrier.h" +#endif #include "storage/freespace.h" #include "storage/standby.h" #include "utils/relmapper.h" - const RmgrData RmgrTable[RM_MAX_ID + 1] = { {"XLOG", xlog_redo, xlog_desc, NULL, NULL, NULL}, {"Transaction", xact_redo, xact_desc, NULL, NULL, NULL}, @@ -42,4 +44,8 @@ const RmgrData RmgrTable[RM_MAX_ID + 1] = { {"Gin", gin_redo, gin_desc, gin_xlog_startup, gin_xlog_cleanup, gin_safe_restartpoint}, {"Gist", gist_redo, gist_desc, gist_xlog_startup, gist_xlog_cleanup, gist_safe_restartpoint}, {"Sequence", seq_redo, seq_desc, NULL, NULL, NULL} +#ifdef PGXC + , + {"Barrier", barrier_redo, barrier_desc, NULL, NULL, NULL} +#endif }; diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c index 7fbccc5350..368cd69c46 100644 --- a/src/backend/access/transam/xlog.c +++ b/src/backend/access/transam/xlog.c @@ -39,6 +39,7 @@ #include "funcapi.h" #include "libpq/pqsignal.h" #include "miscadmin.h" +#include "pgxc/barrier.h" #include "pgstat.h" #include "postmaster/bgwriter.h" #include "replication/walreceiver.h" @@ -184,6 +185,7 @@ static RecoveryTargetType recoveryTarget = RECOVERY_TARGET_UNSET; static bool recoveryTargetInclusive = true; static TransactionId recoveryTargetXid; static TimestampTz recoveryTargetTime; +static char *recoveryTargetBarrierId; /* options taken from recovery.conf for XLOG streaming */ static bool StandbyMode = false; @@ -5258,6 +5260,13 @@ readRecoveryCommandFile(void) (errmsg("trigger_file = '%s'", TriggerFile))); } +#ifdef PGXC + else if (strcmp(tok1, "recovery_barrier_id") == 0) + { + recoveryTarget = true; + recoveryTargetBarrierId = pstrdup(tok2); + } +#endif else ereport(FATAL, (errmsg("unrecognized recovery parameter \"%s\"", @@ -5451,11 +5460,20 @@ static bool recoveryStopsHere(XLogRecord *record, bool *includeThis) { bool stopsHere; +#ifdef PGXC + bool stopsAtThisBarrier; + char *recordBarrierId; +#endif uint8 record_info; TimestampTz recordXtime; +#ifdef PGXC + /* We only consider stoppping at COMMIT, ABORT or BARRIER records */ + if ((record->xl_rmid != RM_XACT_ID) && (record->xl_rmid != RM_BARRIER_ID)) +#else /* We only consider stopping at COMMIT or ABORT records */ if (record->xl_rmid != RM_XACT_ID) +#endif return false; record_info = record->xl_info & ~XLR_INFO_MASK; if (record_info == XLOG_XACT_COMMIT) @@ -5472,6 +5490,12 @@ recoveryStopsHere(XLogRecord *record, bool *includeThis) recordXactAbortData = (xl_xact_abort *) XLogRecGetData(record); recordXtime = recordXactAbortData->xact_time; } +#ifdef PGXC + else if (record_info == XLOG_BARRIER_CREATE) + { + recordBarrierId = (char *) XLogRecGetData(record); + } +#endif else return false; @@ -5497,6 +5521,13 @@ recoveryStopsHere(XLogRecord *record, bool *includeThis) if (stopsHere) *includeThis = recoveryTargetInclusive; } +#ifdef PGXC + else if (recoveryTargetBarrierId) + { + if (strcmp(recoveryTargetBarrierId, recordBarrierId) == 0) + stopsAtThisBarrier = true; + } +#endif else { /* @@ -5548,6 +5579,17 @@ recoveryStopsHere(XLogRecord *record, bool *includeThis) if (recoveryStopAfter) SetLatestXTime(recordXtime); } +#ifdef PGXC + else if (stopsAtThisBarrier) + { + recoveryStopTime = recordXtime; + ereport(LOG, + (errmsg("recovery stopping at barrier %s, time %s", + recoveryTargetBarrierId, + timestamptz_to_str(recoveryStopTime)))); + return true; + } +#endif else SetLatestXTime(recordXtime); diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c index 2c7fee4c3c..c9581e19cf 100644 --- a/src/backend/nodes/copyfuncs.c +++ b/src/backend/nodes/copyfuncs.c @@ -3726,6 +3726,18 @@ _copyValue(Value *from) return newnode; } +#ifdef PGXC +static BarrierStmt * +_copyBarrierStmt(BarrierStmt *from) +{ + BarrierStmt *newnode = makeNode(BarrierStmt); + + COPY_STRING_FIELD(id); + + return newnode; +} +#endif + /* * copyObject * @@ -4307,6 +4319,11 @@ copyObject(void *from) case T_CheckPointStmt: retval = (void *) makeNode(CheckPointStmt); break; +#ifdef PGXC + case T_BarrierStmt: + retval = _copyBarrierStmt(from); + break; +#endif case T_CreateSchemaStmt: retval = _copyCreateSchemaStmt(from); break; diff --git a/src/backend/nodes/equalfuncs.c b/src/backend/nodes/equalfuncs.c index c5b46bbbbc..d1dad1ce3f 100644 --- a/src/backend/nodes/equalfuncs.c +++ b/src/backend/nodes/equalfuncs.c @@ -2361,6 +2361,16 @@ _equalValue(Value *a, Value *b) return true; } +#ifdef PGXC + +static bool +_equalBarrierStmt(BarrierStmt *a, BarrierStmt *b) +{ + COMPARE_STRING_FIELD(id); + return true; +} +#endif + /* * equal * returns whether two nodes are equal @@ -2811,6 +2821,11 @@ equal(void *a, void *b) case T_CheckPointStmt: retval = true; break; +#ifdef PGXC + case T_BarrierStmt: + retval = _equalBarrierStmt(a, b); + break; +#endif case T_CreateSchemaStmt: retval = _equalCreateSchemaStmt(a, b); break; diff --git a/src/backend/parser/gram.y b/src/backend/parser/gram.y index 8ac6002994..7ec949154c 100644 --- a/src/backend/parser/gram.y +++ b/src/backend/parser/gram.y @@ -216,6 +216,7 @@ static TypeName *TableFuncTypeName(List *columns); DeallocateStmt PrepareStmt ExecuteStmt DropOwnedStmt ReassignOwnedStmt AlterTSConfigurationStmt AlterTSDictionaryStmt + BarrierStmt %type <node> select_no_parens select_with_parens select_clause simple_select values_clause @@ -445,6 +446,7 @@ static TypeName *TableFuncTypeName(List *columns); opt_frame_clause frame_extent frame_bound %type <str> opt_existing_window_name /* PGXC_BEGIN */ +%type <str> opt_barrier_id %type <distby> OptDistributeBy /* PGXC_END */ @@ -470,12 +472,12 @@ static TypeName *TableFuncTypeName(List *columns); */ /* ordinary key words in alphabetical order */ -/* PGXC - added REPLICATION, DISTRIBUTE, MODULO and HASH */ +/* PGXC - added REPLICATION, DISTRIBUTE, MODULO, BARRIER and HASH */ %token <keyword> ABORT_P ABSOLUTE_P ACCESS ACTION ADD_P ADMIN AFTER AGGREGATE ALL ALSO ALTER ALWAYS ANALYSE ANALYZE AND ANY ARRAY AS ASC ASSERTION ASSIGNMENT ASYMMETRIC AT AUTHORIZATION - BACKWARD BEFORE BEGIN_P BETWEEN BIGINT BINARY BIT + BACKWARD BARRIER BEFORE BEGIN_P BETWEEN BIGINT BINARY BIT BOOLEAN_P BOTH BY CACHE CALLED CASCADE CASCADED CASE CAST CATALOG_P CHAIN CHAR_P @@ -683,6 +685,7 @@ stmt : | AlterUserSetStmt | AlterUserStmt | AnalyzeStmt + | BarrierStmt | CheckPointStmt | CleanConnStmt | ClosePortalStmt @@ -6985,6 +6988,28 @@ opt_name_list: ; +/* PGXC_BEGIN */ +BarrierStmt: CREATE BARRIER opt_barrier_id + { + BarrierStmt *n = makeNode(BarrierStmt); + n->id = $3; + $$ = (Node *)n; + } + ; + +opt_barrier_id: + Sconst + { + $$ = pstrdup($1); + } + | /* EMPTY */ + { + $$ = NULL; + } + ; + +/* PGXC_END */ + /***************************************************************************** * * QUERY: @@ -10997,7 +11022,7 @@ ColLabel: IDENT { $$ = $1; } /* "Unreserved" keywords --- available for use as any kind of name. */ -/* PGXC - added DISTRIBUTE, HASH, REPLICATION, MODULO */ +/* PGXC - added DISTRIBUTE, HASH, REPLICATION, MODULO, BARRIER */ unreserved_keyword: ABORT_P | ABSOLUTE_P @@ -11014,6 +11039,9 @@ unreserved_keyword: | ASSIGNMENT | AT | BACKWARD +/* PGXC_BEGIN */ + | BARRIER +/* PGXC_END */ | BEFORE | BEGIN_P | BY diff --git a/src/backend/pgxc/Makefile b/src/backend/pgxc/Makefile index eecac20a86..ad6bb6472c 100644 --- a/src/backend/pgxc/Makefile +++ b/src/backend/pgxc/Makefile @@ -11,6 +11,6 @@ subdir = src/backend/pgxc top_builddir = ../../.. include $(top_builddir)/src/Makefile.global -SUBDIRS = locator plan pool +SUBDIRS = locator plan pool barrier include $(top_srcdir)/src/backend/common.mk diff --git a/src/backend/pgxc/barrier/Makefile b/src/backend/pgxc/barrier/Makefile new file mode 100644 index 0000000000..d80bbeced4 --- /dev/null +++ b/src/backend/pgxc/barrier/Makefile @@ -0,0 +1,19 @@ +#------------------------------------------------------------------------- +# +# Makefile-- +# Makefile for pool +# +# Portions Copyright (c) 2010-2011 Nippon Telegraph and Telephone Corporation +# +# IDENTIFICATION +# $PostgreSQL$ +# +#------------------------------------------------------------------------- + +subdir = src/backend/pgxc/barrier +top_builddir = ../../../.. +include $(top_builddir)/src/Makefile.global + +OBJS = barrier.o + +include $(top_srcdir)/src/backend/common.mk diff --git a/src/backend/pgxc/barrier/barrier.c b/src/backend/pgxc/barrier/barrier.c new file mode 100644 index 0000000000..3e1d7cca01 --- /dev/null +++ b/src/backend/pgxc/barrier/barrier.c @@ -0,0 +1,493 @@ +/*------------------------------------------------------------------------- + * + * barrier.c + * + * Barrier handling for PITR + * + * + * Portions Copyright (c) 1996-2009, PostgreSQL Global Development Group + * Portions Copyright (c) 2010-2011 Nippon Telegraph and Telephone Corporation + * + * IDENTIFICATION + * $$ + * + *------------------------------------------------------------------------- + */ + +#include "postgres.h" +#include "libpq/libpq.h" +#include "libpq/pqformat.h" +#include "pgxc/barrier.h" +#include "pgxc/execRemote.h" +#include "pgxc/locator.h" +#include "pgxc/pgxc.h" +#include "pgxc/pgxcnode.h" +#include "storage/lwlock.h" +#include "tcop/dest.h" + +static const char *generate_barrier_id(const char *id); +static PGXCNodeAllHandles *PrepareBarrier(const char *id); +static void ExecuteBarrier(const char *id); +static void EndBarrier(PGXCNodeAllHandles *handles, const char *id); + +extern void ProcessCreateBarrierPrepare(const char *id); +extern void ProcessCreateBarrierEnd(const char *id); +extern void ProcessCreateBarrierExecute(const char *id); + +/* + * Prepare ourselves for an incoming BARRIER. We must disable all new 2PC + * commits and let the ongoing commits to finish. We then remember the + * barrier id (so that it can be matched with the final END message) and + * tell the driving coordinator to proceed with the next step. + * + * A simple way to implement this is to grab a lock in an exclusive mode + * while all other backend starting a 2PC will grab the lock in shared + * mode. So as long as we hold the exclusive lock, no other backend start a + * new 2PC and there can not be any 2PC in-progress. This technique would + * rely on assumption that an exclsuive lock requester is not starved by + * share lock requesters. + * + * Note: To ensure that the 2PC are not blocked for a long time, we should + * set a timeout. The lock should be release after the timeout and the + * barrier should be canceled. + */ +void +ProcessCreateBarrierPrepare(const char *id) +{ + StringInfoData buf; + + if (!IS_PGXC_COORDINATOR || !IsConnFromCoord()) + ereport(ERROR, + (errcode(ERRCODE_INTERNAL_ERROR), + errmsg("The CREATE BARRIER PREPARE message is expected to " + "arrive at a coordinator from another coordinator"))); + + LWLockAcquire(BarrierLock, LW_EXCLUSIVE); + + pq_beginmessage(&buf, 'b'); + pq_sendstring(&buf, id); + pq_endmessage(&buf); + pq_flush(); + + /* + * TODO Start a timer to terminate the pending barrier after a specified + * timeout + */ +} + +/* + * Mark the completetion of an on-going barrier. We must have remembered the + * barrier ID when we received the CREATE BARRIER PREPARE command + */ +void +ProcessCreateBarrierEnd(const char *id) +{ + StringInfoData buf; + + if (!IS_PGXC_COORDINATOR || !IsConnFromCoord()) + ereport(ERROR, + (errcode(ERRCODE_INTERNAL_ERROR), + errmsg("The CREATE BARRIER END message is expected to " + "arrive at a coordinator from another coordinator"))); + + LWLockRelease(BarrierLock); + + pq_beginmessage(&buf, 'b'); + pq_sendstring(&buf, id); + pq_endmessage(&buf); + pq_flush(); + + /* + * TODO Stop the timer + */ +} + +/* + * Execute the CREATE BARRIER comamnd. Write a BARRIER WAL record and flush the + * WAL buffers to disk before returning to the caller. Writing the WAL record + * does not guarantee successful completion of the barrier command. + */ +void +ProcessCreateBarrierExecute(const char *id) +{ + StringInfoData buf; + + if (!IsConnFromCoord()) + ereport(ERROR, + (errcode(ERRCODE_INTERNAL_ERROR), + errmsg("The CREATE BARRIER EXECUTE message is expected to " + "arrive from a coordinator"))); + { + XLogRecData rdata[1]; + XLogRecPtr recptr; + + rdata[0].data = (char *) id; + rdata[0].len = strlen(id) + 1; + rdata[0].buffer = InvalidBuffer; + rdata[0].next = NULL; + + recptr = XLogInsert(RM_BARRIER_ID, XLOG_BARRIER_CREATE, rdata); + XLogFlush(recptr); + } + + pq_beginmessage(&buf, 'b'); + pq_sendstring(&buf, id); + pq_endmessage(&buf); + pq_flush(); +} + +static const char * +generate_barrier_id(const char *id) +{ + /* + * TODO If the caller can passeed a NULL value, generate an id which is + * guaranteed to be unique across the cluster. We can use a combination of + * the coordinator node id and a timestamp. This may not be complete if we + * support changing coordinator ids without initdb or the system clocks are + * modified. + * + * Another option would be to let the GTM issue globally unique barrier + * IDs. For the time being, we leave it to the user to come up with an + * unique identifier + */ + return id ? id : pstrdup("dummy_barrier_id"); +} + +static PGXCNodeAllHandles * +SendBarrierPrepareRequest(List *coords, const char *id) +{ + PGXCNodeAllHandles *coord_handles; + int conn; + int msglen; + int barrier_idlen; + + coord_handles = get_handles(NIL, coords, true); + + for (conn = 0; conn < coord_handles->co_conn_count; conn++) + { + PGXCNodeHandle *handle = coord_handles->coord_handles[conn]; + + /* Invalid connection state, return error */ + if (handle->state != DN_CONNECTION_STATE_IDLE) + ereport(ERROR, + (errcode(ERRCODE_INTERNAL_ERROR), + errmsg("Failed to send CREATE BARRIER PREPARE request " + "to the node"))); + + barrier_idlen = strlen(id) + 1; + + msglen = 4; /* for the length itself */ + msglen += barrier_idlen; + msglen += 1; /* for barrier command itself */ + + /* msgType + msgLen */ + if (ensure_out_buffer_capacity(handle->outEnd + 1 + msglen, handle) != 0) + { + ereport(ERROR, + (errcode(ERRCODE_INTERNAL_ERROR), + errmsg("Out of memory"))); + } + + handle->outBuffer[handle->outEnd++] = 'b'; + msglen = htonl(msglen); + memcpy(handle->outBuffer + handle->outEnd, &msglen, 4); + handle->outEnd += 4; + + handle->outBuffer[handle->outEnd++] = CREATE_BARRIER_PREPARE; + + memcpy(handle->outBuffer + handle->outEnd, id, barrier_idlen); + handle->outEnd += barrier_idlen; + + handle->state = DN_CONNECTION_STATE_QUERY; + + pgxc_node_flush(handle); + + /* FIXME Use the right context */ + handle->barrier_id = strdup(id); + } + + return coord_handles; +} + +static void +CheckBarrierCommandStatus(PGXCNodeAllHandles *conn_handles, const char *id, + const char *command) +{ + int conn; + int count = conn_handles->co_conn_count + conn_handles->dn_conn_count; + + elog(DEBUG2, "Check CREATE BARRIER <%s> %s command status", id, command); + + for (conn = 0; conn < count; conn++) + { + PGXCNodeHandle *handle; + + if (conn < conn_handles->co_conn_count) + handle = conn_handles->coord_handles[conn]; + else + handle = conn_handles->datanode_handles[conn - conn_handles->co_conn_count]; + + if (pgxc_node_receive(1, &handle, NULL)) + ereport(ERROR, + (errcode(ERRCODE_INTERNAL_ERROR), + errmsg("Failed to receive response from the remote side"))); + + if (handle_response(handle, NULL) != RESPONSE_BARRIER_OK) + ereport(ERROR, + (errcode(ERRCODE_INTERNAL_ERROR), + errmsg("CREATE BARRIER PREPARE command failed " + "with error %s", handle->error))); + } + + elog(DEBUG2, "Successfully completed CREATE BARRIER <%s> %s command on " + "all nodes", id, command); +} + +static void +SendBarrierEndRequest(PGXCNodeAllHandles *coord_handles, const char *id) +{ + int conn; + int msglen; + int barrier_idlen; + + elog(DEBUG2, "Sending CREATE BARRIER <%s> END command to all coordinators", id); + + for (conn = 0; conn < coord_handles->co_conn_count; conn++) + { + PGXCNodeHandle *handle = coord_handles->coord_handles[conn]; + + /* Invalid connection state, return error */ + if (handle->state != DN_CONNECTION_STATE_IDLE) + ereport(ERROR, + (errcode(ERRCODE_INTERNAL_ERROR), + errmsg("Failed to send CREATE BARRIER PREPARE request " + "to the node"))); + + barrier_idlen = strlen(id) + 1; + + msglen = 4; /* for the length itself */ + msglen += barrier_idlen; + msglen += 1; /* for barrier command itself */ + + /* msgType + msgLen */ + if (ensure_out_buffer_capacity(handle->outEnd + 1 + msglen, handle) != 0) + { + ereport(ERROR, + (errcode(ERRCODE_INTERNAL_ERROR), + errmsg("Out of memory"))); + } + + handle->outBuffer[handle->outEnd++] = 'b'; + msglen = htonl(msglen); + memcpy(handle->outBuffer + handle->outEnd, &msglen, 4); + handle->outEnd += 4; + + handle->outBuffer[handle->outEnd++] = CREATE_BARRIER_END; + + memcpy(handle->outBuffer + handle->outEnd, id, barrier_idlen); + handle->outEnd += barrier_idlen; + + handle->state = DN_CONNECTION_STATE_QUERY; + pgxc_node_flush(handle); + + /* FIXME Use the right context */ + handle->barrier_id = strdup(id); + } + +} + +/* + * Prepare all coordinators for barrier. During this step all the coordinators + * are informed to suspend any new 2PC transactions. The coordinators should + * disable new 2PC transactions and then wait for the existing transactions to + * complete. Once all "in-flight" 2PC transactions are over, the coordinators + * respond back. + * + * That completes the first step in barrier generation + * + * Any errors will be reported via ereport. + */ +static PGXCNodeAllHandles * +PrepareBarrier(const char *id) +{ + PGXCNodeAllHandles *coord_handles; + + elog(DEBUG2, "Preparing coordinators for BARRIER"); + + /* + * Send a CREATE BARRIER PREPARE message to all the coordinators. We should + * send an asynchronous request so that we can disable local commits and + * then wait for the remote coordinators to finish the work + */ + coord_handles = SendBarrierPrepareRequest(GetAllCoordNodes(), id); + + /* + * Disable local commits + */ + LWLockAcquire(BarrierLock, LW_EXCLUSIVE); + + elog(DEBUG2, "Disabled 2PC commits origniating at the diriving coordinator"); + + /* + * TODO Start a timer to cancel the barrier request in case of a timeout + */ + + /* + * Local in-flight commits are now over. Check status of the remote + * coordinators + */ + CheckBarrierCommandStatus(coord_handles, id, "PREPARE"); + + return coord_handles; +} + +/* + * Execute the barrier command on all the components, including data nodes and + * coordinators. + */ +static void +ExecuteBarrier(const char *id) +{ + List *barrierDataNodeList = GetAllDataNodes(); + List *barrierCoordList = GetAllCoordNodes(); + PGXCNodeAllHandles *conn_handles; + int conn; + int msglen; + int barrier_idlen; + + conn_handles = get_handles(barrierDataNodeList, barrierCoordList, false); + + elog(DEBUG2, "Sending CREATE BARRIER <%s> EXECUTE message to " + "data nodes and coordinator", id); + /* + * Send a CREATE BARRIER request to all the data nodes and the coordinators + */ + for (conn = 0; conn < conn_handles->co_conn_count + conn_handles->dn_conn_count; conn++) + { + PGXCNodeHandle *handle; + + if (conn < conn_handles->co_conn_count) + handle = conn_handles->coord_handles[conn]; + else + handle = conn_handles->datanode_handles[conn - conn_handles->co_conn_count]; + + /* Invalid connection state, return error */ + if (handle->state != DN_CONNECTION_STATE_IDLE) + ereport(ERROR, + (errcode(ERRCODE_INTERNAL_ERROR), + errmsg("Failed to send CREATE BARRIER PREPARE request " + "to the node"))); + + barrier_idlen = strlen(id) + 1; + + msglen = 4; /* for the length itself */ + msglen += barrier_idlen; + msglen += 1; /* for barrier command itself */ + + /* msgType + msgLen */ + if (ensure_out_buffer_capacity(handle->outEnd + 1 + msglen, handle) != 0) + { + ereport(ERROR, + (errcode(ERRCODE_INTERNAL_ERROR), + errmsg("Out of memory"))); + } + + handle->outBuffer[handle->outEnd++] = 'b'; + msglen = htonl(msglen); + memcpy(handle->outBuffer + handle->outEnd, &msglen, 4); + handle->outEnd += 4; + + handle->outBuffer[handle->outEnd++] = CREATE_BARRIER_EXECUTE; + + memcpy(handle->outBuffer + handle->outEnd, id, barrier_idlen); + handle->outEnd += barrier_idlen; + + handle->state = DN_CONNECTION_STATE_QUERY; + pgxc_node_flush(handle); + + /* FIXME Use the right context */ + handle->barrier_id = strdup(id); + } + + CheckBarrierCommandStatus(conn_handles, id, "EXECUTE"); + + /* + * Also WAL log the BARRIER locally and flush the WAL buffers to disk + */ +} + +/* + * Resume 2PC commits on the local as well as remote coordinators. + */ +static void +EndBarrier(PGXCNodeAllHandles *prepared_handles, const char *id) +{ + /* Resume 2PC locally */ + LWLockRelease(BarrierLock); + + SendBarrierEndRequest(prepared_handles, id); + + CheckBarrierCommandStatus(prepared_handles, id, "END"); +} + +void +RequestBarrier(const char *id, char *completionTag) +{ + PGXCNodeAllHandles *prepared_handles; + const char *barrier_id; + + elog(DEBUG2, "CREATE BARRIER request received"); + /* + * Ensure that we are a coordinator and the request is not from another + * coordinator + */ + if (!IS_PGXC_COORDINATOR) + ereport(ERROR, + (errcode(ERRCODE_INTERNAL_ERROR), + errmsg("CREATE BARRIER command must be sent to a coordinator"))); + + if (IsConnFromCoord()) + ereport(ERROR, + (errcode(ERRCODE_INTERNAL_ERROR), + errmsg("CREATE BARRIER command is not expected from another coordinator"))); + + /* + * Get a barrier id if the user has not supplied it + */ + barrier_id = generate_barrier_id(id); + + elog(DEBUG2, "CREATE BARRIER <%s>", barrier_id); + + /* + * Step One. Prepare all coordinators for upcoming barrier request + */ + prepared_handles = PrepareBarrier(barrier_id); + + /* + * Step two. Issue BARRIER command to all involved components, including + * coordinators and data nodes + */ + ExecuteBarrier(barrier_id); + + /* + * Step three. Inform coordinators about a successfully completed barrier + */ + EndBarrier(prepared_handles, barrier_id); + + if (completionTag) + snprintf(completionTag, COMPLETION_TAG_BUFSIZE, "BARRIER %s", barrier_id); +} + +void +barrier_redo(XLogRecPtr lsn, XLogRecord *record) +{ + /* Nothing to do */ + return; +} + +void +barrier_desc(StringInfo buf, uint8 xl_info, char *rec) +{ + Assert(xl_info == XLOG_BARRIER_CREATE); + appendStringInfo(buf, "BARRIER %s", rec); +} diff --git a/src/backend/pgxc/pool/execRemote.c b/src/backend/pgxc/pool/execRemote.c index 03482a0721..c2970031a7 100644 --- a/src/backend/pgxc/pool/execRemote.c +++ b/src/backend/pgxc/pool/execRemote.c @@ -1389,6 +1389,7 @@ pgxc_node_receive_responses(const int conn_count, PGXCNodeHandle ** connections, * RESPONSE_TUPLEDESC - got tuple description * RESPONSE_DATAROW - got data row * RESPONSE_COPY - got copy response + * RESPONSE_BARRIER_OK - barrier command completed successfully */ int handle_response(PGXCNodeHandle * conn, RemoteQueryState *combiner) @@ -1500,6 +1501,16 @@ handle_response(PGXCNodeHandle * conn, RemoteQueryState *combiner) #endif return result; } + +#ifdef PGXC + case 'b': + { + Assert((strncmp(msg, conn->barrier_id, msg_len) == 0)); + conn->state = DN_CONNECTION_STATE_IDLE; + return RESPONSE_BARRIER_OK; + } +#endif + case 'I': /* EmptyQuery */ default: /* sync lost? */ diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c index 71b139895a..c54bd60f1e 100644 --- a/src/backend/tcop/postgres.c +++ b/src/backend/tcop/postgres.c @@ -80,6 +80,7 @@ #include "access/gtm.h" /* PGXC_COORD */ #include "pgxc/execRemote.h" +#include "pgxc/barrier.h" #include "pgxc/planner.h" #include "pgxc/pgxcnode.h" #include "commands/copy.h" @@ -447,6 +448,7 @@ SocketBackend(StringInfo inBuf) case 'g': /* GXID */ case 's': /* Snapshot */ case 't': /* Timestamp */ + case 'b': /* Barrier */ break; #endif @@ -4290,6 +4292,37 @@ PostgresMain(int argc, char *argv[], const char *username) */ SetCurrentGTMDeltaTimestamp(timestamp); break; + + case 'b': /* barrier */ + { + int command; + char *id; + + command = pq_getmsgbyte(&input_message); + id = pq_getmsgstring(&input_message); + pq_getmsgend(&input_message); + + switch (command) + { + case CREATE_BARRIER_PREPARE: + ProcessCreateBarrierPrepare(id); + break; + + case CREATE_BARRIER_END: + ProcessCreateBarrierEnd(id); + break; + + case CREATE_BARRIER_EXECUTE: + ProcessCreateBarrierExecute(id); + break; + + default: + ereport(ERROR, + (errcode(ERRCODE_INTERNAL_ERROR), + errmsg("Invalid command received"))); + } + } + break; #endif /* PGXC */ default: diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c index e9e7afd271..d8f697e4a8 100644 --- a/src/backend/tcop/utility.c +++ b/src/backend/tcop/utility.c @@ -58,6 +58,7 @@ #include "utils/syscache.h" #ifdef PGXC +#include "pgxc/barrier.h" #include "pgxc/locator.h" #include "pgxc/pgxc.h" #include "pgxc/planner.h" @@ -1702,6 +1703,12 @@ standard_ProcessUtility(Node *parsetree, #endif break; +#ifdef PGXC + case T_BarrierStmt: + RequestBarrier(((BarrierStmt *) parsetree)->id, completionTag); + break; +#endif + case T_ReindexStmt: { ReindexStmt *stmt = (ReindexStmt *) parsetree; @@ -2692,6 +2699,12 @@ CreateCommandTag(Node *parsetree) tag = "CHECKPOINT"; break; +#ifdef PGXC + case T_BarrierStmt: + tag = "BARRIER"; + break; +#endif + case T_ReindexStmt: tag = "REINDEX"; break; diff --git a/src/include/access/rmgr.h b/src/include/access/rmgr.h index 72ee757f70..8253fad69f 100644 --- a/src/include/access/rmgr.h +++ b/src/include/access/rmgr.h @@ -32,6 +32,11 @@ typedef uint8 RmgrId; #define RM_GIN_ID 13 #define RM_GIST_ID 14 #define RM_SEQ_ID 15 +#ifdef PGXC +#define RM_BARRIER_ID 16 +#define RM_MAX_ID RM_BARRIER_ID +#else #define RM_MAX_ID RM_SEQ_ID +#endif #endif /* RMGR_H */ diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h index 5f5b947888..f2d937f222 100644 --- a/src/include/nodes/nodes.h +++ b/src/include/nodes/nodes.h @@ -327,6 +327,9 @@ typedef enum NodeTag T_ConstraintsSetStmt, T_ReindexStmt, T_CheckPointStmt, +#ifdef PGXC + T_BarrierStmt, +#endif T_CreateSchemaStmt, T_AlterDatabaseStmt, T_AlterDatabaseSetStmt, diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h index 79497e23b6..7ba9208ebf 100644 --- a/src/include/nodes/parsenodes.h +++ b/src/include/nodes/parsenodes.h @@ -2292,6 +2292,19 @@ typedef struct VacuumStmt List *va_cols; /* list of column names, or NIL for all */ } VacuumStmt; +#ifdef PGXC +/* + * ---------------------- + * Barrier Statement + */ +typedef struct BarrierStmt +{ + NodeTag type; + const char *id; /* User supplied barrier id, if any */ +} BarrierStmt; + +#endif + /* ---------------------- * Explain Statement * diff --git a/src/include/parser/kwlist.h b/src/include/parser/kwlist.h index 0564bcb8bc..12afba71fa 100644 --- a/src/include/parser/kwlist.h +++ b/src/include/parser/kwlist.h @@ -52,6 +52,9 @@ PG_KEYWORD("asymmetric", ASYMMETRIC, RESERVED_KEYWORD) PG_KEYWORD("at", AT, UNRESERVED_KEYWORD) PG_KEYWORD("authorization", AUTHORIZATION, TYPE_FUNC_NAME_KEYWORD) PG_KEYWORD("backward", BACKWARD, UNRESERVED_KEYWORD) +#ifdef PGXC +PG_KEYWORD("barrier", BARRIER, UNRESERVED_KEYWORD) +#endif PG_KEYWORD("before", BEFORE, UNRESERVED_KEYWORD) PG_KEYWORD("begin", BEGIN_P, UNRESERVED_KEYWORD) PG_KEYWORD("between", BETWEEN, COL_NAME_KEYWORD) diff --git a/src/include/pgxc/barrier.h b/src/include/pgxc/barrier.h new file mode 100644 index 0000000000..37b506731c --- /dev/null +++ b/src/include/pgxc/barrier.h @@ -0,0 +1,40 @@ +/*------------------------------------------------------------------------- + * + * barrier.h + * + * Definitions for the PITR barrier handling + * + * + * Portions Copyright (c) 1996-2009, PostgreSQL Global Development Group + * Portions Copyright (c) 2010-2011 Nippon Telegraph and Telephone Corporation + * + * IDENTIFICATION + * $$ + * + *------------------------------------------------------------------------- + */ + +#ifndef BARRIER_H +#define BARRIER_H + +#include "access/xlog.h" +#include "access/xlogdefs.h" + +#define CREATE_BARRIER_PREPARE 'P' +#define CREATE_BARRIER_EXECUTE 'X' +#define CREATE_BARRIER_END 'E' + +#define CREATE_BARRIER_PREPARE_DONE 'p' +#define CREATE_BARRIER_EXECUTE_DONE 'x' + +typedef struct xl_barrier +{ + char barrier_id[1]; /* variable length data follows */ +} xl_barrier; + +#define XLOG_BARRIER_CREATE 0x00 + +extern void RequestBarrier(const char *id, char *completionTag); +extern void barrier_redo(XLogRecPtr lsn, XLogRecord *record); +extern void barrier_desc(StringInfo buf, uint8 xl_info, char *rec); +#endif diff --git a/src/include/pgxc/execRemote.h b/src/include/pgxc/execRemote.h index fb9232fca1..076100b6f5 100644 --- a/src/include/pgxc/execRemote.h +++ b/src/include/pgxc/execRemote.h @@ -36,6 +36,7 @@ #define RESPONSE_TUPDESC 2 #define RESPONSE_DATAROW 3 #define RESPONSE_COPY 4 +#define RESPONSE_BARRIER_OK 5 typedef enum { diff --git a/src/include/pgxc/pgxcnode.h b/src/include/pgxc/pgxcnode.h index 76f131e10d..8f1eb54f1a 100644 --- a/src/include/pgxc/pgxcnode.h +++ b/src/include/pgxc/pgxcnode.h @@ -66,6 +66,7 @@ struct pgxc_node_handle #ifdef DN_CONNECTION_DEBUG bool have_row_desc; #endif + char *barrier_id; char *error; /* Output buffer */ char *outBuffer; diff --git a/src/include/storage/lwlock.h b/src/include/storage/lwlock.h index 26acb95186..5c88470681 100644 --- a/src/include/storage/lwlock.h +++ b/src/include/storage/lwlock.h @@ -69,6 +69,7 @@ typedef enum LWLockId SyncScanLock, #ifdef PGXC AnalyzeProcArrayLock, + BarrierLock, #endif RelationMappingLock, AsyncCtlLock, |