cdbtm.c 109.5 KB
Newer Older
1 2 3 4 5
/*-------------------------------------------------------------------------
 *
 * cdbtm.c
 *	  Provides routines for performing distributed transaction
 *
6 7 8 9 10 11
 * Portions Copyright (c) 2005-2009, Greenplum inc
 * Portions Copyright (c) 2012-Present Pivotal Software, Inc.
 *
 *
 * IDENTIFICATION
 *	    src/backend/cdb/cdbtm.c
12 13 14 15 16 17
 *
 *-------------------------------------------------------------------------
 */
#include "postgres.h"

#include <time.h>
18
#include <sys/types.h>
19 20
#include <unistd.h>

H
Heikki Linnakangas 已提交
21
#include "catalog/pg_authid.h"
22 23 24 25 26 27
#include "cdb/cdbtm.h"
#include "libpq/libpq-be.h"
#include "miscadmin.h"
#include "storage/shmem.h"
#include "storage/ipc.h"
#include "cdb/cdbdisp.h"
28 29
#include "cdb/cdbdisp_query.h"
#include "cdb/cdbdisp_dtx.h"
30
#include "cdb/cdbdispatchresult.h"
31 32 33 34 35
#include "cdb/cdbdtxcontextinfo.h"

#include "cdb/cdbvars.h"
#include "access/transam.h"
#include "access/xact.h"
36 37
#include "libpq-fe.h"
#include "libpq-int.h"
38 39 40 41 42
#include "cdb/cdbfts.h"
#include "lib/stringinfo.h"
#include "access/twophase.h"
#include "access/distributedlog.h"
#include "postmaster/postmaster.h"
43
#include "storage/procarray.h"
44 45 46 47 48 49
#include "cdb/cdbpersistentrecovery.h"
#include "cdb/cdbpersistentcheck.h"

#include "cdb/cdbllize.h"
#include "utils/faultinjector.h"
#include "utils/fmgroids.h"
50
#include "utils/sharedsnapshot.h"
51
#include "utils/snapmgr.h"
52 53 54 55 56 57 58 59 60 61 62 63 64 65 66 67 68 69 70 71

extern bool Test_print_direct_dispatch_info;
extern struct Port *MyProcPort;

#define DTM_DEBUG3 (Debug_print_full_dtm ? LOG : DEBUG3)
#define DTM_DEBUG5 (Debug_print_full_dtm ? LOG : DEBUG5)

/*
 * Directory where Utility Mode DTM REDO file reside within PGDATA
 */
#define UTILITYMODEDTMREDO_DIR "pg_utilitymodedtmredo"

/*
 * File name for Utility Mode DTM REDO
 */
#define UTILITYMODEDTMREDO_FILE "savedtmredo.file"

static LWLockId shmControlLock;
static volatile bool *shmTmRecoverred;
static volatile DistributedTransactionTimeStamp *shmDistribTimeStamp;
72
static volatile DistributedTransactionId *shmGIDSeq = NULL;
73 74 75 76
static volatile int *shmNumGxacts;

static int	ControlLockCount = 0;

A
Ashwin Agrawal 已提交
77
uint32	   *shmNextSnapshotId;
78

A
Ashwin Agrawal 已提交
79
volatile bool *shmDtmStarted;
80 81 82 83 84 85 86 87 88 89 90 91

/* global transaction array */
static TMGXACT **shmGxactArray;

/**
 * This pointer into shared memory is on the QD, and represents the current open transaction.
 */
static TMGXACT *currentGxact;

static int	max_tm_gxacts = 100;

static int	redoFileFD = -1;
A
Ashwin Agrawal 已提交
92
static int	redoFileOffset;
93 94 95 96

typedef struct InDoubtDtx
{
	char		gid[TMGIDSIZE];
A
Ashwin Agrawal 已提交
97
} InDoubtDtx;
98 99 100 101 102 103 104 105 106 107 108 109 110 111 112 113 114 115 116 117 118 119 120 121 122 123 124 125


/* here are some flag options relationed to the txnOptions field of
 * PQsendGpQuery
 */

/* bit 1 is for statement wants DTX transaction
 *
 * bits 2-3 for iso level  00 read-committed
 *						   01 read-uncommitted
 *						   10 repeatable-read
 *						   11 serializable
 * bit 4 is for read-only
 */
#define GP_OPT_NEED_TWO_PHASE                           0x0001

#define GP_OPT_READ_COMMITTED    						0x0002
#define GP_OPT_READ_UNCOMMITTED  						0x0004
#define GP_OPT_REPEATABLE_READ   						0x0006
#define GP_OPT_SERIALIZABLE 	  						0x0008

#define GP_OPT_READ_ONLY         						0x0010

#define GP_OPT_EXPLICT_BEGIN      						0x0020

/*=========================================================================
 * FUNCTIONS PROTOTYPES
 */
A
Ashwin Agrawal 已提交
126
static void initGxact(TMGXACT *gxact);
127 128 129 130 131 132 133 134 135 136
static void releaseGxact_UnderLocks(void);
static void releaseGxact(void);
static void generateGID(char *gid, DistributedTransactionId *gxid);

static void recoverTM(void);
static bool recoverInDoubtTransactions(void);
static HTAB *gatherRMInDoubtTransactions(void);
static void abortRMInDoubtTransactions(HTAB *htab);

static void dumpAllDtx(void);
A
Ashwin Agrawal 已提交
137

138 139 140 141
/* static void resolveInDoubtDtx(void); */
static void dumpRMOnlyDtx(HTAB *htab, StringInfoData *buff);

static bool doDispatchDtxProtocolCommand(DtxProtocolCommand dtxProtocolCommand, int flags,
A
Ashwin Agrawal 已提交
142 143 144
							 char *gid, DistributedTransactionId gxid,
							 bool *badGangs, bool raiseError, CdbDispatchDirectDesc *direct,
							 char *serializedDtxContextInfo, int serializedDtxContextInfoLen);
145 146 147 148
static void doPrepareTransaction(void);
static void doInsertForgetCommitted(void);
static void doNotifyingCommitPrepared(void);
static void doNotifyingAbort(void);
149
static void retryAbortPrepared(void);
150 151 152 153 154
static bool doNotifyCommittedInDoubt(char *gid);
static void doAbortInDoubt(char *gid);
static void doQEDistributedExplicitBegin(int txnOptions);

static bool isDtxQueryDispatcher(void);
A
Ashwin Agrawal 已提交
155
static void UtilityModeSaveRedo(bool committed, TMGXACT_LOG *gxact_log);
156 157 158 159 160
static void ReplayRedoFromUtilityMode(void);
static void RemoveRedoUtilityModeFile(void);
static void performDtxProtocolCommitPrepared(const char *gid, bool raiseErrorIfNotFound);
static void performDtxProtocolAbortPrepared(const char *gid, bool raiseErrorIfNotFound);

G
Gang Xiong 已提交
161
extern void resetSessionForPrimaryGangLoss(bool resetSession);
162 163 164 165 166 167 168 169 170 171 172
extern void CheckForResetSession(void);

/**
 * All assignments of the global DistributedTransactionContext should go through this function
 *   (so we can add logging here to see all assignments)
 *
 * @param context the new value for DistributedTransactionContext
 */
static void
setDistributedTransactionContext(DtxContext context)
{
A
Ashwin Agrawal 已提交
173 174 175 176
	/*
	 * elog(INFO, "Setting DistributedTransactionContext to '%s'",
	 * DtxContextToString(context));
	 */
177 178 179 180 181 182 183 184 185 186 187 188 189 190 191 192 193 194
	DistributedTransactionContext = context;
}

static void
requireDistributedTransactionContext(DtxContext requiredCurrentContext)
{
	if (DistributedTransactionContext != requiredCurrentContext)
	{
		elog(FATAL, "Expected segment distributed transaction context to be '%s', found '%s'",
			 DtxContextToString(requiredCurrentContext),
			 DtxContextToString(DistributedTransactionContext));
	}
}

static void
setGxactState(TMGXACT *transaction, DtxState state)
{
	Assert(transaction != NULL);
A
Ashwin Agrawal 已提交
195 196 197 198 199

	/*
	 * elog(INFO, "Setting transaction state to '%s'",
	 * DtxStateToString(state));
	 */
200 201 202 203 204 205 206 207 208 209 210 211 212 213 214 215 216 217 218 219 220 221 222
	transaction->state = state;
}

/**
 * All assignments of currentGxact->state should go through this function
 *   (so we can add logging here to see all assignments)
 *
 * This should only be called when currentGxact is non-NULL
 *
 * @param state the new value for currentGxact->state
 */
static void
setCurrentGxactState(DtxState state)
{
	setGxactState(currentGxact, state);
}

/**
 * Does DistributedTransactionContext indicate that this is acting as a QD?
 */
static bool
isQDContext(void)
{
A
Ashwin Agrawal 已提交
223
	switch (DistributedTransactionContext)
224 225 226 227 228 229 230 231 232 233 234 235 236 237 238
	{
		case DTX_CONTEXT_QD_DISTRIBUTED_CAPABLE:
		case DTX_CONTEXT_QD_RETRY_PHASE_2:
			return true;
		default:
			return false;
	}
}

/**
 * Does DistributedTransactionContext indicate that this is acting as a QE?
 */
static bool
isQEContext()
{
A
Ashwin Agrawal 已提交
239
	switch (DistributedTransactionContext)
240 241 242 243 244 245 246 247 248 249 250 251 252 253 254 255 256 257 258 259 260 261 262 263 264 265 266 267
	{
		case DTX_CONTEXT_QE_ENTRY_DB_SINGLETON:
		case DTX_CONTEXT_QE_AUTO_COMMIT_IMPLICIT:
		case DTX_CONTEXT_QE_TWO_PHASE_EXPLICIT_WRITER:
		case DTX_CONTEXT_QE_TWO_PHASE_IMPLICIT_WRITER:
		case DTX_CONTEXT_QE_READER:
			return true;
		default:
			return false;
	}
}

/*=========================================================================
 * VISIBLE FUNCTIONS
 */

DistributedTransactionTimeStamp
getDtxStartTime(void)
{
	if (shmDistribTimeStamp != NULL)
		return *shmDistribTimeStamp;
	else
		return 0;
}

DistributedTransactionId
getDistributedTransactionId(void)
{
A
Ashwin Agrawal 已提交
268
	if (isQDContext())
269 270 271 272 273
	{
		return currentGxact == NULL
			? InvalidDistributedTransactionId
			: currentGxact->gxid;
	}
A
Ashwin Agrawal 已提交
274
	else if (isQEContext())
275 276 277 278 279 280 281 282 283 284 285 286
	{
		return QEDtxContextInfo.distributedXid;
	}
	else
	{
		return InvalidDistributedTransactionId;
	}
}

bool
getDistributedTransactionIdentifier(char *id)
{
A
Ashwin Agrawal 已提交
287
	if (isQDContext())
288 289 290 291
	{
		if (currentGxact != NULL)
		{
			/*
A
Ashwin Agrawal 已提交
292 293
			 * The length check here requires the identifer have a trailing
			 * NUL character.
294 295 296
			 */
			if (strlen(currentGxact->gid) >= TMGIDSIZE)
				elog(PANIC, "Distribute transaction identifier too long (%d)",
A
Ashwin Agrawal 已提交
297
					 (int) strlen(currentGxact->gid));
298 299 300 301
			memcpy(id, currentGxact->gid, TMGIDSIZE);
			return true;
		}
	}
A
Ashwin Agrawal 已提交
302
	else if (isQEContext())
303 304 305 306 307
	{
		if (QEDtxContextInfo.distributedXid != InvalidDistributedTransactionId)
		{
			if (strlen(QEDtxContextInfo.distributedId) >= TMGIDSIZE)
				elog(PANIC, "Distribute transaction identifier too long (%d)",
A
Ashwin Agrawal 已提交
308
					 (int) strlen(QEDtxContextInfo.distributedId));
309 310 311 312 313 314 315 316 317 318 319 320 321 322 323 324 325 326 327 328 329 330 331 332 333 334 335 336 337 338
			memcpy(id, QEDtxContextInfo.distributedId, TMGIDSIZE);
			return true;
		}
	}

	MemSet(id, 0, TMGIDSIZE);
	return false;
}

bool
isPreparedDtxTransaction(void)
{
	if (Gp_role != GP_ROLE_DISPATCH ||
		DistributedTransactionContext != DTX_CONTEXT_QD_DISTRIBUTED_CAPABLE ||
		currentGxact == NULL)
		return false;

	return (currentGxact->state == DTX_STATE_PREPARED);
}

void
getDtxLogInfo(TMGXACT_LOG *gxact_log)
{
	if (currentGxact == NULL)
	{
		elog(FATAL, "getDtxLogInfo found current distributed transaction is NULL");
	}

	if (strlen(currentGxact->gid) >= TMGIDSIZE)
		elog(PANIC, "Distribute transaction identifier too long (%d)",
A
Ashwin Agrawal 已提交
339
			 (int) strlen(currentGxact->gid));
340 341 342 343 344 345 346 347 348
	memcpy(gxact_log->gid, currentGxact->gid, TMGIDSIZE);
	gxact_log->gxid = currentGxact->gxid;
}

bool
notifyCommittedDtxTransactionIsNeeded(void)
{
	if (DistributedTransactionContext != DTX_CONTEXT_QD_DISTRIBUTED_CAPABLE)
	{
A
Ashwin Agrawal 已提交
349
		elog(DTM_DEBUG5, "notifyCommittedDtxTransaction nothing to do (DistributedTransactionContext = '%s')",
350 351 352 353 354 355
			 DtxContextToString(DistributedTransactionContext));
		return false;
	}

	if (currentGxact == NULL)
	{
A
Ashwin Agrawal 已提交
356
		elog(DTM_DEBUG5, "notifyCommittedDtxTransaction nothing to do (currentGxact == NULL)");
357 358 359 360 361 362 363 364 365 366 367 368 369
		return false;
	}

	return true;
}

/*
 * Notify commited a global transaction, called by user commit
 * or by CommitTransaction
 */
void
notifyCommittedDtxTransaction(void)
{
A
Ashwin Agrawal 已提交
370
	Assert(DistributedTransactionContext == DTX_CONTEXT_QD_DISTRIBUTED_CAPABLE);
371

A
Ashwin Agrawal 已提交
372
	Assert(currentGxact != NULL);
373 374 375 376 377 378 379 380 381 382 383 384 385 386 387 388 389 390 391 392 393 394

	doNotifyingCommitPrepared();
}

static inline void
copyDirectDispatchFromTransaction(CdbDispatchDirectDesc *dOut)
{
	if (currentGxact->directTransaction)
	{
		dOut->directed_dispatch = true;
		dOut->count = 1;
		dOut->content[0] = currentGxact->directTransactionContentId;
	}
	else
	{
		dOut->directed_dispatch = false;
	}
}

static bool
GetRootNodeIsDirectDispatch(PlannedStmt *stmt)
{
A
Ashwin Agrawal 已提交
395
	if (stmt == NULL)
396 397
		return false;

A
Ashwin Agrawal 已提交
398
	if (stmt->planTree == NULL)
399 400 401 402 403 404 405 406 407 408 409 410 411 412 413 414
		return false;

	return stmt->planTree->directDispatch.isDirectDispatch;
}

/**
 * note that the ability to look at the root node of a plan in order to determine
 *    direct dispatch overall depends on the way we assign direct dispatch.  Parent slices are
 *    never more directed than child slices.  This could be fixed with an iteration over all slices and
 *    combine from every slice.
 *
 * return true IFF the directDispatch data stored in n should be applied to the transaction
 */
static bool
GetPlannedStmtDirectDispatch_AndUsingNodeIsSufficient(PlannedStmt *stmt)
{
A
Ashwin Agrawal 已提交
415
	if (!GetRootNodeIsDirectDispatch(stmt))
416 417 418
		return false;

	/*
A
Ashwin Agrawal 已提交
419 420 421 422
	 * now look at number initplans .. we do something simple.  ANY initPlans
	 * means we don't do directDispatch at the dtm level.  It's technically
	 * possible that the initPlan and the node share the same direct dispatch
	 * set but we don't bother right now.
423
	 */
A
Ashwin Agrawal 已提交
424
	if (stmt->nInitPlans > 0)
425 426 427 428 429 430 431 432 433 434 435
		return false;

	return true;
}

/*
 * @param needsTwoPhaseCommit if true then marks the current Distributed Transaction as needing to use the
 *       2 phase commit protocol.
 */
void
dtmPreCommand(const char *debugCaller, const char *debugDetail, PlannedStmt *stmt,
A
Ashwin Agrawal 已提交
436
			  bool needsTwoPhaseCommit, bool wantSnapshot, bool inCursor)
437
{
A
Ashwin Agrawal 已提交
438 439 440
	bool		needsPromotionFromDirectDispatch = false;
	const bool	rootNodeIsDirectDispatch = GetRootNodeIsDirectDispatch(stmt);
	const bool	nodeSaysDirectDispatch = GetPlannedStmtDirectDispatch_AndUsingNodeIsSufficient(stmt);
441 442 443 444 445 446 447 448 449 450 451 452 453 454 455 456 457 458 459 460 461 462 463 464 465 466 467

	Assert(debugCaller != NULL);
	Assert(debugDetail != NULL);

	/**
	 * update the information about what segments are participating in the transaction
	 */
	if (currentGxact == NULL)
	{
		/* no open transaction so don't do anything */
	}
	else if (currentGxact->state == DTX_STATE_ACTIVE_NOT_DISTRIBUTED)
	{
		/* Can we direct this transaction to a single content-id ? */
		if (nodeSaysDirectDispatch)
		{
			currentGxact->directTransaction = true;
			currentGxact->directTransactionContentId = linitial_int(stmt->planTree->directDispatch.contentIds);

			elog(DTM_DEBUG5,
				 "dtmPreCommand going distributed (to content %d) for gid = %s (%s, detail = '%s')",
				 currentGxact->directTransactionContentId, currentGxact->gid, debugCaller, debugDetail);
		}
		else
		{
			currentGxact->directTransaction = false;

A
Ashwin Agrawal 已提交
468
			if (rootNodeIsDirectDispatch)
469
			{
A
Ashwin Agrawal 已提交
470 471 472 473
				/*
				 * implicit write on the root, but some initPlan was to all
				 * contents...so send explicit start
				 */
474 475 476 477 478 479 480 481 482 483
				needsPromotionFromDirectDispatch = true;
			}

			elog(DTM_DEBUG5,
				 "dtmPreCommand going distributed (all gangs) for gid = %s (%s, detail = '%s')",
				 currentGxact->gid, debugCaller, debugDetail);
		}
	}
	else if (currentGxact->state == DTX_STATE_ACTIVE_DISTRIBUTED)
	{
A
Ashwin Agrawal 已提交
484 485
		bool		wasDirected = currentGxact->directTransaction;
		int			wasPromotedFromDirectDispatchContentId = wasDirected ? currentGxact->directTransactionContentId : -1;
486 487

		/* Can we still direct this transaction to a single content-id ? */
A
Ashwin Agrawal 已提交
488
		if (currentGxact->directTransaction)
489
		{
A
Ashwin Agrawal 已提交
490 491
			currentGxact->directTransaction = false;
			/* turn off, but may be restored below */
492 493 494

			if (nodeSaysDirectDispatch)
			{
A
Ashwin Agrawal 已提交
495 496 497
				int			contentId = linitial_int(stmt->planTree->directDispatch.contentIds);

				if (contentId == currentGxact->directTransactionContentId)
498
				{
A
Ashwin Agrawal 已提交
499 500 501 502
					/*
					 * it was the same content!  Stay in a single direct
					 * transaction
					 */
503 504 505 506 507
					currentGxact->directTransaction = true;
				}
			}
		}

A
Ashwin Agrawal 已提交
508
		if (currentGxact->directTransaction)
509 510 511 512 513
		{
			/** was not actually promoted */
			wasPromotedFromDirectDispatchContentId = -1;
		}

A
Ashwin Agrawal 已提交
514
		if (wasPromotedFromDirectDispatchContentId != -1)
515 516 517 518 519 520 521
			needsPromotionFromDirectDispatch = true;

		elog(DTM_DEBUG5,
			 "dtmPreCommand gid = %s is already distributed (%s, detail = '%s'), (was %s : now %s)",
			 currentGxact->gid, debugCaller, debugDetail,
			 wasDirected ? "directed" : "all gangs",
			 currentGxact->directTransaction ? "directed" : "all gangs"
A
Ashwin Agrawal 已提交
522
			);
523 524 525 526 527
	}

	/**
	 * If two-phase commit then begin transaction.
	 */
A
Ashwin Agrawal 已提交
528
	if (needsTwoPhaseCommit)
529 530 531 532 533 534 535
	{
		if (currentGxact == NULL)
		{
			elog(ERROR, "DTM transaction is not active (%s, detail = '%s')", debugCaller, debugDetail);
		}
		else if (currentGxact->state == DTX_STATE_ACTIVE_NOT_DISTRIBUTED)
		{
A
Ashwin Agrawal 已提交
536
			setCurrentGxactState(DTX_STATE_ACTIVE_DISTRIBUTED);
537 538 539 540 541 542 543 544 545 546 547 548 549 550 551 552 553 554
		}
		else if (currentGxact->state == DTX_STATE_ACTIVE_DISTRIBUTED)
		{
			/* already distributed, no need to change */
		}
		else
		{
			elog(ERROR, "DTM transaction is not active (state = %s, %s, detail = '%s')",
				 DtxStateToString(currentGxact->state), debugCaller, debugDetail);
		}
	}

	/**
	 * If promotion from direct-dispatch to whole-cluster dispatch was done then tell about it.
	 *
	 * FUTURE: note that this is only needed if the query we are going to run would not itself
	 *   do this (that is, if the query we are going to run is a read-only one)
	 */
A
Ashwin Agrawal 已提交
555 556 557
	if (currentGxact &&
		currentGxact->state == DTX_STATE_ACTIVE_DISTRIBUTED &&
		needsPromotionFromDirectDispatch)
558 559
	{
		CdbDispatchDirectDesc direct = default_dispatch_direct_desc;
A
Ashwin Agrawal 已提交
560 561 562 563
		char	   *serializedDtxContextInfo;
		int			serializedDtxContextInfoLen;
		bool		badGangs,
					succeeded;
564 565

		serializedDtxContextInfo = qdSerializeDtxContextInfo(&serializedDtxContextInfoLen, wantSnapshot, inCursor,
A
Ashwin Agrawal 已提交
566
															 mppTxnOptions(true), "promoteTransactionIn_dtmPreCommand");
567 568 569 570 571 572 573 574 575 576 577 578 579 580 581 582 583 584 585

		succeeded = doDispatchDtxProtocolCommand(DTX_PROTOCOL_COMMAND_STAY_AT_OR_BECOME_IMPLIED_WRITER, /* flags */ 0,
												 currentGxact->gid, currentGxact->gxid,
												 &badGangs, /* raiseError */ false, &direct,
												 serializedDtxContextInfo, serializedDtxContextInfoLen);

		/* send a DTM command to others to tell them about the transaction */
		if (!succeeded)
		{
			ereport(ERROR, (errmsg("Global transaction upgrade from single segment to entire cluster failed for gid = \"%s\" due to error",
								   currentGxact->gid)));
		}
	}
}


/*
 * Routine to dispatch internal sub-transaction calls from UDFs to segments.
 * The calls are BeginInternalSubTransaction, ReleaseCurrentSubTransaction and
A
Ashwin Agrawal 已提交
586
 * RollbackAndReleaseCurrentSubTransaction.
587 588 589 590 591
 */
bool
doDispatchSubtransactionInternalCmd(DtxProtocolCommand cmdType)
{
	CdbDispatchDirectDesc direct = default_dispatch_direct_desc;
A
Ashwin Agrawal 已提交
592 593 594 595
	char	   *serializedDtxContextInfo = NULL;
	int			serializedDtxContextInfoLen = 0;
	bool		badGangs,
				succeeded = false;
596 597

	if (cmdType == DTX_PROTOCOL_COMMAND_SUBTRANSACTION_BEGIN_INTERNAL)
A
Ashwin Agrawal 已提交
598
	{
599 600 601 602 603 604 605
		getTmLock();
		if (currentGxact->state == DTX_STATE_ACTIVE_NOT_DISTRIBUTED)
		{
			setCurrentGxactState(DTX_STATE_ACTIVE_DISTRIBUTED);
		}
		releaseTmLock();
	}
A
Ashwin Agrawal 已提交
606

607
	serializedDtxContextInfo = qdSerializeDtxContextInfo(
A
Ashwin Agrawal 已提交
608 609 610 611 612
														 &serializedDtxContextInfoLen,
														 false /* wantSnapshot */ ,
														 false /* inCursor */ ,
														 mppTxnOptions(true),
														 "doDispatchSubtransactionInternalCmd");
613 614

	succeeded = doDispatchDtxProtocolCommand(
A
Ashwin Agrawal 已提交
615 616 617 618
											 cmdType, /* flags */ 0,
											 currentGxact->gid, currentGxact->gxid,
											 &badGangs, /* raiseError */ true, &direct,
											 serializedDtxContextInfo, serializedDtxContextInfoLen);
619 620 621 622

	/* send a DTM command to others to tell them about the transaction */
	if (!succeeded)
	{
A
Ashwin Agrawal 已提交
623 624 625 626
		ereport(ERROR,
				(errmsg(
						"dispatching subtransaction internal command failed for gid = \"%s\" due to error",
						currentGxact->gid)));
627 628 629 630 631 632 633 634 635 636 637 638 639 640 641 642 643 644 645 646 647 648 649 650 651 652 653 654 655 656 657 658 659 660
	}

	return succeeded;
}

/*
 * The executor can avoid starting a distributed transaction if it knows that
 * the current dtx is clean and we aren't in a user-started global transaction.
 */
bool
isCurrentDtxTwoPhase(void)
{
	if (currentGxact == NULL)
	{
		return false;
	}
	else
	{
		return currentGxact->state == DTX_STATE_ACTIVE_DISTRIBUTED;
	}
}

DtxState
getCurrentDtxState(void)
{
	if (currentGxact != NULL)
		return currentGxact->state;

	return DTX_STATE_NONE;
}

static void
doPrepareTransaction(void)
{
A
Ashwin Agrawal 已提交
661 662
	bool		succeeded;
	CdbDispatchDirectDesc direct = default_dispatch_direct_desc;
663 664 665 666

	CHECK_FOR_INTERRUPTS();

	elog(DTM_DEBUG5, "doPrepareTransaction entering in state = %s",
A
Ashwin Agrawal 已提交
667
		 DtxStateToString(currentGxact->state));
668

A
Ashwin Agrawal 已提交
669 670 671 672
	/*
	 * Don't allow a cancel while we're dispatching our prepare (we wrap our
	 * state change as well; for good measure.
	 */
673 674 675 676 677 678
	HOLD_INTERRUPTS();

	copyDirectDispatchFromTransaction(&direct);

	getTmLock();
	Assert(currentGxact->state == DTX_STATE_ACTIVE_DISTRIBUTED);
A
Ashwin Agrawal 已提交
679
	setCurrentGxactState(DTX_STATE_PREPARING);
680 681 682 683 684 685 686 687
	releaseTmLock();

	elog(DTM_DEBUG5, "doPrepareTransaction moved to state = %s", DtxStateToString(currentGxact->state));

	succeeded = doDispatchDtxProtocolCommand(DTX_PROTOCOL_COMMAND_PREPARE, /* flags */ 0,
											 currentGxact->gid, currentGxact->gxid,
											 &currentGxact->badPrepareGangs, /* raiseError */ false, &direct, NULL, 0);

A
Ashwin Agrawal 已提交
688 689 690 691
	/*
	 * Now we've cleaned up our dispatched statement, cancels are allowed
	 * again.
	 */
692 693 694 695 696 697 698 699 700 701 702 703 704 705
	RESUME_INTERRUPTS();

	if (!succeeded)
	{
		elog(DTM_DEBUG5, "doPrepareTransaction error finds badPrimaryGangs = %s",
			 (currentGxact->badPrepareGangs ? "true" : "false"));
		elog(ERROR, "The distributed transaction 'Prepare' broadcast failed to one or more segments for gid = %s.",
			 currentGxact->gid);
	}
	elog(DTM_DEBUG5, "The distributed transaction 'Prepare' broadcast succeeded to the segments for gid = %s.",
		 currentGxact->gid);

	getTmLock();
	Assert(currentGxact->state == DTX_STATE_PREPARING);
A
Ashwin Agrawal 已提交
706
	setCurrentGxactState(DTX_STATE_PREPARED);
707 708
	releaseTmLock();

709
	SIMPLE_FAULT_INJECTOR(DtmBroadcastPrepare);
710 711 712 713 714 715 716 717 718 719 720 721 722 723 724

	elog(DTM_DEBUG5, "doPrepareTransaction leaving in state = %s", DtxStateToString(currentGxact->state));
}

/*
 * Insert FORGET COMMITTED into the xlog.
 * Call with both ProcArrayLock and DTM lock already held.
 */
static void
doInsertForgetCommitted(void)
{
	TMGXACT_LOG gxact_log;

	elog(DTM_DEBUG5, "doInsertForgetCommitted entering in state = %s", DtxStateToString(currentGxact->state));

A
Ashwin Agrawal 已提交
725
	setCurrentGxactState(DTX_STATE_INSERTING_FORGET_COMMITTED);
726 727 728

	if (strlen(currentGxact->gid) >= TMGIDSIZE)
		elog(PANIC, "Distribute transaction identifier too long (%d)",
A
Ashwin Agrawal 已提交
729
			 (int) strlen(currentGxact->gid));
730 731 732 733 734
	memcpy(&gxact_log.gid, currentGxact->gid, TMGIDSIZE);
	gxact_log.gxid = currentGxact->gxid;

	RecordDistributedForgetCommitted(&gxact_log);

A
Ashwin Agrawal 已提交
735
	setCurrentGxactState(DTX_STATE_INSERTED_FORGET_COMMITTED);
736 737

	/*
A
Ashwin Agrawal 已提交
738 739 740 741
	 * These two actions must be performed for a distributed transaction under
	 * the same locking of ProceArrayLock so the visibility of the transaction
	 * changes for local master readers (e.g. those using  SnapshotNow for
	 * reading) the same as for distributed transactions.
742
	 */
743
	ClearTransactionFromPgProc_UnderLock(MyProc, true);
744 745 746 747 748 749 750 751
	releaseGxact_UnderLocks();

	elog(DTM_DEBUG5, "doInsertForgetCommitted called releaseGxact");
}

static void
doNotifyingCommitPrepared(void)
{
A
Ashwin Agrawal 已提交
752 753 754
	bool		succeeded;
	bool		badGangs;
	int			retry = 0;
A
Ashwin Agrawal 已提交
755
	volatile int savedInterruptHoldoffCount;
756

A
Ashwin Agrawal 已提交
757
	CdbDispatchDirectDesc direct = default_dispatch_direct_desc;
758 759 760 761 762 763 764 765

	elog(DTM_DEBUG5, "doNotifyingCommitPrepared entering in state = %s", DtxStateToString(currentGxact->state));

	getTmLock();

	copyDirectDispatchFromTransaction(&direct);

	Assert(currentGxact->state == DTX_STATE_FORCED_COMMITTED);
A
Ashwin Agrawal 已提交
766
	setCurrentGxactState(DTX_STATE_NOTIFYING_COMMIT_PREPARED);
767 768 769 770
	releaseTmLock();

	if (strlen(currentGxact->gid) >= TMGIDSIZE)
		elog(PANIC, "Distribute transaction identifier too long (%d)",
A
Ashwin Agrawal 已提交
771
			 (int) strlen(currentGxact->gid));
772

773
	SIMPLE_FAULT_INJECTOR(DtmBroadcastCommitPrepared);
A
Ashwin Agrawal 已提交
774
	savedInterruptHoldoffCount = InterruptHoldoffCount;
775

776 777 778 779 780 781 782 783 784
	PG_TRY();
	{
		succeeded = doDispatchDtxProtocolCommand(DTX_PROTOCOL_COMMAND_COMMIT_PREPARED, /* flags */ 0,
												 currentGxact->gid, currentGxact->gxid,
												 &badGangs, /* raiseError */ false,
												 &direct, NULL, 0);
	}
	PG_CATCH();
	{
A
Ashwin Agrawal 已提交
785 786 787 788
		/*
		 * restore the previous value, which is reset to 0 in errfinish.
		 */
		InterruptHoldoffCount = savedInterruptHoldoffCount;
789 790 791 792
		succeeded = false;
	}
	PG_END_TRY();

793 794
	if (!succeeded)
	{
795 796 797 798
		getTmLock();
		Assert(currentGxact->state == DTX_STATE_NOTIFYING_COMMIT_PREPARED);
		elog(DTM_DEBUG5, "marking retry needed for distributed transaction"
			 " 'Commit Prepared' broadcast to the segments for gid = %s.",
799
			 currentGxact->gid);
800 801 802 803 804 805 806 807 808 809
		setCurrentGxactState(DTX_STATE_RETRY_COMMIT_PREPARED);
		setDistributedTransactionContext(DTX_CONTEXT_QD_RETRY_PHASE_2);
		releaseTmLock();
	}

	while (!succeeded && dtx_phase2_retry_count > retry++)
	{
		elog(WARNING, "the distributed transaction 'Commit Prepared' broadcast "
			 "failed to one or more segments for gid = %s.  Retrying ... try %d",
			 currentGxact->gid, retry);
810 811

		/*
A
Ashwin Agrawal 已提交
812 813
		 * We must succeed in delivering the commit to all segment instances,
		 * or any failed segment instances must be marked INVALID.
814 815
		 */
		elog(NOTICE, "Releasing segworker group to retry broadcast.");
816
		DisconnectAndDestroyAllGangs(true);
817 818

		/*
A
Ashwin Agrawal 已提交
819 820
		 * This call will at a minimum change the session id so we will not
		 * have SharedSnapshotAdd colissions.
821 822
		 */
		CheckForResetSession();
A
Ashwin Agrawal 已提交
823
		savedInterruptHoldoffCount = InterruptHoldoffCount;
824

825 826 827
		PG_TRY();
		{
			succeeded = doDispatchDtxProtocolCommand(
A
Ashwin Agrawal 已提交
828 829 830 831
													 DTX_PROTOCOL_COMMAND_RETRY_COMMIT_PREPARED, /* flags */ 0,
													 currentGxact->gid, currentGxact->gxid,
													 &badGangs, /* raiseError */ false,
													 &direct, NULL, 0);
832 833 834
		}
		PG_CATCH();
		{
A
Ashwin Agrawal 已提交
835 836 837 838
			/*
			 * restore the previous value, which is reset to 0 in errfinish.
			 */
			InterruptHoldoffCount = savedInterruptHoldoffCount;
839 840 841
			succeeded = false;
		}
		PG_END_TRY();
842 843
	}

844 845
	if (!succeeded)
		elog(PANIC, "unable to complete 'Commit Prepared' broadcast for gid = %s",
A
Ashwin Agrawal 已提交
846
			 currentGxact->gid);
847 848 849
	elog(DTM_DEBUG5, "the distributed transaction 'Commit Prepared' broadcast "
		 "succeeded to all the segments for gid = %s.", currentGxact->gid);

850
	/*
A
Ashwin Agrawal 已提交
851 852
	 * Global locking order: ProcArrayLock then DTM lock since calls
	 * doInsertForgetCommitted calls releaseGxact.
853 854 855 856 857 858 859 860 861 862 863 864
	 */
	LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);

	getTmLock();

	doInsertForgetCommitted();

	releaseTmLock();

	LWLockRelease(ProcArrayLock);
}

865 866 867
static void
retryAbortPrepared(void)
{
A
Ashwin Agrawal 已提交
868 869 870
	int			retry = 0;
	bool		succeeded = false;
	bool		badGangs = false;
A
Ashwin Agrawal 已提交
871
	volatile int savedInterruptHoldoffCount;
872 873 874 875 876 877

	CdbDispatchDirectDesc direct = default_dispatch_direct_desc;

	while (!succeeded && dtx_phase2_retry_count > retry++)
	{
		/*
A
Ashwin Agrawal 已提交
878 879 880
		 * By deallocating the gang, we will force a new gang to connect to
		 * all the segment instances.  And, we will abort the transactions in
		 * the segments. What's left are possibily prepared transactions.
881 882 883 884 885
		 */
		elog(NOTICE, "Releasing segworker groups to retry broadcast.");
		DisconnectAndDestroyAllGangs(true);

		/*
A
Ashwin Agrawal 已提交
886 887
		 * This call will at a minimum change the session id so we will not
		 * have SharedSnapshotAdd colissions.
888 889 890
		 */
		CheckForResetSession();

A
Ashwin Agrawal 已提交
891 892
		savedInterruptHoldoffCount = InterruptHoldoffCount;

893 894 895
		PG_TRY();
		{
			succeeded = doDispatchDtxProtocolCommand(
A
Ashwin Agrawal 已提交
896 897 898 899
													 DTX_PROTOCOL_COMMAND_RETRY_ABORT_PREPARED, /* flags */ 0,
													 currentGxact->gid, currentGxact->gxid,
													 &badGangs, /* raiseError */ false,
													 &direct, NULL, 0);
900 901 902 903 904 905 906
			if (!succeeded)
				elog(WARNING, "the distributed transaction 'Abort' broadcast "
					 "failed to one or more segments for gid = %s.  "
					 "Retrying ... try %d", currentGxact->gid, retry);
		}
		PG_CATCH();
		{
A
Ashwin Agrawal 已提交
907 908 909 910
			/*
			 * restore the previous value, which is reset to 0 in errfinish.
			 */
			InterruptHoldoffCount = savedInterruptHoldoffCount;
911 912 913 914 915 916 917 918 919 920 921 922 923
			succeeded = false;
		}
		PG_END_TRY();
	}

	if (!succeeded)
		elog(PANIC, "unable to complete 'Abort' broadcast for gid = %s",
			 currentGxact->gid);
	elog(DTM_DEBUG5, "The distributed transaction 'Abort' broadcast succeeded to "
		 "all the segments for gid = %s.", currentGxact->gid);
}


924 925 926
static void
doNotifyingAbort(void)
{
A
Ashwin Agrawal 已提交
927 928
	bool		succeeded;
	bool		badGangs;
A
Ashwin Agrawal 已提交
929
	volatile int savedInterruptHoldoffCount;
930

A
Ashwin Agrawal 已提交
931
	CdbDispatchDirectDesc direct = default_dispatch_direct_desc;
932 933 934 935 936

	elog(DTM_DEBUG5, "doNotifyingAborted entering in state = %s", DtxStateToString(currentGxact->state));

	getTmLock();
	Assert(currentGxact->state == DTX_STATE_NOTIFYING_ABORT_NO_PREPARED ||
A
Ashwin Agrawal 已提交
937 938
		   currentGxact->state == DTX_STATE_NOTIFYING_ABORT_SOME_PREPARED ||
		   currentGxact->state == DTX_STATE_NOTIFYING_ABORT_PREPARED);
939 940 941 942 943 944
	releaseTmLock();

	copyDirectDispatchFromTransaction(&direct);

	if (currentGxact->state == DTX_STATE_NOTIFYING_ABORT_NO_PREPARED)
	{
945
		if (GangsExist())
946 947 948 949 950 951 952 953 954 955 956
		{
			succeeded = doDispatchDtxProtocolCommand(DTX_PROTOCOL_COMMAND_ABORT_NO_PREPARED, /* flags */ 0,
													 currentGxact->gid, currentGxact->gxid,
													 &badGangs, /* raiseError */ false,
													 &direct, NULL, 0);
			if (!succeeded)
			{
				elog(WARNING, "The distributed transaction 'Abort' broadcast failed to one or more segments for gid = %s.",
					 currentGxact->gid);

				/*
A
Ashwin Agrawal 已提交
957 958
				 * Reset the dispatch logic and disconnect from any segment
				 * that didn't respond to our abort.
959 960
				 */
				elog(NOTICE, "Releasing segworker groups to finish aborting the transaction.");
961
				DisconnectAndDestroyAllGangs(true);
962 963

				/*
A
Ashwin Agrawal 已提交
964 965
				 * This call will at a minimum change the session id so we
				 * will not have SharedSnapshotAdd colissions.
966 967 968 969 970 971 972 973 974 975 976 977 978 979 980 981 982 983 984 985
				 */
				CheckForResetSession();
			}
			else
			{
				elog(DTM_DEBUG5,
					 "The distributed transaction 'Abort' broadcast succeeded to all the segments for gid = %s.",
					 currentGxact->gid);
			}
		}
		else
		{
			elog(DTM_DEBUG5,
				 "The distributed transaction 'Abort' broadcast was omitted (segworker group already dead) gid = %s.",
				 currentGxact->gid);
		}
	}
	else
	{
		DtxProtocolCommand dtxProtocolCommand;
A
Ashwin Agrawal 已提交
986 987
		char	   *abortString;
		int			retry = 0;
988 989

		Assert(currentGxact->state == DTX_STATE_NOTIFYING_ABORT_SOME_PREPARED ||
A
Ashwin Agrawal 已提交
990
			   currentGxact->state == DTX_STATE_NOTIFYING_ABORT_PREPARED);
991 992 993 994 995 996 997 998 999 1000 1001 1002

		if (currentGxact->state == DTX_STATE_NOTIFYING_ABORT_SOME_PREPARED)
		{
			dtxProtocolCommand = DTX_PROTOCOL_COMMAND_ABORT_SOME_PREPARED;
			abortString = "Abort [Prepared]";
		}
		else
		{
			dtxProtocolCommand = DTX_PROTOCOL_COMMAND_ABORT_PREPARED;
			abortString = "Abort Prepared";
		}

A
Ashwin Agrawal 已提交
1003 1004
		savedInterruptHoldoffCount = InterruptHoldoffCount;

1005
		PG_TRY();
1006
		{
1007 1008 1009 1010 1011 1012 1013
			succeeded = doDispatchDtxProtocolCommand(dtxProtocolCommand, /* flags */ 0,
													 currentGxact->gid, currentGxact->gxid,
													 &badGangs, /* raiseError */ false,
													 &direct, NULL, 0);
		}
		PG_CATCH();
		{
A
Ashwin Agrawal 已提交
1014 1015 1016 1017
			/*
			 * restore the previous value, which is reset to 0 in errfinish.
			 */
			InterruptHoldoffCount = savedInterruptHoldoffCount;
1018 1019 1020
			succeeded = false;
		}
		PG_END_TRY();
1021

1022 1023 1024 1025 1026
		if (!succeeded)
		{
			elog(WARNING, "the distributed transaction '%s' broadcast failed"
				 " to one or more segments for gid = %s.  Retrying ... try %d",
				 abortString, currentGxact->gid, retry);
1027 1028

			getTmLock();
A
Ashwin Agrawal 已提交
1029
			setCurrentGxactState(DTX_STATE_RETRY_ABORT_PREPARED);
1030
			setDistributedTransactionContext(DTX_CONTEXT_QD_RETRY_PHASE_2);
1031 1032
			releaseTmLock();
		}
1033
		retryAbortPrepared();
1034 1035
	}

1036
	SIMPLE_FAULT_INJECTOR(DtmBroadcastAbortPrepared);
1037 1038 1039 1040 1041 1042 1043 1044 1045

	/*
	 * Global locking order: ProcArrayLock then DTM lock.
	 */
	LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);

	getTmLock();

	Assert(currentGxact->state == DTX_STATE_NOTIFYING_ABORT_NO_PREPARED ||
A
Ashwin Agrawal 已提交
1046 1047
		   currentGxact->state == DTX_STATE_NOTIFYING_ABORT_SOME_PREPARED ||
		   currentGxact->state == DTX_STATE_NOTIFYING_ABORT_PREPARED ||
1048
		   currentGxact->state == DTX_STATE_RETRY_ABORT_PREPARED);
1049 1050 1051 1052 1053 1054 1055 1056 1057 1058 1059
	releaseGxact_UnderLocks();
	elog(DTM_DEBUG5, "doNotifyingAbort called releaseGxact");

	releaseTmLock();

	LWLockRelease(ProcArrayLock);
}

static bool
doNotifyCommittedInDoubt(char *gid)
{
A
Ashwin Agrawal 已提交
1060 1061
	bool		succeeded;
	bool		badGangs;
1062

A
Ashwin Agrawal 已提交
1063
	CdbDispatchDirectDesc direct = default_dispatch_direct_desc;
1064 1065 1066 1067 1068 1069 1070 1071 1072 1073 1074 1075 1076 1077 1078 1079 1080 1081 1082 1083 1084

	/* UNDONE: Pass real gxid instead of InvalidDistributedTransactionId. */
	succeeded = doDispatchDtxProtocolCommand(DTX_PROTOCOL_COMMAND_RECOVERY_COMMIT_PREPARED, /* flags */ 0,
											 gid, InvalidDistributedTransactionId,
											 &badGangs, /* raiseError */ false,
											 &direct, NULL, 0);
	if (!succeeded)
	{
		elog(FATAL, "Crash recovery broadcast of the distributed transaction 'Commit Prepared' broadcast failed to one or more segments for gid = %s.", gid);
	}
	else
	{
		elog(LOG, "Crash recovery broadcast of the distributed transaction 'Commit Prepared' broadcast succeeded for gid = %s.", gid);
	}

	return succeeded;
}

static void
doAbortInDoubt(char *gid)
{
A
Ashwin Agrawal 已提交
1085 1086
	bool		succeeded;
	bool		badGangs;
1087

A
Ashwin Agrawal 已提交
1088
	CdbDispatchDirectDesc direct = default_dispatch_direct_desc;
1089 1090 1091 1092 1093 1094 1095 1096 1097 1098 1099 1100 1101 1102 1103 1104 1105 1106 1107 1108 1109 1110 1111 1112 1113

	/* UNDONE: Pass real gxid instead of InvalidDistributedTransactionId. */
	succeeded = doDispatchDtxProtocolCommand(DTX_PROTOCOL_COMMAND_RECOVERY_ABORT_PREPARED, /* flags */ 0,
											 gid, InvalidDistributedTransactionId,
											 &badGangs, /* raiseError */ false,
											 &direct, NULL, 0);
	if (!succeeded)
	{
		elog(FATAL, "Crash recovery retry of the distributed transaction 'Abort Prepared' broadcast failed to one or more segments for gid = %s.  System will retry again later", gid);
	}
	else
	{
		elog(LOG, "Crash recovery broadcast of the distributed transaction 'Abort Prepared' broadcast succeeded for gid = %s", gid);
	}
}

/*
 * prepare a global transaction, called by user commit
 * or by CommitTransaction
 */
void
prepareDtxTransaction(void)
{
	if (DistributedTransactionContext != DTX_CONTEXT_QD_DISTRIBUTED_CAPABLE)
	{
A
Ashwin Agrawal 已提交
1114
		elog(DTM_DEBUG5, "prepareDtxTransaction nothing to do (DistributedTransactionContext = '%s')",
1115 1116 1117 1118 1119 1120 1121 1122 1123 1124 1125 1126 1127 1128 1129 1130 1131 1132 1133 1134 1135 1136 1137 1138 1139 1140 1141 1142 1143 1144 1145 1146 1147 1148 1149 1150 1151 1152 1153 1154
			 DtxContextToString(DistributedTransactionContext));
		return;
	}

	if (currentGxact == NULL)
	{
		return;
	}

	if (currentGxact->state == DTX_STATE_ACTIVE_NOT_DISTRIBUTED)
	{
		/*
		 * This transaction did not go distributed.
		 */
		elog(DTM_DEBUG5, "prepareDtxTransaction ignoring not distributed gid = %s", currentGxact->gid);
		releaseGxact();
		return;
	}

	elog(DTM_DEBUG5,
		 "prepareDtxTransaction called with state = %s",
		 DtxStateToString(currentGxact->state));

	Assert(currentGxact->state == DTX_STATE_ACTIVE_DISTRIBUTED);

	/*
	 * Broadcast PREPARE TRANSACTION to segments.
	 */
	doPrepareTransaction();
}

/*
 * rollback a global transaction, called by user rollback
 * or by AbortTransaction during Postgres automatic rollback
 */
void
rollbackDtxTransaction(void)
{
	if (DistributedTransactionContext != DTX_CONTEXT_QD_DISTRIBUTED_CAPABLE)
	{
A
Ashwin Agrawal 已提交
1155
		elog(DTM_DEBUG5, "rollbackDtxTransaction nothing to do (DistributedTransactionContext = '%s')",
1156 1157 1158 1159 1160
			 DtxContextToString(DistributedTransactionContext));
		return;
	}
	if (currentGxact == NULL)
	{
A
Ashwin Agrawal 已提交
1161
		elog(DTM_DEBUG5, "rollbackDtxTransaction nothing to do (currentGxact == NULL)");
1162 1163 1164 1165 1166 1167 1168 1169
		return;
	}

	elog(DTM_DEBUG5, "rollbackDtxTransaction called with state = %s, gid = %s",
		 DtxStateToString(currentGxact->state), currentGxact->gid);

	switch (currentGxact->state)
	{
A
Ashwin Agrawal 已提交
1170
		case DTX_STATE_ACTIVE_NOT_DISTRIBUTED:
1171 1172

			/*
A
Ashwin Agrawal 已提交
1173
			 * Let go of these...
1174
			 */
1175
			releaseGxact();
1176 1177
			return;

A
Ashwin Agrawal 已提交
1178 1179 1180
		case DTX_STATE_ACTIVE_DISTRIBUTED:
			setCurrentGxactState(DTX_STATE_NOTIFYING_ABORT_NO_PREPARED);
			break;
1181

A
Ashwin Agrawal 已提交
1182 1183 1184 1185
		case DTX_STATE_PREPARING:
			if (currentGxact->badPrepareGangs)
			{
				setCurrentGxactState(DTX_STATE_RETRY_ABORT_PREPARED);
1186

A
Ashwin Agrawal 已提交
1187 1188 1189 1190 1191 1192 1193 1194 1195 1196
				/*
				 * DisconnectAndDestroyAllGangs and ResetSession happens
				 * inside retryAbortPrepared.
				 */
				retryAbortPrepared();
				releaseGxact();
				return;
			}
			setCurrentGxactState(DTX_STATE_NOTIFYING_ABORT_SOME_PREPARED);
			break;
1197

A
Ashwin Agrawal 已提交
1198 1199 1200
		case DTX_STATE_PREPARED:
			setCurrentGxactState(DTX_STATE_NOTIFYING_ABORT_PREPARED);
			break;
1201

A
Ashwin Agrawal 已提交
1202
		case DTX_STATE_NOTIFYING_ABORT_NO_PREPARED:
1203

A
Ashwin Agrawal 已提交
1204 1205 1206 1207 1208 1209 1210
			/*
			 * By deallocating the gang, we will force a new gang to connect
			 * to all the segment instances.  And, we will abort the
			 * transactions in the segments.
			 */
			elog(NOTICE, "Releasing segworker groups to finish aborting the transaction.");
			DisconnectAndDestroyAllGangs(true);
1211

A
Ashwin Agrawal 已提交
1212 1213 1214 1215 1216 1217 1218 1219 1220 1221 1222 1223 1224 1225 1226 1227 1228 1229 1230 1231 1232 1233 1234 1235 1236 1237 1238 1239 1240 1241 1242 1243 1244
			/*
			 * This call will at a minimum change the session id so we will
			 * not have SharedSnapshotAdd colissions.
			 */
			CheckForResetSession();

			releaseGxact();
			return;

		case DTX_STATE_NOTIFYING_ABORT_SOME_PREPARED:
		case DTX_STATE_NOTIFYING_ABORT_PREPARED:
			elog(FATAL, "Unable to complete the 'Abort Prepared' broadcast for gid '%s'",
				 currentGxact->gid);
			break;

		case DTX_STATE_FORCED_COMMITTED:
		case DTX_STATE_NOTIFYING_COMMIT_PREPARED:
		case DTX_STATE_INSERTING_COMMITTED:
		case DTX_STATE_INSERTED_COMMITTED:
		case DTX_STATE_INSERTING_FORGET_COMMITTED:
		case DTX_STATE_INSERTED_FORGET_COMMITTED:
		case DTX_STATE_RETRY_COMMIT_PREPARED:
		case DTX_STATE_RETRY_ABORT_PREPARED:
		case DTX_STATE_CRASH_COMMITTED:
			elog(DTM_DEBUG5, "rollbackDtxTransaction dtx state \"%s\" not expected here",
				 DtxStateToString(currentGxact->state));
			releaseGxact();
			return;

		default:
			elog(PANIC, "Unrecognized dtx state: %d",
				 (int) currentGxact->state);
			break;
1245 1246 1247 1248
	}


	Assert(currentGxact->state == DTX_STATE_NOTIFYING_ABORT_NO_PREPARED ||
A
Ashwin Agrawal 已提交
1249 1250
		   currentGxact->state == DTX_STATE_NOTIFYING_ABORT_SOME_PREPARED ||
		   currentGxact->state == DTX_STATE_NOTIFYING_ABORT_PREPARED);
1251 1252 1253 1254 1255

	/*
	 * if the process is in the middle of blowing up... then we don't do
	 * anything here.  we can resolve any in-doubt transactions later.
	 *
A
Ashwin Agrawal 已提交
1256
	 * We can't dispatch -- but we *do* need to free up shared-memory entries.
1257 1258 1259 1260 1261 1262 1263 1264
	 */
	if (proc_exit_inprogress)
	{
		/*
		 * Unable to complete distributed abort broadcast with possible
		 * prepared transactions...
		 */
		if (currentGxact->state == DTX_STATE_NOTIFYING_ABORT_SOME_PREPARED ||
A
Ashwin Agrawal 已提交
1265
			currentGxact->state == DTX_STATE_NOTIFYING_ABORT_PREPARED)
1266 1267 1268 1269 1270 1271
		{
			elog(FATAL, "Unable to complete the 'Abort Prepared' broadcast for gid '%s'",
				 currentGxact->gid);
		}

		Assert(currentGxact->state == DTX_STATE_NOTIFYING_ABORT_NO_PREPARED);
A
Ashwin Agrawal 已提交
1272

1273
		/*
A
Ashwin Agrawal 已提交
1274 1275 1276
		 * By deallocating the gang, we will force a new gang to connect to
		 * all the segment instances.  And, we will abort the transactions in
		 * the segments.
1277
		 */
1278
		DisconnectAndDestroyAllGangs(true);
1279 1280

		/*
A
Ashwin Agrawal 已提交
1281 1282
		 * This call will at a minimum change the session id so we will not
		 * have SharedSnapshotAdd colissions.
1283 1284 1285 1286 1287 1288 1289 1290 1291 1292 1293 1294 1295 1296 1297 1298 1299 1300 1301 1302 1303 1304 1305 1306 1307 1308 1309 1310 1311 1312 1313
		 */
		CheckForResetSession();

		releaseGxact();
		return;
	}

	doNotifyingAbort();

	return;
}

/*
 * Error handling in initTM() is our caller.
 *
 * recoverTM() may throw errors.
 */
static void
initTM_recover_as_needed(void)
{
	Assert(shmTmRecoverred != NULL);

	/* Need to recover ? */
	if (!*shmTmRecoverred)
	{
		getTmLock();

		/* Still need to recover? */
		if (!*shmTmRecoverred)
		{
			volatile int savedInterruptHoldoffCount = InterruptHoldoffCount;
A
Ashwin Agrawal 已提交
1314

1315 1316 1317 1318 1319 1320 1321 1322 1323 1324 1325
			/*
			 * We have to catch errors here, otherwise the silly TmLock will
			 * stay in the backend process until this process goes away.
			 */
			PG_TRY();
			{
				recoverTM();
				*shmTmRecoverred = true;

				/*
				 * The in-doubt transactions are recovered. Perform
A
Ashwin Agrawal 已提交
1326 1327
				 * PersistentTable-Catalog non-database specific
				 * verifications, if requested and if needed
1328 1329 1330 1331 1332 1333 1334 1335 1336 1337 1338 1339 1340
				 */
				if (debug_persistent_ptcat_verification &&
					Persistent_PostDTMRecv_PTCatVerificationNeeded())
				{
					Persistent_PrintHash();
					Persistent_PostDTMRecv_NonDBSpecificPTCatVerification();
				}
			}
			PG_CATCH();
			{
				/*
				 * We can't simply use HOLD_INTERRUPTS as in LWLockRelease,
				 * because at this point we don't know if other LWLocks have
A
Ashwin Agrawal 已提交
1341 1342 1343 1344
				 * been acquired by myself.  Also, we don't know if
				 * releaseTmLock actually releases the lock, depending on
				 * ControlLockCount. Instead, restore the previous value,
				 * which is reset to 0 in errfinish.
1345 1346 1347 1348 1349 1350 1351 1352 1353 1354 1355 1356 1357 1358 1359 1360 1361
				 */
				InterruptHoldoffCount = savedInterruptHoldoffCount;
				releaseTmLock();

				/* Assuming we have a catcher above... */
				PG_RE_THROW();
			}
			PG_END_TRY();
		}

		releaseTmLock();
	}
}

static char *
getSuperuser(Oid *userOid)
{
A
Ashwin Agrawal 已提交
1362 1363 1364
	char	   *suser = NULL;
	Relation	auth_rel;
	HeapTuple	auth_tup;
1365 1366
	HeapScanDesc auth_scan;
	ScanKeyData key[2];
A
Ashwin Agrawal 已提交
1367
	bool		isNull;
1368 1369

	ScanKeyInit(&key[0],
A
Ashwin Agrawal 已提交
1370 1371 1372
				Anum_pg_authid_rolsuper,
				BTEqualStrategyNumber, F_BOOLEQ,
				BoolGetDatum(true));
1373 1374

	ScanKeyInit(&key[1],
A
Ashwin Agrawal 已提交
1375 1376 1377
				Anum_pg_authid_rolcanlogin,
				BTEqualStrategyNumber, F_BOOLEQ,
				BoolGetDatum(true));
1378 1379 1380 1381 1382

	auth_rel = heap_open(AuthIdRelationId, AccessShareLock);
	auth_scan = heap_beginscan(auth_rel, SnapshotNow, 2, key);

	while (HeapTupleIsValid(auth_tup = heap_getnext(auth_scan,
A
Ashwin Agrawal 已提交
1383
													ForwardScanDirection)))
1384
	{
A
Ashwin Agrawal 已提交
1385 1386
		Datum		attrName;
		Datum		attrNameOid;
1387 1388 1389 1390 1391 1392 1393 1394

		(void) heap_getattr(auth_tup, Anum_pg_authid_rolvaliduntil,
							auth_rel->rd_att, &isNull);
		/* we actually want it to be NULL, that means always valid */
		if (!isNull)
			continue;

		attrName = heap_getattr(auth_tup, Anum_pg_authid_rolname,
A
Ashwin Agrawal 已提交
1395
								auth_rel->rd_att, &isNull);
1396 1397 1398 1399 1400 1401

		Assert(!isNull);

		suser = pstrdup(DatumGetCString(attrName));

		attrNameOid = heap_getattr(auth_tup, ObjectIdAttributeNumber,
A
Ashwin Agrawal 已提交
1402
								   auth_rel->rd_att, &isNull);
1403 1404 1405 1406 1407 1408 1409 1410 1411 1412 1413 1414 1415 1416 1417
		Assert(!isNull);
		*userOid = DatumGetObjectId(attrNameOid);

		break;
	}

	heap_endscan(auth_scan);
	heap_close(auth_rel, AccessShareLock);

	return suser;
}

static char *
ChangeToSuperuser()
{
A
Ashwin Agrawal 已提交
1418 1419 1420
	char	   *olduser = NULL;
	char	   *newuser;
	Oid			userOid = InvalidOid;
1421 1422 1423 1424 1425 1426 1427 1428 1429 1430 1431 1432 1433 1434 1435 1436 1437 1438 1439 1440 1441 1442 1443 1444 1445 1446 1447 1448 1449 1450 1451 1452 1453 1454 1455 1456 1457 1458 1459 1460 1461 1462 1463 1464 1465 1466
	MemoryContext oldcontext;

	if (!IsAuthenticatedUserSuperUser())
	{
		oldcontext = MemoryContextSwitchTo(TopMemoryContext);
		newuser = getSuperuser(&userOid);
		MemoryContextSwitchTo(oldcontext);

		olduser = MyProcPort->user_name;
		SetSessionUserId(userOid, true);
		MyProcPort->user_name = newuser;
	}

	return olduser;
}

static void
RestoreToUser(char *olduser)
{
	MemoryContext oldcontext;

	if (!IsAuthenticatedUserSuperUser())
	{
		oldcontext = MemoryContextSwitchTo(TopMemoryContext);
		pfree(MyProcPort->user_name);
		MemoryContextSwitchTo(oldcontext);

		MyProcPort->user_name = olduser;
		SetSessionUserId(GetAuthenticatedUserId(), false);
	}
}

/*
 * Initialize TM, called by cdb_setup() for each QD process.
 *
 * First call to this function will trigger tm recovery.
 *
 * MPP-9894: in 4.0, if we've been started with enough segments to
 * run, but without having them in the right "roles" (see
 * gp_segment_configuration), we need to prober to convert them -- our
 * first attempt to dispatch will fail, we've got to catch that! The
 * retry should be fine, if not we're in serious "FATAL" trouble.
 */
void
initTM(void)
{
A
Ashwin Agrawal 已提交
1467 1468 1469 1470 1471
	char	   *olduser = NULL;
	bool		exists = false;
	MemoryContext oldcontext;
	bool		succeeded,
				first;
1472 1473 1474 1475 1476 1477 1478

	Assert(shmTmRecoverred != NULL);

	/* Need to recover ? */
	if (!*shmTmRecoverred)
	{
		/*
A
Ashwin Agrawal 已提交
1479 1480 1481 1482
		 * DTM initialization should be done in the context of the superuser,
		 * and not the user who initiated this backend (MPP-13866). Following
		 * code changes the context to superuser and and then restores it
		 * back.
1483 1484
		 */
		olduser = ChangeToSuperuser();
1485 1486

		SIMPLE_FAULT_INJECTOR(DtmInit);
1487 1488 1489 1490 1491 1492 1493 1494

		oldcontext = CurrentMemoryContext;
		succeeded = false;
		first = true;
		while (true)
		{
			/*
			 * MPP-9894: during startup, we don't have a top-level
A
Ashwin Agrawal 已提交
1495 1496
			 * PG_TRY/PG_CATCH block yet, the dispatcher may throw errors: we
			 * need to catch them.
1497 1498 1499 1500
			 */
			PG_TRY();
			{
				/*
A
Ashwin Agrawal 已提交
1501 1502
				 * FtsNotifyProber could throw ERROR, so we should catch it if
				 * it happens.
1503 1504
				 */
				if (!first)
G
Gang Xiong 已提交
1505
					FtsNotifyProber();
1506 1507 1508 1509 1510 1511 1512 1513 1514

				initTM_recover_as_needed();
				succeeded = true;
			}
			PG_CATCH();
			{
				MemoryContextSwitchTo(oldcontext);

				elog(LOG, "DTM initialization, caught exception: "
A
Ashwin Agrawal 已提交
1515
					 "looking for failed segments.");
1516 1517 1518

				/* Log the error. */
				EmitErrorReport();
1519 1520 1521 1522 1523 1524 1525 1526 1527 1528 1529 1530 1531 1532 1533 1534 1535 1536 1537 1538
				FlushErrorState();

				/*
				 * Keep going outside of PG_TRY block even if we want to
				 * retry; don't jumping out of this block without PG_END_TRY.
				 */
			}
			PG_END_TRY();

			if (!succeeded)
			{
				if (first)
				{
					first = false;
					continue;
				}
				else
				{
					elog(LOG, "DTM initialization, failed on retry.");
					elog(FATAL, "DTM initialization: failure during startup "
A
Ashwin Agrawal 已提交
1539
						 "recovery, retry failed, check segment status");
1540 1541 1542
				}
			}

1543
			Assert(!LWLockHeldByMe(shmControlLock));
A
Ashwin Agrawal 已提交
1544

1545 1546 1547 1548 1549 1550 1551 1552 1553 1554 1555 1556 1557 1558 1559
			/*
			 * We are done with the recovery.
			 */
			break;
		}

		RestoreToUser(olduser);

		freeGangsForPortal(NULL);
	}
	else
	{
		PG_TRY();
		{
			/*
A
Ashwin Agrawal 已提交
1560 1561 1562 1563
			 * Do Database-specific PTCat verification ? Yes, if 1- GUC is
			 * turned ON and if 2- DTM recovery INDEED happened and if 3-
			 * Current Database has not been verified before (If the Database
			 * is present in the Hash Table)
1564 1565 1566 1567 1568 1569 1570 1571 1572 1573 1574 1575 1576 1577
			 */
			Assert(MyDatabaseId != InvalidOid);
			if (debug_persistent_ptcat_verification &&
				Persistent_PostDTMRecv_PTCatVerificationNeeded())
			{
				getTmLock();
				Persistent_PostDTMRecv_LookupHashEntry(MyDatabaseId, &exists);
				if (exists)
				{
					olduser = ChangeToSuperuser();

					Persistent_PostDTMRecv_DBSpecificPTCatVerification();

					/*
A
Ashwin Agrawal 已提交
1578 1579 1580 1581
					 * The current database is now verified. Remove its entry
					 * from the Hash table to avoid performing these
					 * verifications next time a session connects to the
					 * current database
1582 1583 1584 1585 1586 1587 1588 1589 1590 1591 1592 1593
					 */
					Persistent_PostDTMRecv_RemoveHashEntry(MyDatabaseId);

					RestoreToUser(olduser);
					freeGangsForPortal(NULL);
				}
				releaseTmLock();
			}
		}
		PG_CATCH();
		{
			elog(FATAL, " Failure during DTM Post Recovery PersistentTables-Catalog"
A
Ashwin Agrawal 已提交
1594
				 " DB-specific Verification");
1595 1596 1597 1598 1599 1600 1601 1602 1603 1604 1605 1606 1607 1608 1609 1610 1611 1612 1613 1614 1615 1616 1617 1618 1619 1620 1621 1622
		}
		PG_END_TRY();
	}
}

/* get tm share memory size */
int
tmShmemSize(void)
{
	if ((Gp_role != GP_ROLE_DISPATCH) && (Gp_role != GP_ROLE_UTILITY))
		return 0;

	return
		MAXALIGN(TMCONTROLBLOCK_BYTES(max_tm_gxacts) + max_tm_gxacts * sizeof(TMGXACT));
}


/*
 * tmShmemInit - should be called only once from postmaster and inherit by all
 * postgres processes
 */
void
tmShmemInit(void)
{
	bool		found;
	TmControlBlock *shared;

	/*
A
Ashwin Agrawal 已提交
1623 1624 1625 1626
	 * max_prepared_xacts is a guc which is postmaster-startup setable -- it
	 * can only be updated by restarting the system. Global transactions will
	 * all use two-phase commit, so the number of global transactions is bound
	 * to the number of prepared.
1627 1628 1629 1630 1631 1632 1633 1634 1635 1636 1637 1638 1639 1640 1641 1642 1643 1644
	 */
	max_tm_gxacts = max_prepared_xacts;

	if ((Gp_role != GP_ROLE_DISPATCH) && (Gp_role != GP_ROLE_UTILITY))
		return;

	shared = (TmControlBlock *) ShmemInitStruct("Transaction manager", tmShmemSize(), &found);
	if (!shared)
		elog(FATAL, "could not initialize transaction manager share memory");

	shmControlLock = shared->ControlLock;
	shmTmRecoverred = &shared->recoverred;
	shmDistribTimeStamp = &shared->distribTimeStamp;
	shmGIDSeq = &shared->seqno;
	/* Only initialize this if we are the creator of the shared memory */
	if (!found)
	{
		time_t		t = time(NULL);
A
Ashwin Agrawal 已提交
1645

1646 1647 1648 1649 1650
		if (t == (time_t) -1)
		{
			elog(PANIC, "cannot generate global transaction id");
		}

A
Ashwin Agrawal 已提交
1651 1652
		*shmDistribTimeStamp = (DistributedTransactionTimeStamp) t;
		elog(DEBUG1, "DTM start timestamp %u", *shmDistribTimeStamp);
1653 1654 1655 1656 1657 1658 1659 1660 1661 1662 1663

		*shmGIDSeq = FirstDistributedTransactionId;
	}
	shmDtmStarted = &shared->DtmStarted;
	shmNextSnapshotId = &shared->NextSnapshotId;
	shmNumGxacts = &shared->num_active_xacts;
	shmGxactArray = shared->gxact_array;

	if (!IsUnderPostmaster)
		/* Initialize locks and shared memory area */
	{
A
Ashwin Agrawal 已提交
1664
		int			i = 0;
1665 1666 1667 1668 1669 1670 1671 1672 1673 1674 1675 1676 1677 1678 1679 1680 1681 1682
		TMGXACT    *gxact = NULL;

		shared->ControlLock = LWLockAssign();
		shmControlLock = shared->ControlLock;

		/* initialize gxact array */
		gxact = (TMGXACT *) (shmGxactArray + max_tm_gxacts);
		for (i = 0; i < max_tm_gxacts; i++)
		{
			gxact->debugIndex = i;
			shmGxactArray[i] = gxact++;
		}
	}
}

/*
 * restore global transaction during tm log recovery
 */
1683
static void
A
Ashwin Agrawal 已提交
1684
restoreGxact(TMGXACT_LOG *gxact_log, DtxState state)
1685
{
A
Ashwin Agrawal 已提交
1686
	Assert(gxact_log != NULL);
1687 1688 1689 1690 1691 1692

	initGxact(currentGxact);

	/*
	 * Copy the log fields.
	 *
A
Ashwin Agrawal 已提交
1693 1694
	 * The length check here requires the identifer have a trailing NUL
	 * character.
1695 1696 1697
	 */
	if (strlen(gxact_log->gid) >= TMGIDSIZE)
		elog(PANIC, "Distribute transaction identifier too long (%d)",
A
Ashwin Agrawal 已提交
1698
			 (int) strlen(gxact_log->gid));
1699 1700 1701
	memcpy(currentGxact->gid, gxact_log->gid, TMGIDSIZE);
	currentGxact->gxid = gxact_log->gxid;

A
Ashwin Agrawal 已提交
1702
	setCurrentGxactState(state);
1703 1704 1705 1706 1707 1708 1709 1710 1711 1712
}

/* mppTxnOptions:
 * Generates an int containing the appropriate flags to direct the remote
 * segdb QE process to perform any needed transaction commands before or
 * after the statement.
 */
int
mppTxnOptions(bool needTwoPhase)
{
A
Ashwin Agrawal 已提交
1713
	int			options = 0;
1714 1715 1716 1717 1718 1719 1720 1721 1722 1723 1724 1725 1726 1727 1728 1729 1730 1731 1732 1733 1734 1735 1736 1737 1738 1739 1740 1741 1742 1743 1744 1745 1746 1747 1748 1749 1750 1751 1752

	elog(DTM_DEBUG5,
		 "mppTxnOptions DefaultXactIsoLevel = %s, DefaultXactReadOnly = %s, XactIsoLevel = %s, XactReadOnly = %s.",
		 IsoLevelAsUpperString(DefaultXactIsoLevel), (DefaultXactReadOnly ? "true" : "false"),
		 IsoLevelAsUpperString(XactIsoLevel), (XactReadOnly ? "true" : "false"));

	if (needTwoPhase)
		options |= GP_OPT_NEED_TWO_PHASE;

	if (XactIsoLevel == XACT_READ_COMMITTED)
		options |= GP_OPT_READ_COMMITTED;
	else if (XactIsoLevel == XACT_REPEATABLE_READ)
		options |= GP_OPT_REPEATABLE_READ;
	else if (XactIsoLevel == XACT_SERIALIZABLE)
		options |= GP_OPT_SERIALIZABLE;

	if (XactReadOnly)
		options |= GP_OPT_READ_ONLY;

	if (currentGxact != NULL && currentGxact->explicitBeginRemembered)
		options |= GP_OPT_EXPLICT_BEGIN;

	elog(DTM_DEBUG5,
		 "mppTxnOptions txnOptions = 0x%x, needTwoPhase = %s, explicitBegin = %s, isoLevel = %s, readOnly = %s.",
		 options,
		 (isMppTxOptions_NeedTwoPhase(options) ? "true" : "false"), (isMppTxOptions_ExplicitBegin(options) ? "true" : "false"),
		 IsoLevelAsUpperString(mppTxOptions_IsoLevel(options)), (isMppTxOptions_ReadOnly(options) ? "true" : "false"));

	return options;

}

int
mppTxOptions_IsoLevel(int txnOptions)
{
	if (txnOptions & GP_OPT_READ_COMMITTED)
		return XACT_READ_COMMITTED;
	else if (txnOptions & GP_OPT_SERIALIZABLE)
		return XACT_SERIALIZABLE;
A
Ashwin Agrawal 已提交
1753
	else if (txnOptions & GP_OPT_REPEATABLE_READ)
1754 1755 1756 1757 1758 1759 1760 1761 1762 1763 1764 1765 1766 1767 1768 1769 1770 1771 1772 1773 1774 1775 1776 1777 1778 1779 1780 1781 1782 1783 1784 1785 1786 1787 1788 1789 1790 1791 1792 1793 1794 1795 1796 1797 1798 1799 1800 1801 1802 1803 1804 1805 1806 1807 1808 1809 1810 1811 1812 1813 1814 1815 1816 1817 1818 1819 1820 1821 1822
		return XACT_REPEATABLE_READ;
	else
		return XACT_READ_UNCOMMITTED;
}

bool
isMppTxOptions_ReadOnly(int txnOptions)
{
	return ((txnOptions & GP_OPT_READ_ONLY) != 0);
}



/* unpackMppTxnOptions:
 * Unpack an int containing the appropriate flags to direct the remote
 * segdb QE process to perform any needed transaction commands before or
 * after the statement.
 */
void
unpackMppTxnOptions(int txnOptions, int *isoLevel, bool *readOnly)
{
	*isoLevel = mppTxOptions_IsoLevel(txnOptions);

	*readOnly = isMppTxOptions_ReadOnly(txnOptions);
}

/* isMppTxOptions_StatementWantsDtxTransaction:
 * Return the NeedTwoPhase flag.
 */
bool
isMppTxOptions_NeedTwoPhase(int txnOptions)
{
	return ((txnOptions & GP_OPT_NEED_TWO_PHASE) != 0);
}

/* isMppTxOptions_ExplicitBegin:
 * Return the ExplicitBegin flag.
 */
bool
isMppTxOptions_ExplicitBegin(int txnOptions)
{
	return ((txnOptions & GP_OPT_EXPLICT_BEGIN) != 0);
}


/* acquire tm lw lock */
void
getTmLock(void)
{

	if (ControlLockCount++ == 0)
		LWLockAcquire(shmControlLock, LW_EXCLUSIVE);
}

/* release tm lw lock */
void
releaseTmLock(void)
{
	if (--ControlLockCount == 0)
		LWLockRelease(shmControlLock);

}

/*
 * Redo transaction commit log record.
 */
void
redoDtxCheckPoint(TMGXACT_CHECKPOINT *gxact_checkpoint)
{
A
Ashwin Agrawal 已提交
1823
	int			committedCount;
1824

A
Ashwin Agrawal 已提交
1825
	int			i;
1826 1827

	/*
A
Ashwin Agrawal 已提交
1828 1829
	 * For checkpoint same as REDO, lets add entries to file in utility and
	 * in-memory if Dispatch.
1830 1831 1832
	 */

	committedCount = gxact_checkpoint->committedCount;
1833
	elog(DTM_DEBUG5, "redoDtxCheckPoint has committedCount = %d", committedCount);
1834 1835 1836 1837 1838 1839 1840 1841 1842 1843 1844 1845 1846 1847 1848 1849 1850
	if (Debug_persistent_recovery_print)
	{
		elog(PersistentRecovery_DebugPrintLevel(),
			 "redoDtxCheckPoint: committedCount = %d",
			 committedCount);
	}

	for (i = 0; i < committedCount; i++)
	{
		redoDistributedCommitRecord(&gxact_checkpoint->committedGxactArray[i]);
	}
}

static void
GetRedoFileName(char *path)
{
	snprintf(path, MAXPGPATH,
A
Ashwin Agrawal 已提交
1851 1852
			 "%s/" UTILITYMODEDTMREDO_DIR "/" UTILITYMODEDTMREDO_FILE, DataDir);
	elog(DTM_DEBUG3, "Returning save DTM redo file path = %s", path);
1853 1854 1855 1856 1857 1858 1859 1860 1861 1862 1863 1864 1865 1866 1867 1868 1869 1870 1871 1872 1873 1874 1875 1876 1877 1878 1879 1880 1881 1882 1883 1884 1885
}

void
UtilityModeFindOrCreateDtmRedoFile(void)
{
	char		path[MAXPGPATH];

	if (Gp_role != GP_ROLE_UTILITY)
	{
		elog(DTM_DEBUG3, "Not in Utility Mode (role = %s) -- skipping finding or creating DTM redo file",
			 role_to_string(Gp_role));
		return;
	}
	GetRedoFileName(path);

	redoFileFD = open(path, O_RDWR | O_CREAT, S_IRUSR | S_IWUSR);
	if (redoFileFD < 0)
	{
		ereport(ERROR,
				(errcode_for_file_access(),
				 errmsg("could not create save DTM redo file \"%s\"",
						path)));
	}

	redoFileOffset = lseek(redoFileFD, 0, SEEK_END);
	elog(DTM_DEBUG3, "Succesfully opened DTM redo file %s (end offset %d)",
		 path, redoFileOffset);
}

/*
 *
 */
static void
A
Ashwin Agrawal 已提交
1886
UtilityModeSaveRedo(bool committed, TMGXACT_LOG *gxact_log)
1887 1888
{
	TMGXACT_UTILITY_MODE_REDO utilityModeRedo;
A
Ashwin Agrawal 已提交
1889
	int			write_len;
1890 1891 1892 1893 1894

	utilityModeRedo.committed = committed;
	memcpy(&utilityModeRedo.gxact_log, gxact_log, sizeof(TMGXACT_LOG));

	elog(DTM_DEBUG5, "Writing {committed = %s, gid = %s, gxid = %u} to DTM redo file",
A
Ashwin Agrawal 已提交
1895 1896 1897
		 (utilityModeRedo.committed ? "true" : "false"),
		 utilityModeRedo.gxact_log.gid,
		 utilityModeRedo.gxact_log.gxid);
1898 1899 1900 1901 1902 1903 1904 1905 1906 1907 1908 1909 1910 1911 1912 1913 1914 1915 1916 1917 1918 1919 1920 1921 1922 1923 1924 1925 1926 1927 1928

	write_len = write(redoFileFD, &utilityModeRedo, sizeof(TMGXACT_UTILITY_MODE_REDO));
	if (write_len != sizeof(TMGXACT_UTILITY_MODE_REDO))
	{
		ereport(ERROR,
				(errcode_for_file_access(),
				 errmsg("could not write save DTM redo file : %m")));
	}

}

void
UtilityModeCloseDtmRedoFile(void)
{
	if (Gp_role != GP_ROLE_UTILITY)
	{
		elog(DTM_DEBUG3, "Not in Utility Mode (role = %s)-- skipping closing DTM redo file",
			 role_to_string(Gp_role));
		return;
	}
	elog(DTM_DEBUG3, "Closing DTM redo file");
	close(redoFileFD);
}

static void
ReplayRedoFromUtilityMode(void)
{
	TMGXACT_UTILITY_MODE_REDO utilityModeRedo;

	int			fd;
	int			read_len;
A
Ashwin Agrawal 已提交
1929
	int			errno;
1930 1931 1932 1933 1934 1935 1936 1937 1938 1939 1940 1941
	char		path[MAXPGPATH];
	int			entries;

	entries = 0;

	GetRedoFileName(path);

	fd = open(path, O_RDONLY, 0);
	if (fd < 0)
	{
		/* UNDONE: Distinquish "not found" from other errors. */
		elog(DTM_DEBUG3, "Could not open DTM redo file %s for reading",
A
Ashwin Agrawal 已提交
1942
			 path);
1943 1944 1945 1946 1947 1948
		return;
	}

	elog(DTM_DEBUG3, "Succesfully opened DTM redo file %s for reading",
		 path);

A
Ashwin Agrawal 已提交
1949
	while (true)
1950 1951 1952 1953 1954 1955 1956 1957
	{
		errno = 0;
		read_len = read(fd, &utilityModeRedo, sizeof(TMGXACT_UTILITY_MODE_REDO));

		if (read_len == 0)
			break;
		else if (read_len != sizeof(TMGXACT_UTILITY_MODE_REDO) && errno == 0)
			elog(ERROR, "Bad redo length (expected %d and found %d)",
A
Ashwin Agrawal 已提交
1958
				 (int) sizeof(TMGXACT_UTILITY_MODE_REDO), read_len);
1959 1960 1961 1962 1963 1964 1965 1966 1967
		else if (errno != 0)
		{
			close(fd);
			ereport(ERROR,
					(errcode_for_file_access(),
					 errmsg("error reading DTM redo file: %m")));
		}

		elog(DTM_DEBUG5, "Read {committed = %s, gid = %s, gxid = %u} from DTM redo file",
A
Ashwin Agrawal 已提交
1968 1969 1970
			 (utilityModeRedo.committed ? "true" : "false"),
			 utilityModeRedo.gxact_log.gid,
			 utilityModeRedo.gxact_log.gxid);
1971 1972 1973 1974 1975 1976 1977 1978 1979 1980 1981 1982 1983 1984 1985 1986 1987 1988 1989 1990 1991 1992
		if (utilityModeRedo.committed)
		{
			redoDistributedCommitRecord(&utilityModeRedo.gxact_log);
		}
		else
		{
			redoDistributedForgetCommitRecord(&utilityModeRedo.gxact_log);
		}

		entries++;
	}

	elog(DTM_DEBUG5, "Processed %d entries from DTM redo file",
		 entries);
	close(fd);

}

static void
RemoveRedoUtilityModeFile(void)
{
	char		path[MAXPGPATH];
A
Ashwin Agrawal 已提交
1993
	bool		removed;
1994 1995 1996 1997 1998 1999 2000 2001 2002 2003 2004

	GetRedoFileName(path);
	removed = (unlink(path) == 0);
	elog(DTM_DEBUG5, "Removed DTM redo file %s (%s)",
		 path, (removed ? "true" : "false"));
}

/*
 * Redo transaction commit log record.
 */
void
A
Ashwin Agrawal 已提交
2005
redoDistributedCommitRecord(TMGXACT_LOG *gxact_log)
2006 2007 2008 2009 2010
{

	int			i;

	/*
A
Ashwin Agrawal 已提交
2011 2012
	 * The length check here requires the identifer have a trailing NUL
	 * character.
2013 2014 2015
	 */
	if (strlen(gxact_log->gid) >= TMGIDSIZE)
		elog(PANIC, "Distribute transaction identifier too long (%d)",
A
Ashwin Agrawal 已提交
2016
			 (int) strlen(gxact_log->gid));
2017 2018 2019 2020 2021 2022 2023 2024 2025 2026 2027 2028 2029 2030 2031 2032 2033 2034 2035 2036 2037 2038 2039 2040 2041 2042 2043 2044 2045 2046 2047 2048 2049 2050 2051 2052

	if (Gp_role == GP_ROLE_UTILITY)
	{
		elog(DTM_DEBUG3, "DB in Utility mode.  Save DTM distributed commit until later.");
		UtilityModeSaveRedo(true, gxact_log);
		return;
	}

	for (i = 0; i < *shmNumGxacts; i++)
	{
		if (strcmp(gxact_log->gid, shmGxactArray[i]->gid) == 0)
		{
			/* found an active global transaction */
			currentGxact = shmGxactArray[i];
			break;
		}
	}
	if (i == *shmNumGxacts)
	{
		/*
		 * Transaction not found, this is the first log of this transaction.
		 */
		if (*shmNumGxacts >= max_tm_gxacts)
		{
			ereport(FATAL,
					(errmsg("the limit of %d distributed transactions has been reached.",
							max_tm_gxacts),
					 errdetail("The global user configuration (GUC) server parameter max_prepared_transactions controls this limit.")));
		}

		currentGxact = shmGxactArray[(*shmNumGxacts)++];
	}

	restoreGxact(gxact_log, DTX_STATE_CRASH_COMMITTED);

	elog((Debug_print_full_dtm ? LOG : DEBUG5),
A
Ashwin Agrawal 已提交
2053
		 "Crash recovery redo added committed distributed transaction gid = %s", currentGxact->gid);
2054 2055

	elog((Debug_print_full_dtm ? INFO : DEBUG5),
A
Ashwin Agrawal 已提交
2056
		 "Crash recovery redo added committed distributed transaction gid = %s", currentGxact->gid);
2057 2058 2059 2060

	/*
	 * Don't leave the currentGxact point in-use.
	 */
A
Ashwin Agrawal 已提交
2061
	currentGxact = NULL;
2062 2063 2064 2065 2066 2067
}

/*
 * Redo transaction forget commit log record.
 */
void
A
Ashwin Agrawal 已提交
2068
redoDistributedForgetCommitRecord(TMGXACT_LOG *gxact_log)
2069 2070 2071 2072 2073 2074 2075 2076 2077 2078 2079 2080 2081 2082 2083 2084 2085 2086
{
	int			i;

	if (Gp_role == GP_ROLE_UTILITY)
	{
		elog(DTM_DEBUG3, "DB in Utility mode.  Save DTM disributed forget until later.");
		UtilityModeSaveRedo(false, gxact_log);
		return;
	}

	for (i = 0; i < *shmNumGxacts; i++)
	{
		if (strcmp(gxact_log->gid, shmGxactArray[i]->gid) == 0)
		{
			/* found an active global transaction */
			currentGxact = shmGxactArray[i];
			elog((Debug_print_full_dtm ? INFO : DEBUG5),
				 "Crash recovery redo removed committed distributed transaction gid = %s for forget",
A
Ashwin Agrawal 已提交
2087
				 currentGxact->gid);
2088 2089 2090 2091 2092 2093 2094 2095 2096 2097 2098 2099
			releaseGxact();
			return;
		}
	}

	elog((Debug_print_full_dtm ? WARNING : DEBUG5),
		 "Crash recovery redo did not find committed distributed transaction gid = %s for forget",
		 gxact_log->gid);

}

static void
A
Ashwin Agrawal 已提交
2100
descGxactLog(StringInfo buf, TMGXACT_LOG *gxact_log)
2101 2102 2103 2104 2105 2106 2107 2108 2109
{
	appendStringInfo(buf, " gid = %s, gxid = %u",
					 gxact_log->gid, gxact_log->gxid);
}

/*
 * Describe redo transaction commit log record.
 */
void
A
Ashwin Agrawal 已提交
2110
descDistributedCommitRecord(StringInfo buf, TMGXACT_LOG *gxact_log)
2111 2112 2113 2114 2115 2116 2117 2118
{
	descGxactLog(buf, gxact_log);
}

/*
 * Describe redo transaction forget commit log record.
 */
void
A
Ashwin Agrawal 已提交
2119
descDistributedForgetCommitRecord(StringInfo buf, TMGXACT_LOG *gxact_log)
2120 2121 2122 2123 2124 2125 2126 2127 2128 2129 2130
{
	descGxactLog(buf, gxact_log);
}


/*=========================================================================
 * HELPER FUNCTIONS
 */

static bool
doDispatchDtxProtocolCommand(DtxProtocolCommand dtxProtocolCommand, int flags,
A
Ashwin Agrawal 已提交
2131
							 char *gid, DistributedTransactionId gxid,
2132 2133
							 bool *badGangs, bool raiseError,
							 CdbDispatchDirectDesc *direct,
2134 2135
							 char *serializedDtxContextInfo,
							 int serializedDtxContextInfoLen)
2136
{
A
Ashwin Agrawal 已提交
2137 2138 2139
	int			i,
				resultCount,
				numOfFailed = 0;
2140

A
Ashwin Agrawal 已提交
2141
	char	   *dtxProtocolCommandStr = 0;
2142

2143 2144
	struct pg_result **results = NULL;
	StringInfoData errbuf;
2145 2146 2147

	dtxProtocolCommandStr = DtxProtocolCommandToString(dtxProtocolCommand);

A
Ashwin Agrawal 已提交
2148
	if (Test_print_direct_dispatch_info)
2149
	{
A
Ashwin Agrawal 已提交
2150
		if (direct->directed_dispatch)
2151
			elog(INFO, "Distributed transaction command '%s' to SINGLE content", dtxProtocolCommandStr);
A
Ashwin Agrawal 已提交
2152 2153
		else
			elog(INFO, "Distributed transaction command '%s' to ALL contents", dtxProtocolCommandStr);
2154 2155 2156 2157
	}
	elog(DTM_DEBUG5,
		 "dispatchDtxProtocolCommand: %d ('%s'), direct content #: %d",
		 dtxProtocolCommand, dtxProtocolCommandStr,
A
Ashwin Agrawal 已提交
2158
		 direct->directed_dispatch ? direct->content[0] : -1);
2159 2160

	initStringInfo(&errbuf);
2161
	results = CdbDispatchDtxProtocolCommand(dtxProtocolCommand, flags,
A
Ashwin Agrawal 已提交
2162 2163 2164 2165
											dtxProtocolCommandStr,
											gid, gxid,
											&errbuf, &resultCount, badGangs, direct,
											serializedDtxContextInfo, serializedDtxContextInfoLen);
2166 2167 2168 2169

	if (errbuf.len > 0)
	{
		ereport((raiseError ? ERROR : LOG),
2170
				(errmsg("DTM error (gathered results from cmd '%s')", dtxProtocolCommandStr),
2171 2172 2173 2174 2175 2176
				 errdetail("%s", errbuf.data)));
		return false;
	}

	if (results == NULL)
	{
A
Ashwin Agrawal 已提交
2177 2178
		numOfFailed++;			/* If we got no results, we need to treat it
								 * as an error! */
2179
	}
2180

2181 2182
	for (i = 0; i < resultCount; i++)
	{
A
Ashwin Agrawal 已提交
2183 2184
		char	   *cmdStatus;
		ExecStatusType resultStatus;
2185

A
Ashwin Agrawal 已提交
2186 2187 2188 2189
		/*
		 * note: PQresultStatus() is smart enough to deal with results[i] ==
		 * NULL
		 */
2190 2191 2192 2193 2194 2195 2196 2197 2198 2199 2200 2201 2202 2203 2204 2205 2206 2207 2208 2209 2210 2211 2212 2213 2214 2215 2216 2217 2218
		resultStatus = PQresultStatus(results[i]);
		if (resultStatus != PGRES_COMMAND_OK &&
			resultStatus != PGRES_TUPLES_OK)
		{
			numOfFailed++;
		}
		else
		{
			/*
			 * success ? If an error happened during a transaction which
			 * hasn't already been caught when we try a prepare we'll get a
			 * rollback from our prepare ON ONE SEGMENT: so we go look at the
			 * status, otherwise we could issue a COMMIT when we don't want
			 * to!
			 */
			cmdStatus = PQcmdStatus(results[i]);

			elog(DEBUG3, "DTM: status message cmd '%s' [%d] result '%s'", dtxProtocolCommandStr, i, cmdStatus);
			if (strncmp(cmdStatus, dtxProtocolCommandStr, strlen(cmdStatus)) != 0)
			{
				/* failed */
				numOfFailed++;
			}
		}
	}

	/* discard the errbuf text */
	pfree(errbuf.data);

2219 2220
	for (i = 0; i < resultCount; i++)
		PQclear(results[i]);
2221 2222 2223

	if (results)
		free(results);
2224 2225 2226 2227 2228 2229

	return (numOfFailed == 0);
}


bool
2230
dispatchDtxCommand(const char *cmd)
2231
{
A
Ashwin Agrawal 已提交
2232 2233
	int			i,
				numOfFailed = 0;
2234

2235
	CdbPgResults cdb_pgresults = {NULL, 0};
2236 2237 2238

	elog(DTM_DEBUG5, "dispatchDtxCommand: '%s'", cmd);

2239
	CdbDispatchCommand(cmd, DF_NONE, &cdb_pgresults);
2240

2241
	if (cdb_pgresults.numResults == 0)
2242
	{
A
Ashwin Agrawal 已提交
2243 2244
		return false;			/* If we got no results, we need to treat it
								 * as an error! */
2245 2246
	}

2247
	for (i = 0; i < cdb_pgresults.numResults; i++)
2248
	{
A
Ashwin Agrawal 已提交
2249 2250
		char	   *cmdStatus;
		ExecStatusType resultStatus;
2251

A
Ashwin Agrawal 已提交
2252 2253 2254 2255
		/*
		 * note: PQresultStatus() is smart enough to deal with results[i] ==
		 * NULL
		 */
2256
		resultStatus = PQresultStatus(cdb_pgresults.pg_results[i]);
2257 2258 2259 2260 2261 2262 2263 2264 2265 2266 2267 2268 2269 2270
		if (resultStatus != PGRES_COMMAND_OK &&
			resultStatus != PGRES_TUPLES_OK)
		{
			numOfFailed++;
		}
		else
		{
			/*
			 * success ? If an error happened during a transaction which
			 * hasn't already been caught when we try a prepare we'll get a
			 * rollback from our prepare ON ONE SEGMENT: so we go look at the
			 * status, otherwise we could issue a COMMIT when we don't want
			 * to!
			 */
2271
			cmdStatus = PQcmdStatus(cdb_pgresults.pg_results[i]);
2272 2273 2274 2275 2276 2277 2278 2279 2280 2281

			elog(DEBUG3, "DTM: status message cmd '%s' [%d] result '%s'", cmd, i, cmdStatus);
			if (strncmp(cmdStatus, cmd, strlen(cmdStatus)) != 0)
			{
				/* failed */
				numOfFailed++;
			}
		}
	}

2282
	cdbdisp_clearCdbPgResults(&cdb_pgresults);
2283 2284 2285 2286 2287 2288

	return (numOfFailed == 0);
}

/* initialize a global transaction context */
static void
A
Ashwin Agrawal 已提交
2289
initGxact(TMGXACT *gxact)
2290 2291 2292 2293 2294 2295 2296 2297 2298 2299 2300 2301 2302
{
	MemSet(gxact->gid, 0, TMGIDSIZE);
	gxact->gxid = InvalidDistributedTransactionId;
	setGxactState(gxact, DTX_STATE_NONE);

	/*
	 * Memory only fields.
	 */

	gxact->sessionId = gp_session_id;

	gxact->explicitBeginRemembered = false;

2303
	gxact->xminDistributedSnapshot = InvalidDistributedTransactionId;
2304 2305 2306 2307 2308 2309 2310 2311 2312 2313 2314

	gxact->badPrepareGangs = false;

	gxact->directTransaction = false;
	gxact->directTransactionContentId = 0;
}

void
getAllDistributedXactStatus(TMGALLXACTSTATUS **allDistributedXactStatus)
{
	TMGALLXACTSTATUS *all;
A
Ashwin Agrawal 已提交
2315
	int			count;
2316 2317 2318 2319 2320 2321 2322 2323 2324 2325 2326 2327

	all = palloc(sizeof(TMGALLXACTSTATUS));
	all->next = 0;
	all->count = 0;
	all->statusArray = NULL;

	if (shmDtmStarted != NULL && *shmDtmStarted)
	{
		getTmLock();
		count = *shmNumGxacts;
		if (count > 0)
		{
A
Ashwin Agrawal 已提交
2328
			int			i;
2329 2330 2331 2332 2333

			all->statusArray =
				palloc(MAXALIGN(count * sizeof(TMGXACTSTATUS)));
			for (i = 0; i < count; i++)
			{
A
Ashwin Agrawal 已提交
2334
				TMGXACT    *gxact = shmGxactArray[i];
2335 2336 2337 2338

				all->statusArray[i].gxid = gxact->gxid;
				if (strlen(gxact->gid) >= TMGIDSIZE)
					elog(PANIC, "Distribute transaction identifier too long (%d)",
A
Ashwin Agrawal 已提交
2339
						 (int) strlen(gxact->gid));
2340 2341 2342 2343 2344 2345 2346 2347 2348 2349 2350 2351 2352 2353 2354 2355 2356 2357 2358 2359 2360 2361 2362 2363 2364 2365 2366 2367 2368 2369 2370
				memcpy(all->statusArray[i].gid, gxact->gid, TMGIDSIZE);
				all->statusArray[i].state = gxact->state;
				all->statusArray[i].sessionId = gxact->sessionId;
				all->statusArray[i].xminDistributedSnapshot = gxact->xminDistributedSnapshot;
			}

			all->count = count;
		}

		releaseTmLock();

	}

	*allDistributedXactStatus = all;
}

bool
getNextDistributedXactStatus(TMGALLXACTSTATUS *allDistributedXactStatus, TMGXACTSTATUS **distributedXactStatus)
{
	if (allDistributedXactStatus->next >= allDistributedXactStatus->count)
	{
		return false;
	}

	*distributedXactStatus = &allDistributedXactStatus->statusArray[allDistributedXactStatus->next];
	allDistributedXactStatus->next++;

	return true;
}

/*
2371 2372
 * DistributedSnapshotMappedEntry_Compare: A compare function for
 * DistributedTransactionId for use with qsort.
2373 2374 2375 2376
 */
static int
DistributedSnapshotMappedEntry_Compare(const void *p1, const void *p2)
{
2377 2378
	const DistributedTransactionId distribXid1 = *(DistributedTransactionId *) p1;
	const DistributedTransactionId distribXid2 = *(DistributedTransactionId *) p2;
2379

2380
	if (distribXid1 == distribXid2)
2381
		return 0;
2382
	else if (distribXid1 > distribXid2)
2383 2384 2385 2386 2387 2388
		return 1;
	else
		return -1;
}

bool
2389
CreateDistributedSnapshot(DistributedSnapshotWithLocalMapping *distribSnapshotWithLocalMapping)
2390
{
A
Ashwin Agrawal 已提交
2391 2392 2393 2394 2395 2396 2397 2398 2399 2400 2401
	int			globalCount;
	int			i;
	TMGXACT    *gxact_candidate;
	DtxState	state;
	int			count;
	DistributedTransactionId xmin;
	DistributedTransactionId xmax;
	DistributedTransactionId inProgressXid;
	DistributedSnapshotId distribSnapshotId;
	DistributedTransactionId globalXminDistributedSnapshots;
	DistributedSnapshot *ds;
2402 2403 2404

	if (currentGxact == NULL)
	{
2405
		elog(DTM_DEBUG5, "CreateDistributedSnapshot found currentGxact is NULL");
2406 2407 2408 2409
		return false;
	}

	xmin = LastDistributedTransactionId;
A
Ashwin Agrawal 已提交
2410

2411 2412
	/*
	 * This is analogous to the code in GetSnapshotData() (which calls
A
Ashwin Agrawal 已提交
2413 2414
	 * ReadNewTransactionId(), the distributed-xmax of a transaction is the
	 * last distributed-xmax available
2415
	 */
2416
	xmax = getMaxDistributedXid();
A
Ashwin Agrawal 已提交
2417

2418 2419 2420 2421 2422
	/*
	 * initialize for calculation with xmax, the calculation for this is on
	 * same lines as globalxmin for local snapshot.
	 */
	globalXminDistributedSnapshots = xmax;
2423
	count = 0;
2424
	ds = &distribSnapshotWithLocalMapping->ds;
2425 2426 2427 2428

	getTmLock();

	/*
A
Ashwin Agrawal 已提交
2429 2430
	 * Gather up current in-progress global transactions for the distributed
	 * snapshot.
2431 2432 2433 2434 2435
	 */
	globalCount = *shmNumGxacts;

	for (i = 0; i < globalCount; i++)
	{
2436 2437
		DistributedTransactionId dxid;

2438 2439 2440 2441 2442 2443 2444 2445 2446 2447 2448 2449 2450 2451 2452 2453 2454
		gxact_candidate = shmGxactArray[i];

		state = gxact_candidate->state;
		switch (state)
		{
			case DTX_STATE_ACTIVE_NOT_DISTRIBUTED:
			case DTX_STATE_ACTIVE_DISTRIBUTED:
			case DTX_STATE_PREPARING:
			case DTX_STATE_PREPARED:
			case DTX_STATE_INSERTED_COMMITTED:
			case DTX_STATE_FORCED_COMMITTED:
			case DTX_STATE_NOTIFYING_COMMIT_PREPARED:
			case DTX_STATE_NOTIFYING_ABORT_NO_PREPARED:
			case DTX_STATE_NOTIFYING_ABORT_SOME_PREPARED:
			case DTX_STATE_NOTIFYING_ABORT_PREPARED:
			case DTX_STATE_RETRY_COMMIT_PREPARED:
			case DTX_STATE_RETRY_ABORT_PREPARED:
A
Ashwin Agrawal 已提交
2455

2456 2457 2458 2459 2460 2461 2462 2463 2464 2465 2466 2467 2468 2469 2470 2471 2472 2473 2474 2475
				/*
				 * Active or commit/abort not complete.  Keep this transaction
				 * considered for distributed snapshots.
				 */
				break;

			case DTX_STATE_INSERTED_FORGET_COMMITTED:
				elog(FATAL, "Should not see this transitional state with TM lock held -- dtx state \"%s\" not expected here",
					 DtxStateToString(state));
				break;

			case DTX_STATE_INSERTING_COMMITTED:
				elog(FATAL, "Cannot also be inserting COMMITTED into log buffer from another process with TM lock held");
				break;

			case DTX_STATE_INSERTING_FORGET_COMMITTED:
				elog(FATAL, "Cannot also be inserting FORGET COMMITTED into log buffer from another process with TM lock held");
				break;

			case DTX_STATE_CRASH_COMMITTED:
A
Ashwin Agrawal 已提交
2476

2477 2478 2479 2480 2481 2482 2483 2484 2485 2486 2487
				/*
				 * From a previous system incarnation.
				 */
				continue;

			default:
				elog(PANIC, "Unexpected dtm state: %d",
					 (int) state);
				break;
		}

2488 2489 2490 2491 2492 2493 2494 2495
		/* Update globalXminDistributedSnapshots to be the smallest valid dxid */
		dxid = gxact_candidate->xminDistributedSnapshot;
		if ((dxid != InvalidDistributedTransactionId) &&
			dxid < globalXminDistributedSnapshots)
		{
			globalXminDistributedSnapshots = dxid;
		}

2496 2497 2498 2499 2500 2501 2502 2503 2504 2505 2506 2507 2508 2509 2510 2511 2512
		/*
		 * Include the current distributed transaction in the min/max
		 * calculation.
		 */
		inProgressXid = gxact_candidate->gxid;
		if (inProgressXid < xmin)
		{
			xmin = inProgressXid;
		}
		if (inProgressXid > xmax)
		{
			xmax = inProgressXid;
		}

		if (gxact_candidate == currentGxact)
			continue;

2513
		if (count >= ds->maxCount)
2514 2515
			elog(ERROR, "Too many distributed transactions for snapshot");

2516
		ds->inProgressXidArray[count] = inProgressXid;
2517 2518 2519 2520

		count++;

		elog(DTM_DEBUG5,
2521
			 "CreateDistributedSnapshot added inProgressDistributedXid = %u to snapshot",
2522
			 ds->inProgressXidArray[count]);
2523 2524 2525 2526 2527
	}

	distribSnapshotId = (*shmNextSnapshotId)++;
	releaseTmLock();

2528 2529
	/*
	 * Above globalXminDistributedSnapshots was calculated based on lowest
A
Ashwin Agrawal 已提交
2530 2531
	 * dxid in all snapshots but update it to also include actual process
	 * dxids.
2532 2533 2534 2535
	 */
	if (xmin < globalXminDistributedSnapshots)
		globalXminDistributedSnapshots = xmin;

2536
	/*
2537 2538
	 * Sort the entry {distribXid} to support the QEs doing culls on their
	 * DisribToLocalXact sorted lists.
2539 2540
	 */
	qsort(
A
Ashwin Agrawal 已提交
2541 2542 2543 2544
		  ds->inProgressXidArray,
		  count,
		  sizeof(DistributedTransactionId),
		  DistributedSnapshotMappedEntry_Compare);
2545 2546 2547 2548 2549

	/*
	 * Copy the information we just captured under lock and then sorted into
	 * the distributed snapshot.
	 */
2550 2551 2552 2553 2554 2555
	ds->distribTransactionTimeStamp = *shmDistribTimeStamp;
	ds->xminAllDistributedSnapshots = globalXminDistributedSnapshots;
	ds->distribSnapshotId = distribSnapshotId;
	ds->xmin = xmin;
	ds->xmax = xmax;
	ds->count = count;
2556 2557 2558 2559 2560

	if (xmin < currentGxact->xminDistributedSnapshot)
		currentGxact->xminDistributedSnapshot = xmin;

	elog(DTM_DEBUG5,
2561
		 "CreateDistributedSnapshot distributed snapshot has xmin = %u, count = %u, xmax = %u.",
2562 2563 2564 2565 2566 2567 2568
		 xmin, count, xmax);
	elog((Debug_print_snapshot_dtm ? LOG : DEBUG5),
		 "[Distributed Snapshot #%u] *Create* (gxid = %u, '%s')",
		 distribSnapshotId,
		 currentGxact->gxid,
		 DtxContextToString(DistributedTransactionContext));

2569 2570 2571 2572 2573 2574 2575 2576 2577 2578 2579 2580 2581 2582 2583 2584
	/*
	 * At snapshot creation time, local xid cache is empty. Gets populated as
	 * reverse mapping takes place during visibility checks using this
	 * snapshot.
	 */
	distribSnapshotWithLocalMapping->currentLocalXidsCount = 0;
	distribSnapshotWithLocalMapping->minCachedLocalXid = InvalidTransactionId;
	distribSnapshotWithLocalMapping->maxCachedLocalXid = InvalidTransactionId;

	Assert(distribSnapshotWithLocalMapping->maxLocalXidsCount != 0);
	Assert(distribSnapshotWithLocalMapping->inProgressMappedLocalXids != NULL);

	memset(distribSnapshotWithLocalMapping->inProgressMappedLocalXids,
		   InvalidTransactionId,
		   sizeof(TransactionId) * distribSnapshotWithLocalMapping->maxLocalXidsCount);

2585 2586 2587 2588 2589 2590 2591
	return true;
}

/*
 * Create a global transaction context from share memory.
 */
void
A
Ashwin Agrawal 已提交
2592
createDtx(DistributedTransactionId *distribXid)
2593
{
A
Ashwin Agrawal 已提交
2594
	TMGXACT    *gxact;
2595 2596 2597 2598 2599 2600 2601 2602 2603 2604 2605 2606 2607 2608 2609 2610 2611 2612 2613 2614 2615 2616 2617 2618 2619 2620 2621

	MIRRORED_LOCK_DECLARE;

	/*
	 * Global locking order: ProcArrayLock then DTM lock.
	 */
	MIRRORED_LOCK;

	LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);

	getTmLock();

	if (*shmNumGxacts >= max_tm_gxacts)
	{
		dumpAllDtx();
		releaseTmLock();
		LWLockRelease(ProcArrayLock);
		ereport(FATAL,
				(errmsg("the limit of %d distributed transactions has been reached.",
						max_tm_gxacts),
				 errdetail("The global user configuration (GUC) server parameter max_prepared_transactions controls this limit.")));
	}

	gxact = shmGxactArray[(*shmNumGxacts)++];
	initGxact(gxact);
	generateGID(gxact->gid, &gxact->gxid);

2622
	*distribXid = gxact->gxid;
2623 2624

	/*
A
Ashwin Agrawal 已提交
2625 2626
	 * Until we get our first distributed snapshot, we use our distributed
	 * transaction identifier for the minimum.
2627 2628 2629 2630 2631 2632 2633 2634 2635 2636 2637 2638 2639 2640 2641 2642 2643 2644 2645 2646 2647 2648 2649 2650 2651 2652 2653 2654 2655 2656 2657 2658 2659 2660 2661
	 */
	gxact->xminDistributedSnapshot = gxact->gxid;

	setGxactState(gxact, DTX_STATE_ACTIVE_NOT_DISTRIBUTED);

	releaseTmLock();

	LWLockRelease(ProcArrayLock);

	MIRRORED_UNLOCK;

	currentGxact = gxact;

	elog(DTM_DEBUG5,
		 "createDtx created new distributed transaction gid = %s, gxid = %u.",
		 currentGxact->gid, currentGxact->gxid);
}


/*
 * Release global transaction's shared memory.
 * Must already hold ProcArrayLock and the DTM lock.
 */
static void
releaseGxact_UnderLocks(void)
{
	int			i;
	int			curr;

	if (currentGxact == NULL)
	{
		elog(FATAL, "releaseGxact expected currentGxact to not be NULL");
	}

	elog(DTM_DEBUG5,
2662 2663
		 "releaseGxact called for gid = %s (index = %d)",
		 currentGxact->gid, currentGxact->debugIndex);
2664 2665

	/* find slot of current transaction */
A
Ashwin Agrawal 已提交
2666
	curr = *shmNumGxacts;		/* A bad value we can safely test. */
2667 2668 2669 2670 2671
	for (i = 0; i < *shmNumGxacts; i++)
	{
		if (shmGxactArray[i] == currentGxact)
		{
			curr = i;
2672
			break;
2673 2674 2675 2676 2677 2678 2679 2680 2681 2682 2683 2684 2685 2686 2687 2688 2689 2690 2691 2692 2693 2694 2695 2696 2697 2698 2699 2700 2701 2702 2703 2704 2705 2706 2707 2708 2709 2710 2711 2712 2713 2714 2715
		}
	}

	/* move this to the next available slot */
	(*shmNumGxacts)--;
	if (curr != *shmNumGxacts)
	{
		shmGxactArray[curr] = shmGxactArray[*shmNumGxacts];
		shmGxactArray[*shmNumGxacts] = currentGxact;
	}

	currentGxact = NULL;
}

/*
 * Release global transaction's shared memory.
 */
static void
releaseGxact(void)
{
	/*
	 * Global locking order: ProcArrayLock then DTM lock.
	 */
	LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);

	getTmLock();

	releaseGxact_UnderLocks();

	releaseTmLock();

	LWLockRelease(ProcArrayLock);
}

/*
 * Get lock that serializes commits with DTM checkpoint info.
 * Change state to DTX_STATE_INSERTING_COMMITTED.
 */
void
insertingDistributedCommitted(void)
{
	elog(DTM_DEBUG5,
		 "insertingDistributedCommitted entering in state = %s",
A
Ashwin Agrawal 已提交
2716
		 DtxStateToString(currentGxact->state));
2717 2718 2719

	getTmLock();
	Assert(currentGxact->state == DTX_STATE_PREPARED);
A
Ashwin Agrawal 已提交
2720
	setCurrentGxactState(DTX_STATE_INSERTING_COMMITTED);
2721 2722 2723 2724 2725 2726 2727 2728 2729 2730 2731
}

/*
 * Change state to DTX_STATE_INSERTED_COMMITTED.
 * Release lock.
 */
void
insertedDistributedCommitted(void)
{
	elog(DTM_DEBUG5,
		 "insertedDistributedCommitted entering in state = %s for gid = %s",
A
Ashwin Agrawal 已提交
2732
		 DtxStateToString(currentGxact->state), currentGxact->gid);
2733 2734

	Assert(currentGxact->state == DTX_STATE_INSERTING_COMMITTED);
A
Ashwin Agrawal 已提交
2735
	setCurrentGxactState(DTX_STATE_INSERTED_COMMITTED);
2736 2737 2738 2739 2740 2741 2742 2743 2744 2745 2746 2747
	releaseTmLock();
}


/*
 * Change state to DTX_STATE_FORCED_COMMITTED.
 */
void
forcedDistributedCommitted(XLogRecPtr *recptr)
{
	elog(DTM_DEBUG5,
		 "forcedDistributedCommitted entering in state = %s for gid = %s (xlog record %X/%X)",
2748
		 DtxStateToString(currentGxact->state), currentGxact->gid, recptr->xlogid, recptr->xrecoff);
2749 2750 2751

	getTmLock();
	Assert(currentGxact->state == DTX_STATE_INSERTED_COMMITTED);
A
Ashwin Agrawal 已提交
2752
	setCurrentGxactState(DTX_STATE_FORCED_COMMITTED);
2753 2754 2755 2756 2757 2758 2759 2760 2761 2762 2763 2764 2765 2766 2767 2768 2769 2770 2771 2772
	releaseTmLock();
}



/*
 * Get check point information
 *
 * Whether DTM started or not, we must always store DTM information in
 * this checkpoint record.  A possible case to consider is we might have
 * in-progress global transactions in shared memory after postmaster reset,
 * and shutting down without performing DTM recovery.  The subsequent
 * recovery after this shutdown will read this checkpoint, so we would
 * lose the in-progress global transaction information if we didn't write it
 * here.  Note we will certainly read this global transaction information
 * even if this is a clean shutdown (i.e. not performing multi-pass recovery.)
 */
void
getDtxCheckPointInfoAndLock(char **result, int *result_size)
{
A
Ashwin Agrawal 已提交
2773 2774 2775 2776 2777 2778 2779 2780 2781 2782
	TMGXACT    *gxact;
	DtxState	state;
	int			n;
	TMGXACT_CHECKPOINT *gxact_checkpoint;
	TMGXACT_LOG *gxact_log_array;
	int			i;
	int			actual;
	TMGXACT_LOG *gxact_log;

	getTmLock();				/* We will return with lock held below. */
2783 2784 2785 2786 2787 2788 2789 2790 2791 2792 2793 2794 2795 2796 2797
	n = *shmNumGxacts;

	gxact_checkpoint = palloc(TMGXACT_CHECKPOINT_BYTES(n));
	gxact_log_array = &gxact_checkpoint->committedGxactArray[0];

	actual = 0;
	for (i = 0; i < n; i++)
	{
		gxact = shmGxactArray[i];

		gxact_log = &gxact_log_array[actual];

		state = gxact->state;
		switch (state)
		{
A
Ashwin Agrawal 已提交
2798 2799 2800 2801
			case DTX_STATE_ACTIVE_NOT_DISTRIBUTED:
			case DTX_STATE_ACTIVE_DISTRIBUTED:
			case DTX_STATE_PREPARING:
			case DTX_STATE_PREPARED:
2802

A
Ashwin Agrawal 已提交
2803 2804 2805 2806
				/*
				 * Active or attempting to commit -- ignore.
				 */
				continue;
2807

A
Ashwin Agrawal 已提交
2808 2809 2810 2811
			case DTX_STATE_INSERTED_FORGET_COMMITTED:
				elog(FATAL, "Should not see this transitional state with TM lock held -- dtx state \"%s\" not expected here",
					 DtxStateToString(state));
				continue;
2812

A
Ashwin Agrawal 已提交
2813 2814 2815
			case DTX_STATE_INSERTING_COMMITTED:
				elog(FATAL, "Cannot also be buffering COMMITTED from another process with TM lock held");
				continue;
2816

A
Ashwin Agrawal 已提交
2817 2818 2819
			case DTX_STATE_INSERTING_FORGET_COMMITTED:
				elog(FATAL, "Cannot also be buffering FORGET COMMITTED from another process with TM lock held");
				continue;
2820

A
Ashwin Agrawal 已提交
2821 2822 2823 2824 2825 2826
			case DTX_STATE_INSERTED_COMMITTED:
			case DTX_STATE_FORCED_COMMITTED:
			case DTX_STATE_NOTIFYING_COMMIT_PREPARED:
			case DTX_STATE_RETRY_COMMIT_PREPARED:
			case DTX_STATE_CRASH_COMMITTED:
				break;
2827

A
Ashwin Agrawal 已提交
2828 2829 2830 2831 2832 2833 2834 2835 2836 2837 2838 2839 2840 2841
			case DTX_STATE_NOTIFYING_ABORT_NO_PREPARED:
			case DTX_STATE_NOTIFYING_ABORT_SOME_PREPARED:
			case DTX_STATE_NOTIFYING_ABORT_PREPARED:
			case DTX_STATE_RETRY_ABORT_PREPARED:

				/*
				 * We don't checkpoint abort.
				 */
				continue;

			default:
				elog(PANIC, "Unexpected dtm state: %d",
					 (int) state);
				break;
2842 2843 2844 2845
		}

		if (strlen(gxact->gid) >= TMGIDSIZE)
			elog(PANIC, "Distribute transaction identifier too long (%d)",
A
Ashwin Agrawal 已提交
2846
				 (int) strlen(gxact->gid));
2847 2848 2849 2850 2851 2852 2853 2854 2855 2856 2857 2858 2859 2860 2861 2862 2863 2864 2865 2866 2867 2868 2869 2870 2871
		memcpy(gxact_log->gid, gxact->gid, TMGIDSIZE);
		gxact_log->gxid = gxact->gxid;

		elog(DTM_DEBUG5, "Add DTM checkpoint entry gid = %s.",
			 gxact->gid);
		if (Debug_persistent_recovery_print)
		{
			SUPPRESS_ERRCONTEXT_DECLARE;

			SUPPRESS_ERRCONTEXT_PUSH();

			elog(PersistentRecovery_DebugPrintLevel(),
				 "getDtxCheckPointInfoAndLock[%d]: distributed transaction identifier %s (distributed xid %u)",
				 actual,
				 gxact_log->gid,
				 gxact_log->gxid);

			SUPPRESS_ERRCONTEXT_POP();
		}

		actual++;
	}

	gxact_checkpoint->committedCount = actual;

A
Ashwin Agrawal 已提交
2872
	*result = (char *) gxact_checkpoint;
2873 2874 2875 2876 2877
	*result_size = TMGXACT_CHECKPOINT_BYTES(actual);

	/* Return with lock held. */

	elog(DTM_DEBUG5, "Filled in DTM checkpoint information (count = %d).",
A
Ashwin Agrawal 已提交
2878
		 actual);
2879 2880 2881
}

void
A
Ashwin Agrawal 已提交
2882
			freeDtxCheckPointInfoAndUnlock(char *info, int info_size __attribute__((unused)), XLogRecPtr *recptr)
2883 2884 2885 2886 2887 2888 2889 2890 2891 2892 2893 2894 2895 2896 2897 2898 2899 2900
{
	pfree(info);

	releaseTmLock();

	elog(DTM_DEBUG5, "Checkpoint with DTM information written at %X/%X.",
		 recptr->xlogid, recptr->xrecoff);
}

/* generate global transaction id */
static void
generateGID(char *gid, DistributedTransactionId *gxid)
{
	/* tm lock acquired by caller */
	if (*shmGIDSeq >= LastDistributedTransactionId)
	{
		releaseTmLock();
		ereport(FATAL,
A
Ashwin Agrawal 已提交
2901
				(errmsg("reached limit of %u global transactions per start", LastDistributedTransactionId)));
2902
	}
2903
	Assert(*shmDistribTimeStamp != 0);
2904

2905
	*gxid = ++(*shmGIDSeq);
2906 2907 2908
	sprintf(gid, "%u-%.10u", *shmDistribTimeStamp, (*gxid));
	if (strlen(gid) >= TMGIDSIZE)
		elog(PANIC, "Distribute transaction identifier too long (%d)",
A
Ashwin Agrawal 已提交
2909
			 (int) strlen(gid));
2910 2911

	Assert(*gxid != InvalidDistributedTransactionId);
2912 2913
}

2914 2915 2916 2917 2918 2919 2920 2921 2922 2923 2924 2925
/*
 * Return the highest global transaction id that has been generated.
 */
DistributedTransactionId
getMaxDistributedXid(void)
{
	if (!shmGIDSeq)
		return 0;

	return *shmGIDSeq;
}

2926 2927 2928 2929 2930 2931 2932 2933 2934 2935 2936 2937 2938 2939 2940 2941 2942 2943 2944
/*
 * recoverTM:
 * perform TM recovery, this connects to all QE and resolve all in-doubt txn.
 *
 * This gets called when there is not any other DTM activity going on.
 *
 * First, we'll replay the dtm log and get our shmem as up to date as possible
 * in order to help resolve in-doubt transactions.	Then we'll go through and
 * try and resolve in-doubt transactions based on information in the DTM log.
 * The remaining in-doubt transactions that remain (ones the DTM doesn't know
 * about) are all ABORTed.
 *
 * If we're in read-only mode; we need to get started, but we can't run the
 * full recovery. So we go get the highest distributed-xid, but don't run
 * the recovery.
 */
static void
recoverTM(void)
{
2945 2946
	bool		dtmRecoveryDeferred;

2947 2948 2949 2950 2951 2952
	/* intialize fts sync count */
	verifyFtsSyncCount();

	elog(DTM_DEBUG3, "Starting to Recover DTM...");

	/*
A
Ashwin Agrawal 已提交
2953 2954 2955
	 * do not do recovery if read only mode is set. in this case, there may be
	 * in-doubt transaction in down segdb , which will not be resolved at this
	 * time.
2956 2957 2958 2959 2960 2961 2962 2963 2964 2965 2966 2967
	 */
	if (isFtsReadOnlySet())
	{
		elog(DTM_DEBUG3, "FTS is Read Only.  Defer DTM recovery till later.");
		if (currentGxact != NULL)
		{
			elog(DTM_DEBUG5,
				 "recoverTM setting currentGxact to NULL for gid = %s (index = %d)",
				 currentGxact->gid, currentGxact->debugIndex);
		}
		currentGxact = NULL;

2968
		dtmRecoveryDeferred = true;
2969 2970
	}
	else
2971
		dtmRecoveryDeferred = false;
2972 2973 2974 2975 2976 2977 2978

	if (Gp_role == GP_ROLE_UTILITY)
	{
		elog(DTM_DEBUG3, "DB in Utility mode.  Defer DTM recovery till later.");
		return;
	}

2979
	if (!dtmRecoveryDeferred)
2980 2981 2982 2983 2984
	{
		/*
		 * Attempt to recover all in-doubt transactions.
		 *
		 * first resolve all in-doubt transactions from the DTM's perspective
A
Ashwin Agrawal 已提交
2985 2986
		 * and then resolve any remaining in-doubt transactions that the RMs
		 * have.
2987 2988 2989 2990 2991 2992 2993 2994
		 */
		recoverInDoubtTransactions();
	}
	else
		elog(LOG, "DTM starting in readonly-mode: deferring recovery");

	/* finished recovery successfully. */

2995
	*shmGIDSeq = 1;
2996 2997 2998 2999 3000 3001 3002 3003 3004 3005 3006 3007

	*shmDtmStarted = true;
	elog(LOG, "DTM Started");
}

/* recoverInDoubtTransactions:
 * Go through all in-doubt transactions that the DTM knows about and
 * resolve them.
 */
static bool
recoverInDoubtTransactions(void)
{
A
Ashwin Agrawal 已提交
3008 3009 3010
	int			i;
	HTAB	   *htab;
	TMGXACT    *saved_currentGxact;
3011 3012 3013 3014 3015 3016 3017 3018 3019 3020 3021 3022

	elog(DTM_DEBUG3, "recover in-doubt distributed transactions");

	ReplayRedoFromUtilityMode();

	/*
	 * For each committed transaction found in the redo pass that was not
	 * matched by a forget committed record, change its state indicating
	 * committed notification needed.  Attempt a notification.
	 */
	elog(DTM_DEBUG5,
		 "Going to retry commit notification for distributed transactions (count = %d)",
A
Ashwin Agrawal 已提交
3023
		 *shmNumGxacts);
3024 3025 3026 3027
	dumpAllDtx();

	saved_currentGxact = currentGxact;

A
Ashwin Agrawal 已提交
3028
	for (i = 0; i < *shmNumGxacts;)
3029
	{
A
Ashwin Agrawal 已提交
3030
		TMGXACT    *gxact = shmGxactArray[i];
3031 3032

		/*
A
Ashwin Agrawal 已提交
3033 3034
		 * MPP-4867: if we are running deferred, skip any transaction we're
		 * already inside.
3035 3036 3037
		 */
		if (saved_currentGxact != NULL && gxact == saved_currentGxact)
		{
3038
			i++;
3039 3040 3041 3042 3043
			continue;
		}
		else if (gxact->state == DTX_STATE_ACTIVE_NOT_DISTRIBUTED)
		{
			/* should take care of other sessions. */
3044
			i++;
3045 3046 3047 3048 3049 3050 3051 3052 3053 3054 3055 3056 3057
			continue;
		}

		if (gxact->state != DTX_STATE_CRASH_COMMITTED)
		{
			dumpAllDtx();
			elog(PANIC,
				 "recoverInDoubtTransactions found transaction in '%s' state and was expecting it to be in '%s' state",
				 DtxStateToString(gxact->state), DtxStateToString(DTX_STATE_CRASH_COMMITTED));
		}

		elog(DTM_DEBUG5,
			 "Recovering committed distributed transaction gid = %s",
A
Ashwin Agrawal 已提交
3058
			 gxact->gid);
3059 3060 3061 3062 3063 3064 3065

		/*
		 * Can't start system if we cannot deliiver commits.
		 */
		doNotifyCommittedInDoubt(gxact->gid);

		/*
A
Ashwin Agrawal 已提交
3066 3067 3068
		 * This means there was atleast one in-doubt transactions for whom
		 * prepared commit was sent out. Hence, we've a chance to perform
		 * PT-Catalog verification if requested.
3069 3070 3071 3072 3073 3074 3075
		 */
		if (debug_persistent_ptcat_verification)
			Persistent_Set_PostDTMRecv_PTCatVerificationNeeded();

		currentGxact = gxact;

		/*
A
Ashwin Agrawal 已提交
3076 3077
		 * Global locking order: ProcArrayLock then DTM lock since calls
		 * doInsertForgetCommitted calls releaseGxact.
3078 3079 3080 3081 3082
		 */
		LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);

		getTmLock();

A
Ashwin Agrawal 已提交
3083 3084 3085 3086
		/*
		 * This routine would call releaseGxact_UnderLocks, which would
		 * decrease *shmNumGxacts and do a swap, so no need to increase i
		 */
3087 3088 3089 3090 3091 3092 3093 3094 3095 3096 3097 3098
		doInsertForgetCommitted();

		releaseTmLock();

		LWLockRelease(ProcArrayLock);

	}

	currentGxact = saved_currentGxact;
	dumpAllDtx();

	/*
A
Ashwin Agrawal 已提交
3099 3100
	 * UNDONE: Thus, any in-doubt transctions found will be for aborted
	 * transactions. UNDONE: Gather in-boubt transactions and issue aborts.
3101 3102 3103 3104 3105 3106 3107 3108 3109 3110 3111 3112 3113 3114
	 */
	htab = gatherRMInDoubtTransactions();

	/*
	 * go through and resolve any remaining in-doubt transactions that the
	 * RM's have AFTER recoverDTMInDoubtTransactions.  ALL of these in doubt
	 * transactions will be ABORT'd.  The fact that the DTM doesn't know about
	 * them means that something bad happened before everybody voted to
	 * COMMIT.
	 */
	abortRMInDoubtTransactions(htab);

	/*
	 * If there were any In-Doubt transactions collected from the segment
A
Ashwin Agrawal 已提交
3115 3116
	 * instances and if the verification checks are requested then make the
	 * flag green so that the system is informed to run the verification
3117 3118 3119 3120 3121 3122 3123 3124 3125 3126 3127 3128 3129 3130 3131 3132 3133 3134 3135 3136 3137 3138 3139 3140 3141 3142 3143 3144 3145 3146 3147 3148 3149 3150 3151 3152 3153 3154 3155 3156 3157 3158 3159 3160 3161 3162 3163 3164 3165 3166 3167 3168 3169 3170 3171 3172 3173
	 * checks post DTM is recovered
	 */
	if (debug_persistent_ptcat_verification && htab)
		Persistent_Set_PostDTMRecv_PTCatVerificationNeeded();

	/* get rid of the hashtable */
	hash_destroy(htab);

	/* yes... we are paranoid and will double check */
	htab = gatherRMInDoubtTransactions();

	/*
	 * Hmm.  we still have some remaining indoubt transactions.  For now we
	 * dont have an automated way to clean this mess up.  So we'll have to
	 * rely on smart Admins to do the job manually.  We'll error out of here
	 * and try and provide as much info as possible.
	 *
	 * TODO: We really want to be able to say this particular segdb has these
	 * remaining in-doubt transactions.
	 */
	if (htab != NULL)
	{
		StringInfoData indoubtBuff;

		initStringInfo(&indoubtBuff);

		dumpAllDtx();
		dumpRMOnlyDtx(htab, &indoubtBuff);

		ereport(ERROR, (errcode(ERRCODE_INTERNAL_ERROR),
						errmsg("DTM Log recovery failed.  There are still unresolved "
							   "in-doubt transactions on some of the segment databaes "
							   "that were not able to be resolved for an unknown reason. "),
						errdetail("Here is a list of in-doubt transactions in the system: %s",
								  indoubtBuff.data),
						errhint("Try restarting the Greenplum Database array.  If the problem persists "
								" an Administrator will need to resolve these transactions "
								" manually.")));

	}

	RemoveRedoUtilityModeFile();

	return true;
}

/*
 * gatherRMInDoubtTransactions:
 * Builds a hashtable of all of the in-doubt transactions that exist on the
 * segment databases.  The hashtable basically just serves as a single list
 * without duplicates of all the in-doubt transactions.  It does not keep track
 * of which seg db's have which transactions in-doubt.  It currently doesn't
 * need to due to the way we handle this information later.
 */
static HTAB *
gatherRMInDoubtTransactions(void)
{
3174
	CdbPgResults cdb_pgresults = {NULL, 0};
3175
	const char *cmdbuf = "select gid from pg_prepared_xacts";
3176
	PGresult   *rs;
3177 3178 3179 3180 3181

	InDoubtDtx *lastDtx = NULL;

	HASHCTL		hctl;
	HTAB	   *htab = NULL;
3182
	int			i;
3183 3184 3185 3186 3187
	int			j,
				rows;
	bool		found;

	/* call to all QE to get in-doubt transactions */
3188
	CdbDispatchCommand(cmdbuf, DF_NONE, &cdb_pgresults);
3189 3190

	/* If any result set is nonempty, there are in-doubt transactions. */
3191
	for (i = 0; i < cdb_pgresults.numResults; i++)
3192
	{
A
Ashwin Agrawal 已提交
3193
		rs = cdb_pgresults.pg_results[i];
3194
		rows = PQntuples(rs);
3195 3196 3197 3198 3199 3200 3201 3202 3203 3204 3205 3206 3207

		for (j = 0; j < rows; j++)
		{
			char	   *gid;

			/*
			 * we dont setup our hashtable until we know we have at least one
			 * in doubt transaction
			 */
			if (htab == NULL)
			{

				/* setup a hash table */
A
Ashwin Agrawal 已提交
3208
				hctl.keysize = TMGIDSIZE;	/* GID */
3209 3210 3211 3212 3213 3214 3215 3216 3217 3218 3219
				hctl.entrysize = sizeof(InDoubtDtx);

				htab = hash_create("InDoubtDtxHash", 10, &hctl, HASH_ELEM);

				if (htab == NULL)
				{
					ereport(FATAL, (errcode(ERRCODE_OUT_OF_MEMORY),
									errmsg("DTM could not allocate hash table for InDoubtDtxList.")));
				}
			}

3220
			gid = PQgetvalue(rs, j, 0);
3221 3222 3223 3224 3225 3226 3227 3228 3229 3230 3231 3232 3233 3234 3235 3236 3237 3238

			/* Now we can add entry to hash table */
			lastDtx = (InDoubtDtx *) hash_search(htab, gid, HASH_ENTER, &found);

			/*
			 * only need to bother doing work if there isn't already an entry
			 * for our GID
			 */
			if (!found)
			{
				elog(DEBUG3, "Found in-doubt transaction with GID: %s on remote RM", gid);

				strcpy(lastDtx->gid, gid);
			}

		}
	}

3239
	cdbdisp_clearCdbPgResults(&cdb_pgresults);
3240 3241 3242 3243 3244

	return htab;
}

static bool
A
Ashwin Agrawal 已提交
3245
FindRetryNotifyCommitted(char *gid, TMGXACT **gxact)
3246
{
A
Ashwin Agrawal 已提交
3247
	int			i;
3248 3249 3250 3251 3252 3253 3254 3255 3256 3257 3258 3259 3260 3261 3262 3263 3264 3265 3266 3267 3268 3269 3270 3271 3272

	for (i = 0; i < *shmNumGxacts; i++)
	{
		if (strcmp(gid, shmGxactArray[i]->gid) == 0)
		{
			/* found an retry notify committed */
			*gxact = shmGxactArray[i];
			return true;
		}
	}

	return false;
}

/*
 * abortRMInDoubtTransactions:
 * Goes through all the InDoubtDtx's in the provided htab and ABORTs them
 * across all of the QEs by sending a ROLLBACK PREPARED.
 *
 */
static void
abortRMInDoubtTransactions(HTAB *htab)
{
	HASH_SEQ_STATUS status;
	InDoubtDtx *entry = NULL;
A
Ashwin Agrawal 已提交
3273
	TMGXACT    *gxact = NULL;
3274 3275 3276 3277 3278 3279 3280 3281 3282 3283 3284 3285 3286 3287 3288 3289 3290 3291 3292 3293 3294 3295 3296 3297 3298 3299 3300 3301 3302 3303 3304 3305 3306 3307 3308 3309 3310 3311 3312 3313 3314 3315 3316 3317 3318 3319 3320 3321 3322 3323 3324 3325 3326 3327 3328 3329 3330 3331 3332 3333 3334 3335 3336 3337 3338 3339 3340 3341 3342 3343 3344 3345 3346 3347 3348 3349 3350 3351 3352 3353 3354 3355 3356 3357 3358 3359 3360

	if (htab == NULL)
		return;

	/*
	 * now we have a nice hashtable full of in-doubt dtx's that we need to
	 * resolve.  so we'll use a nice big hammer to get this job done.  instead
	 * of keeping track of which QEs have a prepared txn to be aborted and
	 * which ones don't.  we just issue a ROLLBACK to all of them and ignore
	 * any pesky errors.  This is certainly not an elegant solution but is OK
	 * for now.
	 */
	hash_seq_init(&status, htab);


	while ((entry = (InDoubtDtx *) hash_seq_search(&status)) != NULL)
	{
		if (FindRetryNotifyCommitted(entry->gid, &gxact))
		{
			elog(DTM_DEBUG5, "Skipping in-doubt transaction gid = %s since it is \"%s\"",
				 gxact->gid, DtxStateToString(gxact->state));
			continue;
		}

		elog(DTM_DEBUG3, "Aborting in-doubt transaction with gid = %s", entry->gid);

		doAbortInDoubt(entry->gid);

	}
}

/*
 * dumpAllDtx:
 * used to log the current state (according to the DTM) of all the
 * global distributed transactions that are still active (in shmem).
 */
static void
dumpAllDtx(void)
{
	int			i;

	elog(LOG, "dumping all global transactions: ");
	for (i = 0; i < *shmNumGxacts; i++)
	{
		elog(LOG, "%d - GID: %s  STATE: %s",
			 i, shmGxactArray[i]->gid,
			 DtxStateToString(shmGxactArray[i]->state));
	}
}


static void
dumpRMOnlyDtx(HTAB *htab, StringInfoData *buff)
{
	HASH_SEQ_STATUS status;
	InDoubtDtx *entry = NULL;

	if (htab == NULL)
		return;

	hash_seq_init(&status, htab);

	appendStringInfo(buff, "List of In-doubt transactions remaining across the segdbs: (");

	while ((entry = (InDoubtDtx *) hash_seq_search(&status)) != NULL)
	{
		appendStringInfo(buff, "\"%s\" , ", entry->gid);
	}

	appendStringInfo(buff, ")");

}


/*
 * When called, a SET command is dispatched and the writer gang
 * writes the shared snapshot. This function actually does nothing
 * useful besides making sure that a writer gang is alive and has
 * set the shared snapshot so that the readers could access it.
 *
 * At this point this function is added as a helper for cursor
 * query execution since in MPP cursor queries don't use writer
 * gangs. However, it could be used for other purposes as well.
 *
 * See declaration of assign_gp_write_shared_snapshot(...) for more
 * information.
 */
A
Ashwin Agrawal 已提交
3361 3362
void
verify_shared_snapshot_ready(void)
3363 3364 3365
{
	if (Gp_role == GP_ROLE_DISPATCH)
	{
3366
		CdbDispatchCommand("set gp_write_shared_snapshot=true",
A
Ashwin Agrawal 已提交
3367 3368 3369 3370
						   DF_CANCEL_ON_ERROR |
						   DF_WITH_SNAPSHOT |
						   DF_NEED_TWO_PHASE,
						   NULL);
3371 3372 3373 3374 3375 3376 3377 3378 3379 3380 3381 3382 3383 3384 3385 3386 3387 3388 3389

		dumpSharedLocalSnapshot_forCursor();

		/*
		 * To keep our readers in sync (since they will be dispatched
		 * separately) we need to rewind the segmate synchronizer.
		 */
		DtxContextInfo_RewindSegmateSync();
	}
}

/*
 * Force the writer QE to write the shared snapshot. Will get called
 * after a "set gp_write_shared_snapshot=<true/false>" is executed
 * in dispatch mode.
 *
 * See verify_shared_snapshot_ready(...) for additional information.
 */
bool
A
Ashwin Agrawal 已提交
3390
assign_gp_write_shared_snapshot(bool newval, bool doit, GucSource source __attribute__((unused)))
3391 3392 3393 3394 3395 3396
{

#if FALSE
	elog(DEBUG1, "SET gp_write_shared_snapshot: %s, doit=%s",
		 (newval ? "true" : "false"), (doit ? "true" : "false"));
#endif
A
Ashwin Agrawal 已提交
3397

3398
	/*
A
Ashwin Agrawal 已提交
3399 3400
	 * Make sure newval is "true". if it's "false" this could be a part of a
	 * ROLLBACK so we don't want to set the snapshot then.
3401 3402 3403 3404 3405
	 */
	if (doit && newval)
	{
		if (Gp_role == GP_ROLE_EXECUTE)
		{
3406
			PushActiveSnapshot(GetTransactionSnapshot());
3407 3408 3409 3410 3411

			if (Gp_is_writer)
			{
				dumpSharedLocalSnapshot_forCursor();
			}
3412 3413

			PopActiveSnapshot();
3414 3415 3416 3417 3418 3419 3420 3421 3422
		}
	}

	return true;
}

static void
doQEDistributedExplicitBegin(int txnOptions)
{
A
Ashwin Agrawal 已提交
3423 3424
	int			ExplicitIsoLevel;
	bool		ExplicitReadOnly;
3425 3426 3427 3428 3429 3430 3431 3432 3433 3434

	/*
	 * Start a command.
	 */
	StartTransactionCommand();

	/* Here is the explicit BEGIN. */
	BeginTransactionBlock();

	unpackMppTxnOptions(txnOptions,
A
Ashwin Agrawal 已提交
3435
						&ExplicitIsoLevel, &ExplicitReadOnly);
3436 3437 3438 3439 3440 3441 3442 3443 3444 3445 3446 3447 3448 3449 3450 3451 3452 3453

	XactIsoLevel = ExplicitIsoLevel;
	XactReadOnly = ExplicitReadOnly;

	elog(DTM_DEBUG5, "doQEDistributedExplicitBegin setting XactIsoLevel = %s and XactReadOnly = %s",
		 IsoLevelAsUpperString(XactIsoLevel), (XactReadOnly ? "true" : "false"));

	/*
	 * Finish the BEGIN command.  It will leave the explict transaction
	 * in-progress.
	 */
	CommitTransactionCommand();

}

static bool
isDtxQueryDispatcher(void)
{
A
Ashwin Agrawal 已提交
3454 3455
	bool		isDtmStarted;
	bool		isSharedLocalSnapshotSlotPresent;
3456 3457 3458 3459

	isDtmStarted = (shmDtmStarted != NULL && *shmDtmStarted);
	isSharedLocalSnapshotSlotPresent = (SharedLocalSnapshotSlot != NULL);

A
Ashwin Agrawal 已提交
3460 3461 3462
	return (Gp_role == GP_ROLE_DISPATCH &&
			isDtmStarted &&
			isSharedLocalSnapshotSlotPresent);
3463 3464 3465 3466 3467 3468 3469 3470 3471 3472 3473 3474 3475 3476 3477 3478
}

/*
 * Called prior to handling a requested that comes to the QD, or a utility request to a QE.
 *
 * Sets up the distributed transaction context value and does some basic error checking.
 *
 * Essentially:
 *     if the DistributedTransactionContext is already QD_DISTRIBUTED_CAPABLE then leave it
 *     else if the DistributedTransactionContext is already QE_TWO_PHASE_EXPLICIT_WRITER then leave it
 *     else it MUST be a LOCAL_ONLY, and is converted to QD_DISTRIBUTED_CAPABLE if this process is acting
 *          as a QE.
 */
void
setupRegularDtxContext(void)
{
A
Ashwin Agrawal 已提交
3479
	switch (DistributedTransactionContext)
3480 3481 3482 3483 3484 3485 3486 3487 3488 3489 3490 3491
	{
		case DTX_CONTEXT_QD_DISTRIBUTED_CAPABLE:
			/* Continue in this context.  Do not touch QEDtxContextInfo, etc. */
			break;

		case DTX_CONTEXT_QE_TWO_PHASE_EXPLICIT_WRITER:
			/* Allow this for copy...???  Do not touch QEDtxContextInfo, etc. */
			break;

		default:
			if (DistributedTransactionContext != DTX_CONTEXT_LOCAL_ONLY)
			{
A
Ashwin Agrawal 已提交
3492 3493 3494 3495 3496 3497 3498 3499 3500
				/*
				 * we must be one of:
				 *
				 * DTX_CONTEXT_QD_RETRY_PHASE_2,
				 * DTX_CONTEXT_QE_ENTRY_DB_SINGLETON,
				 * DTX_CONTEXT_QE_AUTO_COMMIT_IMPLICIT,
				 * DTX_CONTEXT_QE_TWO_PHASE_IMPLICIT_WRITER,
				 * DTX_CONTEXT_QE_READER, DTX_CONTEXT_QE_PREPARED
				 */
3501 3502 3503 3504 3505 3506 3507 3508 3509 3510 3511 3512 3513 3514 3515 3516 3517 3518 3519 3520 3521 3522 3523 3524 3525 3526 3527 3528 3529 3530 3531

				elog(ERROR, "setupRegularDtxContext finds unexpected DistributedTransactionContext = '%s'",
					 DtxContextToString(DistributedTransactionContext));
			}

			/* DistributedTransactionContext is DTX_CONTEXT_LOCAL_ONLY */

			Assert(QEDtxContextInfo.distributedXid == InvalidDistributedTransactionId);

			/*
			 * Determine if we are strictly local or a distributed capable QD.
			 */
			Assert(DistributedTransactionContext == DTX_CONTEXT_LOCAL_ONLY);

			if (isDtxQueryDispatcher())
			{
				setDistributedTransactionContext(DTX_CONTEXT_QD_DISTRIBUTED_CAPABLE);
			}
			break;
	}

	elog(DTM_DEBUG5, "setupRegularDtxContext leaving with DistributedTransactionContext = '%s'.",
		 DtxContextToString(DistributedTransactionContext));
}

/**
 * Called on the QE when a query to process has been received.
 *
 * This will set up all distributed transaction information and set the state appropriately.
 */
void
A
Ashwin Agrawal 已提交
3532
setupQEDtxContext(DtxContextInfo *dtxContextInfo)
3533 3534
{
	DistributedSnapshot *distributedSnapshot;
A
Ashwin Agrawal 已提交
3535 3536 3537 3538 3539 3540 3541 3542
	int			txnOptions;
	bool		needTwoPhase;
	bool		explicitBegin;
	bool		haveDistributedSnapshot;
	bool		isEntryDbSingleton = false;
	bool		isReaderQE = false;
	bool		isWriterQE = false;
	bool		isSharedLocalSnapshotSlotPresent;
3543

A
Ashwin Agrawal 已提交
3544
	Assert(dtxContextInfo != NULL);
3545 3546 3547 3548 3549 3550 3551 3552 3553 3554 3555

	/*
	 * DTX Context Info (even when empty) only comes in QE requests.
	 */
	distributedSnapshot = &dtxContextInfo->distributedSnapshot;
	txnOptions = dtxContextInfo->distributedTxnOptions;

	needTwoPhase = isMppTxOptions_NeedTwoPhase(txnOptions);
	explicitBegin = isMppTxOptions_ExplicitBegin(txnOptions);

	haveDistributedSnapshot =
A
Ashwin Agrawal 已提交
3556
		(dtxContextInfo->distributedXid != InvalidDistributedTransactionId);
3557 3558 3559 3560 3561 3562 3563 3564 3565 3566 3567
	isSharedLocalSnapshotSlotPresent = (SharedLocalSnapshotSlot != NULL);

	if (DEBUG5 >= log_min_messages || Debug_print_full_dtm)
	{
		elog(DTM_DEBUG5,
			 "setupQEDtxContext inputs (part 1): Gp_role = %s, Gp_is_writer = %s, "
			 "txnOptions = 0x%x, needTwoPhase = %s, explicitBegin = %s, isoLevel = %s, readOnly = %s, haveDistributedSnapshot = %s.",
			 role_to_string(Gp_role), (Gp_is_writer ? "true" : "false"), txnOptions,
			 (needTwoPhase ? "true" : "false"), (explicitBegin ? "true" : "false"),
			 IsoLevelAsUpperString(mppTxOptions_IsoLevel(txnOptions)), (isMppTxOptions_ReadOnly(txnOptions) ? "true" : "false"),
			 (haveDistributedSnapshot ? "true" : "false"));
A
Ashwin Agrawal 已提交
3568
		elog(DTM_DEBUG5,
3569 3570
			 "setupQEDtxContext inputs (part 2): distributedXid = %u, isSharedLocalSnapshotSlotPresent = %s.",
			 dtxContextInfo->distributedXid,
A
Ashwin Agrawal 已提交
3571
			 (isSharedLocalSnapshotSlotPresent ? "true" : "false"));
3572 3573 3574 3575 3576 3577 3578

		if (haveDistributedSnapshot)
		{
			elog(DTM_DEBUG5,
				 "setupQEDtxContext inputs (part 2a): distributedXid = %u, "
				 "distributedSnapshotData (xmin = %u, xmax = %u, xcnt = %u), distributedCommandId = %d",
				 dtxContextInfo->distributedXid,
3579 3580
				 distributedSnapshot->xmin, distributedSnapshot->xmax,
				 distributedSnapshot->count,
3581 3582 3583 3584
				 dtxContextInfo->curcid);
		}
		if (isSharedLocalSnapshotSlotPresent)
		{
3585 3586 3587 3588 3589 3590 3591 3592 3593 3594 3595 3596 3597 3598 3599 3600
			if (DTM_DEBUG5 >= log_min_messages)
			{
				LWLockAcquire(SharedLocalSnapshotSlot->slotLock, LW_SHARED);
				elog(DTM_DEBUG5,
					 "setupQEDtxContext inputs (part 2b):  shared local snapshot xid = %u "
					 "(xmin: %u xmax: %u xcnt: %u) curcid: %d, QDxid = %u/%u, QDcid = %u",
					 SharedLocalSnapshotSlot->xid,
					 SharedLocalSnapshotSlot->snapshot.xmin,
					 SharedLocalSnapshotSlot->snapshot.xmax,
					 SharedLocalSnapshotSlot->snapshot.xcnt,
					 SharedLocalSnapshotSlot->snapshot.curcid,
					 SharedLocalSnapshotSlot->QDxid,
					 SharedLocalSnapshotSlot->segmateSync,
					 SharedLocalSnapshotSlot->QDcid);
				LWLockRelease(SharedLocalSnapshotSlot->slotLock);
			}
3601 3602 3603 3604 3605 3606 3607 3608 3609 3610 3611 3612 3613 3614
		}
	}

	switch (Gp_role)
	{
		case GP_ROLE_EXECUTE:
			if (Gp_segment == -1 && !Gp_is_writer)
			{
				isEntryDbSingleton = true;
			}
			else
			{
				/*
				 * NOTE: this is a bit hackish. It appears as though
A
Ashwin Agrawal 已提交
3615 3616
				 * StartTransaction() gets called during connection setup
				 * before we even have time to setup our shared snapshot slot.
3617 3618 3619 3620 3621 3622 3623 3624 3625 3626 3627 3628 3629 3630 3631 3632 3633 3634 3635 3636 3637 3638 3639 3640 3641
				 */
				if (SharedLocalSnapshotSlot == NULL)
				{
					if (explicitBegin || haveDistributedSnapshot)
					{
						elog(ERROR, "setupQEDtxContext not expecting distributed begin or snapshot when no Snapshot slot exists");
					}
				}
				else
				{
					if (Gp_is_writer)
					{
						isWriterQE = true;
					}
					else
					{
						isReaderQE = true;
					}
				}
			}
			break;

		default:
			Assert(DistributedTransactionContext == DTX_CONTEXT_LOCAL_ONLY);
			elog(DTM_DEBUG5,
A
Ashwin Agrawal 已提交
3642
				 "setupQEDtxContext leaving context = 'Local Only' for Gp_role = %s", role_to_string(Gp_role));
3643 3644 3645 3646 3647 3648 3649 3650 3651 3652 3653 3654 3655 3656 3657 3658 3659 3660 3661 3662 3663 3664 3665
			return;
	}

	elog(DTM_DEBUG5,
		 "setupQEDtxContext intermediate result: isEntryDbSingleton = %s, isWriterQE = %s, isReaderQE = %s.",
		 (isEntryDbSingleton ? "true" : "false"),
		 (isWriterQE ? "true" : "false"), (isReaderQE ? "true" : "false"));

	/*
	 * Copy to our QE global variable.
	 */
	DtxContextInfo_Copy(&QEDtxContextInfo, dtxContextInfo);

	switch (DistributedTransactionContext)
	{
		case DTX_CONTEXT_LOCAL_ONLY:
			if (isEntryDbSingleton && haveDistributedSnapshot)
			{
				/*
				 * Later, in GetSnapshotData, we will adopt the QD's
				 * transaction and snapshot information.
				 */

A
Ashwin Agrawal 已提交
3666
				setDistributedTransactionContext(DTX_CONTEXT_QE_ENTRY_DB_SINGLETON);
3667 3668 3669 3670 3671 3672 3673 3674
			}
			else if (isReaderQE && haveDistributedSnapshot)
			{
				/*
				 * Later, in GetSnapshotData, we will adopt the QE Writer's
				 * transaction and snapshot information.
				 */

A
Ashwin Agrawal 已提交
3675
				setDistributedTransactionContext(DTX_CONTEXT_QE_READER);
3676 3677 3678 3679 3680 3681 3682 3683 3684 3685 3686 3687 3688 3689 3690 3691 3692
			}
			else if (isWriterQE && (explicitBegin || needTwoPhase))
			{
				if (!haveDistributedSnapshot)
				{
					elog(DTM_DEBUG5,
						 "setupQEDtxContext Segment Writer is involved in a distributed transaction without a distributed snapshot...");
				}

				if (IsTransactionOrTransactionBlock())
				{
					elog(ERROR, "Starting an explicit distributed transaction in segment -- cannot already be in a transaction");
				}

				if (explicitBegin)
				{
					/*
A
Ashwin Agrawal 已提交
3693 3694
					 * We set the DistributedTransactionContext BEFORE we
					 * create the transactions to influence the behavior of
3695 3696
					 * StartTransaction.
					 */
A
Ashwin Agrawal 已提交
3697
					setDistributedTransactionContext(DTX_CONTEXT_QE_TWO_PHASE_EXPLICIT_WRITER);
3698 3699 3700 3701 3702 3703

					doQEDistributedExplicitBegin(txnOptions);
				}
				else
				{
					Assert(needTwoPhase);
A
Ashwin Agrawal 已提交
3704
					setDistributedTransactionContext(DTX_CONTEXT_QE_TWO_PHASE_IMPLICIT_WRITER);
3705 3706 3707 3708 3709 3710 3711 3712 3713 3714 3715 3716
				}
			}
			else if (haveDistributedSnapshot)
			{
				if (IsTransactionOrTransactionBlock())
				{
					elog(ERROR,
						 "Going to start a local implicit transaction in segment using a distribute "
						 "snapshot -- cannot already be in a transaction");
				}

				/*
A
Ashwin Agrawal 已提交
3717 3718 3719
				 * Before executing the query, postgres.c make a standard call
				 * to StartTransactionCommand which will begin a local
				 * transaction with StartTransaction.  This is fine.
3720
				 *
A
Ashwin Agrawal 已提交
3721 3722 3723
				 * However, when the snapshot is created later, the state
				 * below will tell GetSnapshotData to make the local snapshot
				 * from the distributed snapshot.
3724
				 */
A
Ashwin Agrawal 已提交
3725
				setDistributedTransactionContext(DTX_CONTEXT_QE_AUTO_COMMIT_IMPLICIT);
3726 3727 3728
			}
			else
			{
A
Ashwin Agrawal 已提交
3729
				Assert(!haveDistributedSnapshot);
3730 3731

				/*
A
Ashwin Agrawal 已提交
3732 3733
				 * A local implicit transaction without reference to a
				 * distributed snapshot.  Stay in NONE state.
3734 3735 3736 3737 3738 3739 3740 3741 3742 3743
				 */
				Assert(DistributedTransactionContext == DTX_CONTEXT_LOCAL_ONLY);
			}
			break;

		case DTX_CONTEXT_QE_TWO_PHASE_IMPLICIT_WRITER:
/*
		elog(NOTICE, "We should have left this transition state '%s' at the end of the previous command...",
			 DtxContextToString(DistributedTransactionContext));
*/
A
Ashwin Agrawal 已提交
3744
			Assert(IsTransactionOrTransactionBlock());
3745 3746 3747 3748 3749 3750 3751 3752 3753 3754 3755 3756 3757

			if (explicitBegin)
			{
				elog(ERROR, "Cannot have an explicit BEGIN statement...");
			}
			break;

		case DTX_CONTEXT_QE_AUTO_COMMIT_IMPLICIT:
			elog(ERROR, "We should have left this transition state '%s' at the end of the previous command",
				 DtxContextToString(DistributedTransactionContext));
			break;

		case DTX_CONTEXT_QE_TWO_PHASE_EXPLICIT_WRITER:
A
Ashwin Agrawal 已提交
3758
			Assert(IsTransactionOrTransactionBlock());
3759 3760 3761 3762
			break;

		case DTX_CONTEXT_QE_ENTRY_DB_SINGLETON:
		case DTX_CONTEXT_QE_READER:
A
Ashwin Agrawal 已提交
3763

3764 3765 3766 3767 3768 3769 3770 3771 3772 3773 3774 3775 3776 3777 3778 3779 3780 3781 3782 3783 3784 3785 3786 3787
			/*
			 * We are playing games with the xact.c code, so we shouldn't test
			 * with the IsTransactionOrTransactionBlock() routine.
			 */
			break;

		case DTX_CONTEXT_QE_PREPARED:
		case DTX_CONTEXT_QE_FINISH_PREPARED:
			elog(ERROR, "We should not be trying to execute a query in state '%s'",
				 DtxContextToString(DistributedTransactionContext));
			break;

		default:
			elog(PANIC, "Unexpected segment distribute transaction context value: %d",
				 (int) DistributedTransactionContext);
			break;
	}

	elog(DTM_DEBUG5, "setupQEDtxContext final result: DistributedTransactionContext = '%s'.",
		 DtxContextToString(DistributedTransactionContext));

	if (haveDistributedSnapshot)
	{
		elog((Debug_print_snapshot_dtm ? LOG : DEBUG5), "[Distributed Snapshot #%u] *Set QE* currcid = %d (gxid = %u, '%s')",
3788
			 dtxContextInfo->distributedSnapshot.distribSnapshotId,
3789 3790 3791 3792 3793 3794 3795 3796
			 dtxContextInfo->curcid,
			 getDistributedTransactionId(),
			 DtxContextToString(DistributedTransactionContext));
	}

}

void
A
Ashwin Agrawal 已提交
3797
finishDistributedTransactionContext(char *debugCaller, bool aborted)
3798 3799 3800 3801
{
	DistributedTransactionId gxid;

	/*
A
Ashwin Agrawal 已提交
3802 3803
	 * We let the 2 retry states go up to PostgresMain.c, otherwise everything
	 * MUST be complete.
3804 3805
	 */
	if (currentGxact != NULL &&
A
Ashwin Agrawal 已提交
3806 3807
		(currentGxact->state != DTX_STATE_RETRY_COMMIT_PREPARED &&
		 currentGxact->state != DTX_STATE_RETRY_ABORT_PREPARED))
3808 3809
	{
		elog(FATAL, "Expected currentGxact to be NULL at this point.  Found gid =%s, gxid = %u (state = %s, caller = %s)",
A
Ashwin Agrawal 已提交
3810
			 currentGxact->gid, currentGxact->gxid, DtxStateToString(currentGxact->state), debugCaller);
3811 3812 3813 3814 3815 3816 3817 3818 3819 3820
	}

	gxid = getDistributedTransactionId();
	elog(DTM_DEBUG5,
		 "finishDistributedTransactionContext called to change DistributedTransactionContext from %s to %s (caller = %s, gxid = %u)",
		 DtxContextToString(DistributedTransactionContext),
		 DtxContextToString(DTX_CONTEXT_LOCAL_ONLY),
		 debugCaller,
		 gxid);

A
Ashwin Agrawal 已提交
3821
	setDistributedTransactionContext(DTX_CONTEXT_LOCAL_ONLY);
3822 3823 3824 3825 3826 3827 3828 3829 3830 3831 3832 3833 3834 3835 3836 3837 3838 3839 3840 3841 3842 3843 3844 3845 3846 3847 3848 3849 3850 3851 3852 3853 3854 3855 3856 3857 3858 3859 3860 3861 3862 3863 3864

	DtxContextInfo_Reset(&QEDtxContextInfo);

}

static void
rememberDtxExplicitBegin(void)
{
	if (currentGxact == NULL)
	{
		return;
	}

	if (!currentGxact->explicitBeginRemembered)
	{
		elog(DTM_DEBUG5, "rememberDtxExplicitBegin explicit BEGIN for gid = %s",
			 currentGxact->gid);
		currentGxact->explicitBeginRemembered = true;
	}
	else
	{
		elog(DTM_DEBUG5, "rememberDtxExplicitBegin already an explicit BEGIN for gid = %s",
			 currentGxact->gid);
	}
}

/*
 * This is mostly here because
 * cdbcopy doesn't use cdbdisp's services.
 */
void
sendDtxExplicitBegin(void)
{
	char		cmdbuf[100];

	if (currentGxact == NULL)
	{
		return;
	}

	rememberDtxExplicitBegin();

	dtmPreCommand("sendDtxExplicitBegin", "(none)", NULL,
A
Ashwin Agrawal 已提交
3865
				   /* is two-phase */ true, /* withSnapshot */ true, /* inCursor */ false);
3866 3867

	/*
A
Ashwin Agrawal 已提交
3868 3869
	 * Be explicit about both the isolation level and the access mode since in
	 * MPP our QEs are in a another process.
3870 3871 3872 3873 3874 3875
	 */
	sprintf(cmdbuf, "BEGIN ISOLATION LEVEL %s, READ %s",
			IsoLevelAsUpperString(XactIsoLevel),
			(XactReadOnly ? "ONLY" : "WRITE"));

	/*
A
Ashwin Agrawal 已提交
3876 3877
	 * dispatch a DTX command, in the event of an error, this call will either
	 * exit via elog()/ereport() or return false
3878
	 */
3879
	if (!dispatchDtxCommand(cmdbuf))
3880 3881 3882 3883 3884 3885 3886 3887 3888 3889 3890 3891 3892 3893 3894
	{
		ereport(ERROR, (errmsg("Global transaction BEGIN failed for gid = \"%s\" due to error",
							   currentGxact->gid)));
	}
}

/**
 * On the QD, run the Prepare operation.
 */
static void
performDtxProtocolPrepare(const char *gid)
{
	StartTransactionCommand();

	elog(DTM_DEBUG5, "performDtxProtocolCommand going to call PrepareTransactionBlock for distributed transaction (id = '%s')", gid);
A
Ashwin Agrawal 已提交
3895
	if (!PrepareTransactionBlock((char *) gid))
3896 3897 3898 3899 3900 3901
	{
		elog(ERROR, "Prepare of distributed transaction %s failed", gid);
		return;
	}

	/*
A
Ashwin Agrawal 已提交
3902 3903
	 * Calling CommitTransactionCommand will cause the actual COMMIT/PREPARE
	 * work to be performed.
3904 3905 3906 3907 3908
	 */
	CommitTransactionCommand();

	elog(DTM_DEBUG5, "Prepare of distributed transaction succeeded (id = '%s')", gid);

A
Ashwin Agrawal 已提交
3909
	setDistributedTransactionContext(DTX_CONTEXT_QE_PREPARED);
3910 3911 3912 3913 3914 3915 3916 3917 3918 3919 3920 3921 3922 3923 3924 3925 3926 3927
}

/**
 * On the QD, run the Commit Prepared operation.
 */
static void
performDtxProtocolCommitPrepared(const char *gid, bool raiseErrorIfNotFound)
{
	elog(DTM_DEBUG5,
		 "performDtxProtocolCommitPrepared going to call FinishPreparedTransaction for distributed transaction %s", gid);

	StartTransactionCommand();

	/*
	 * Since this call may fail, lets setup a handler.
	 */
	PG_TRY();
	{
A
Ashwin Agrawal 已提交
3928
		FinishPreparedTransaction((char *) gid, /* isCommit */ true, raiseErrorIfNotFound);
3929 3930 3931 3932 3933 3934 3935 3936 3937
	}
	PG_CATCH();
	{
		finishDistributedTransactionContext("performDtxProtocolCommitPrepared -- Commit Prepared (error case)", false);
		PG_RE_THROW();
	}
	PG_END_TRY();

	/*
A
Ashwin Agrawal 已提交
3938 3939
	 * Calling CommitTransactionCommand will cause the actual COMMIT/PREPARE
	 * work to be performed.
3940 3941 3942 3943 3944 3945 3946 3947 3948 3949 3950 3951 3952 3953 3954 3955 3956 3957 3958 3959 3960
	 */
	CommitTransactionCommand();

	finishDistributedTransactionContext("performDtxProtocolCommitPrepared -- Commit Prepared", false);
}

/**
 * On the QD, run the Abort Prepared operation.
 */
static void
performDtxProtocolAbortPrepared(const char *gid, bool raiseErrorIfNotFound)
{
	elog(DTM_DEBUG5, "performDtxProtocolAbortPrepared going to call FinishPreparedTransaction for distributed transaction %s", gid);

	StartTransactionCommand();

	/*
	 * Since this call may fail, lets setup a handler.
	 */
	PG_TRY();
	{
A
Ashwin Agrawal 已提交
3961
		FinishPreparedTransaction((char *) gid, /* isCommit */ false, raiseErrorIfNotFound);
3962 3963 3964 3965 3966 3967 3968 3969 3970
	}
	PG_CATCH();
	{
		finishDistributedTransactionContext("performDtxProtocolAbortPrepared -- Commit Prepared (error case)", true);
		PG_RE_THROW();
	}
	PG_END_TRY();

	/*
A
Ashwin Agrawal 已提交
3971 3972
	 * Calling CommitTransactionCommand will cause the actual COMMIT/PREPARE
	 * work to be performed.
3973 3974 3975 3976 3977 3978 3979 3980 3981 3982 3983
	 */
	CommitTransactionCommand();

	finishDistributedTransactionContext("performDtxProtocolAbortPrepared -- Commit Prepared", true);
}

/**
 * On the QE, handle a DtxProtocolCommand
 */
void
performDtxProtocolCommand(DtxProtocolCommand dtxProtocolCommand,
A
Ashwin Agrawal 已提交
3984 3985
						  int flags __attribute__((unused)),
						  const char *loggingStr __attribute__((unused)), const char *gid,
3986 3987 3988 3989 3990 3991 3992 3993 3994 3995
						  DistributedTransactionId gxid __attribute__((unused)),
						  DtxContextInfo *contextInfo)
{
	elog(DTM_DEBUG5,
		 "performDtxProtocolCommand called with DTX protocol = %s, segment distribute transaction context: '%s'",
		 DtxProtocolCommandToString(dtxProtocolCommand), DtxContextToString(DistributedTransactionContext));

	switch (dtxProtocolCommand)
	{
		case DTX_PROTOCOL_COMMAND_STAY_AT_OR_BECOME_IMPLIED_WRITER:
A
Ashwin Agrawal 已提交
3996
			switch (DistributedTransactionContext)
3997
			{
A
Ashwin Agrawal 已提交
3998 3999 4000 4001 4002 4003 4004 4005 4006 4007 4008 4009 4010 4011 4012 4013 4014 4015 4016 4017
				case DTX_CONTEXT_LOCAL_ONLY:
					/** convert to implicit_writer! */
					setupQEDtxContext(contextInfo);
					StartTransactionCommand();
					break;
				case DTX_CONTEXT_QE_TWO_PHASE_IMPLICIT_WRITER:
					/** already the state we like */
					break;
				default:
					if (isQEContext() || isQDContext())
					{
						elog(FATAL, "Unexpected segment distributed transaction context: '%s'",
							 DtxContextToString(DistributedTransactionContext));
					}
					else
					{
						elog(PANIC, "Unexpected segment distributed transaction context value: %d",
							 (int) DistributedTransactionContext);
					}
					break;
4018 4019 4020 4021 4022 4023 4024 4025 4026 4027
			}
			break;

		case DTX_PROTOCOL_COMMAND_ABORT_NO_PREPARED:
			elog(DTM_DEBUG5,
				 "performDtxProtocolCommand going to call AbortOutOfAnyTransaction for distributed transaction %s", gid);
			AbortOutOfAnyTransaction();
			break;

		case DTX_PROTOCOL_COMMAND_PREPARE:
A
Ashwin Agrawal 已提交
4028

4029
			/*
A
Ashwin Agrawal 已提交
4030 4031
			 * The QD has directed us to read-only commit or prepare an
			 * implicit or explicit distributed transaction.
4032 4033 4034 4035
			 */
			switch (DistributedTransactionContext)
			{
				case DTX_CONTEXT_LOCAL_ONLY:
A
Ashwin Agrawal 已提交
4036

4037 4038 4039 4040 4041 4042 4043 4044 4045 4046 4047 4048 4049 4050 4051 4052 4053 4054 4055 4056 4057 4058 4059 4060 4061 4062 4063 4064 4065 4066 4067
					/*
					 * Spontaneously aborted while we were back at the QD?
					 */
					elog(ERROR, "Distributed transaction %s not found", gid);
					break;

				case DTX_CONTEXT_QE_TWO_PHASE_EXPLICIT_WRITER:
				case DTX_CONTEXT_QE_TWO_PHASE_IMPLICIT_WRITER:
					performDtxProtocolPrepare(gid);
					break;

				case DTX_CONTEXT_QD_DISTRIBUTED_CAPABLE:
				case DTX_CONTEXT_QD_RETRY_PHASE_2:
				case DTX_CONTEXT_QE_PREPARED:
				case DTX_CONTEXT_QE_FINISH_PREPARED:
				case DTX_CONTEXT_QE_ENTRY_DB_SINGLETON:
				case DTX_CONTEXT_QE_READER:
					elog(FATAL, "Unexpected segment distribute transaction context: '%s'",
						 DtxContextToString(DistributedTransactionContext));

				default:
					elog(PANIC, "Unexpected segment distribute transaction context value: %d",
						 (int) DistributedTransactionContext);
					break;
			}
			break;

		case DTX_PROTOCOL_COMMAND_ABORT_SOME_PREPARED:
			switch (DistributedTransactionContext)
			{
				case DTX_CONTEXT_LOCAL_ONLY:
A
Ashwin Agrawal 已提交
4068

4069 4070 4071 4072 4073 4074 4075 4076 4077 4078 4079 4080
					/*
					 * Spontaneously aborted while we were back at the QD?
					 */
					elog(ERROR, "Distributed transaction %s not found", gid);
					break;

				case DTX_CONTEXT_QE_TWO_PHASE_EXPLICIT_WRITER:
				case DTX_CONTEXT_QE_TWO_PHASE_IMPLICIT_WRITER:
					AbortOutOfAnyTransaction();
					break;

				case DTX_CONTEXT_QE_PREPARED:
A
Ashwin Agrawal 已提交
4081
					setDistributedTransactionContext(DTX_CONTEXT_QE_FINISH_PREPARED);
4082 4083 4084 4085 4086 4087 4088 4089 4090 4091 4092 4093 4094 4095 4096 4097 4098 4099
					performDtxProtocolAbortPrepared(gid, /* raiseErrorIfNotFound */ true);
					break;

				case DTX_CONTEXT_QD_DISTRIBUTED_CAPABLE:
				case DTX_CONTEXT_QD_RETRY_PHASE_2:
				case DTX_CONTEXT_QE_ENTRY_DB_SINGLETON:
				case DTX_CONTEXT_QE_READER:
					elog(PANIC, "Unexpected segment distribute transaction context: '%s'",
						 DtxContextToString(DistributedTransactionContext));

				default:
					elog(PANIC, "Unexpected segment distribute transaction context value: %d",
						 (int) DistributedTransactionContext);
					break;
			}
			break;

		case DTX_PROTOCOL_COMMAND_COMMIT_PREPARED:
A
Ashwin Agrawal 已提交
4100 4101
			requireDistributedTransactionContext(DTX_CONTEXT_QE_PREPARED);
			setDistributedTransactionContext(DTX_CONTEXT_QE_FINISH_PREPARED);
4102 4103 4104 4105
			performDtxProtocolCommitPrepared(gid, /* raiseErrorIfNotFound */ true);
			break;

		case DTX_PROTOCOL_COMMAND_ABORT_PREPARED:
A
Ashwin Agrawal 已提交
4106 4107
			requireDistributedTransactionContext(DTX_CONTEXT_QE_PREPARED);
			setDistributedTransactionContext(DTX_CONTEXT_QE_FINISH_PREPARED);
4108 4109 4110 4111
			performDtxProtocolAbortPrepared(gid, /* raiseErrorIfNotFound */ true);
			break;

		case DTX_PROTOCOL_COMMAND_RETRY_COMMIT_PREPARED:
A
Ashwin Agrawal 已提交
4112
			requireDistributedTransactionContext(DTX_CONTEXT_LOCAL_ONLY);
4113 4114 4115 4116
			performDtxProtocolCommitPrepared(gid, /* raiseErrorIfNotFound */ false);
			break;

		case DTX_PROTOCOL_COMMAND_RETRY_ABORT_PREPARED:
A
Ashwin Agrawal 已提交
4117
			requireDistributedTransactionContext(DTX_CONTEXT_LOCAL_ONLY);
4118 4119 4120 4121
			performDtxProtocolAbortPrepared(gid, /* raiseErrorIfNotFound */ false);
			break;

		case DTX_PROTOCOL_COMMAND_RECOVERY_COMMIT_PREPARED:
A
Ashwin Agrawal 已提交
4122
			requireDistributedTransactionContext(DTX_CONTEXT_LOCAL_ONLY);
4123 4124 4125 4126
			performDtxProtocolCommitPrepared(gid, /* raiseErrorIfNotFound */ false);
			break;

		case DTX_PROTOCOL_COMMAND_RECOVERY_ABORT_PREPARED:
A
Ashwin Agrawal 已提交
4127
			requireDistributedTransactionContext(DTX_CONTEXT_LOCAL_ONLY);
4128 4129 4130 4131
			performDtxProtocolAbortPrepared(gid, /* raiseErrorIfNotFound */ false);
			break;

		case DTX_PROTOCOL_COMMAND_SUBTRANSACTION_BEGIN_INTERNAL:
A
Ashwin Agrawal 已提交
4132
			switch (DistributedTransactionContext)
4133 4134
			{
				case DTX_CONTEXT_LOCAL_ONLY:
A
Ashwin Agrawal 已提交
4135

4136
					/*
A
Ashwin Agrawal 已提交
4137 4138
					 * QE is not aware of DTX yet. A typical case is SELECT
					 * foo(), where foo() opens internal subtransaction
4139 4140 4141 4142 4143
					 */
					setupQEDtxContext(contextInfo);
					StartTransactionCommand();
					break;
				case DTX_CONTEXT_QE_TWO_PHASE_IMPLICIT_WRITER:
A
Ashwin Agrawal 已提交
4144 4145 4146 4147 4148

					/*
					 * We already marked this QE to be writer, and transaction
					 * is open.
					 */
4149 4150 4151 4152 4153
				case DTX_CONTEXT_QE_TWO_PHASE_EXPLICIT_WRITER:
				case DTX_CONTEXT_QE_READER:
					break;
				default:
					/* Lets flag this situation out, with explicit crash */
A
Ashwin Agrawal 已提交
4154 4155 4156
					Assert(false);
					elog(DTM_DEBUG5,
						 " SUBTRANSACTION_BEGIN_INTERNAL distributed transaction context invalid: %d",
4157 4158 4159 4160 4161
						 (int) DistributedTransactionContext);
					break;
			}

			BeginInternalSubTransaction(NULL);
A
Ashwin Agrawal 已提交
4162
			Assert(contextInfo->nestingLevel + 1 == GetCurrentTransactionNestLevel());
4163 4164 4165 4166 4167 4168
			break;

		case DTX_PROTOCOL_COMMAND_SUBTRANSACTION_RELEASE_INTERNAL:
			Assert(contextInfo->nestingLevel == GetCurrentTransactionNestLevel());
			ReleaseCurrentSubTransaction();
			break;
A
Ashwin Agrawal 已提交
4169

4170
		case DTX_PROTOCOL_COMMAND_SUBTRANSACTION_ROLLBACK_INTERNAL:
A
Ashwin Agrawal 已提交
4171 4172 4173 4174

			/*
			 * Rollback performs work on master and then dispatches, hence has
			 * nestingLevel its expecting post operation
4175 4176 4177
			 */
			if ((contextInfo->nestingLevel + 1) > GetCurrentTransactionNestLevel())
			{
A
Ashwin Agrawal 已提交
4178 4179 4180
				ereport(ERROR,
						(errmsg("transaction %s at level %d already processed (current level %d)",
								gid, contextInfo->nestingLevel, GetCurrentTransactionNestLevel())));
4181
			}
A
Ashwin Agrawal 已提交
4182

4183
			unsigned int i = GetCurrentTransactionNestLevel() - contextInfo->nestingLevel;
A
Ashwin Agrawal 已提交
4184

4185 4186 4187 4188 4189 4190 4191 4192 4193 4194 4195 4196 4197 4198 4199 4200
			while (i > 0)
			{
				RollbackAndReleaseCurrentSubTransaction();
				i--;
			}

			Assert(contextInfo->nestingLevel == GetCurrentTransactionNestLevel());
			break;

		default:
			elog(ERROR, "Unrecognized dtx protocol command: %d",
				 (int) dtxProtocolCommand);
			break;
	}
	elog(DTM_DEBUG5, "performDtxProtocolCommand successful return for distributed transaction %s", gid);
}