提交 200a083e 编写于 作者: P Paul Guo

Fix a bug that two-phase sub-transaction is considered as one-phase.

QD backend should not forget whether a sub transaction performed writes

QD backend process can avoid two-phase commit overhead if it knows that no QEs
involved in this transaction or any of its sub transactions performed any
writes. Previously, if a sub transaction performed write on one or more QEs, it
was remembered in that sub transaction's global state. However, the sub
transaction state was lost after sub transaction commit. That resulted in QD
not performing two-phase commit at the end of top transaction.

In fact, regardless of the transaction nesting level, we only need to remember
whether a write was performed by a sub transaction. Therefore, use a backend
global variable, instead of current transaction state to record this
information.
Reviewed-by: NGang Xiong <gxiong@pivotal.io>
Reviewed-by: NHao Wu <gfphoenix78@gmail.com>
Reviewed-by: NAsim R P <apraveen@pivotal.io>
上级 934d87c6
......@@ -219,12 +219,13 @@ typedef struct TransactionStateData
bool didLogXid; /* has xid been included in WAL record? */
int parallelModeLevel; /* Enter/ExitParallelMode counter */
bool executorSaysXactDoesWrites; /* GP executor says xact does writes */
bool executorDidWriteXLog; /* QE has wrote xlog */
struct TransactionStateData *parent; /* back link to parent */
struct TransactionStateData *fastLink; /* back link to jump to parent for efficient search */
} TransactionStateData;
static bool TopXactexecutorDidWriteXLog; /* QE has wrote xlog */
typedef TransactionStateData *TransactionState;
#define NUM_NODES_TO_SKIP_FOR_FAST_SEARCH 100
......@@ -258,7 +259,6 @@ static TransactionStateData TopTransactionStateData = {
false, /* didLogXid */
0, /* parallelMode */
false, /* executorSaysXactDoesWrites */
false, /* executorDidWriteXLog */
NULL /* link to parent state block */
};
......@@ -458,10 +458,9 @@ TransactionDidWriteXLog(void)
}
bool
ExecutorDidWriteXLog(void)
TopXactExecutorDidWriteXLog(void)
{
TransactionState s = CurrentTransactionState;
return s->executorDidWriteXLog;
return TopXactexecutorDidWriteXLog;
}
void
......@@ -547,9 +546,9 @@ MarkCurrentTransactionIdLoggedIfAny(void)
}
void
MarkCurrentTransactionWriteXLogOnExecutor(void)
MarkTopTransactionWriteXLogOnExecutor(void)
{
CurrentTransactionState->executorDidWriteXLog = true;
TopXactexecutorDidWriteXLog = true;
}
/*
......@@ -2315,7 +2314,7 @@ StartTransaction(void)
*/
nUnreportedXids = 0;
s->didLogXid = false;
s->executorDidWriteXLog = false;
TopXactexecutorDidWriteXLog = false;
/*
* must initialize resource-management stuff first
......
......@@ -823,11 +823,11 @@ prepareDtxTransaction(void)
/*
* If only one segment was involved in the transaction, and no local XID
* has been assigned on the QD either, we can perform one-phase commit
* on that one segment. Otherwise, broadcast PREPARE TRANSACTION to the
* segments.
* has been assigned on the QD either, or there is no xlog writing related
* to this transaction on all segments, we can perform one-phase commit.
* Otherwise, broadcast PREPARE TRANSACTION to the segments.
*/
if (!ExecutorDidWriteXLog() ||
if (!TopXactExecutorDidWriteXLog() ||
(!markXidCommitted && list_length(MyTmGxactLocal->dtxSegments) < 2))
{
setCurrentDtxState(DTX_STATE_ONE_PHASE_COMMIT);
......
......@@ -907,7 +907,7 @@ processResults(CdbDispatchResult *dispatchResult)
}
if (segdbDesc->conn->wrote_xlog)
MarkCurrentTransactionWriteXLogOnExecutor();
MarkTopTransactionWriteXLogOnExecutor();
/*
* Attach the PGresult object to the CdbDispatchResult object.
......
......@@ -361,7 +361,7 @@ extern bool IsAbortInProgress(void);
extern bool IsTransactionPreparing(void);
extern bool IsAbortedTransactionBlockState(void);
extern bool TransactionDidWriteXLog(void);
extern bool ExecutorDidWriteXLog(void);
extern bool TopXactExecutorDidWriteXLog(void);
extern void GetAllTransactionXids(
DistributedTransactionId *distribXid,
TransactionId *localXid,
......@@ -373,7 +373,7 @@ extern TransactionId GetCurrentTransactionIdIfAny(void);
extern TransactionId GetStableLatestTransactionId(void);
extern SubTransactionId GetCurrentSubTransactionId(void);
extern void MarkCurrentTransactionIdLoggedIfAny(void);
extern void MarkCurrentTransactionWriteXLogOnExecutor(void);
extern void MarkTopTransactionWriteXLogOnExecutor(void);
extern bool SubTransactionIsActive(SubTransactionId subxid);
extern CommandId GetCurrentCommandId(bool used);
extern TimestampTz GetCurrentTransactionStartTimestamp(void);
......
......@@ -863,3 +863,25 @@ select count(1) = 1 from pg_catalog.pg_export_snapshot();
select pg_cancel_backend(pg_backend_pid());
ERROR: canceling statement due to user request
rollback;
-- Test a bug that a two-phase subtransaction is considered as one-phase.
set optimizer = off; -- orca optimizes value scan so the output is different between orca and postgres optimizer.
truncate distxact1_4;
set test_print_direct_dispatch_info = true;
begin;
savepoint sp1;
insert into distxact1_4 values (2),(1);
INFO: (slice 0) Dispatch command to ALL contents: 0 1 2
INFO: (slice 1) Dispatch command to SINGLE content
release sp1;
end;
INFO: Distributed transaction command 'Distributed Prepare' to ALL contents: 0 1 2
INFO: Distributed transaction command 'Distributed Commit Prepared' to ALL contents: 0 1 2
reset test_print_direct_dispatch_info;
reset optimizer;
select count(gp_segment_id) from distxact1_4 group by gp_segment_id; -- sanity check: tuples should be in > 1 segments
count
-------
1
1
(2 rows)
......@@ -595,3 +595,16 @@ begin;
select count(1) = 1 from pg_catalog.pg_export_snapshot();
select pg_cancel_backend(pg_backend_pid());
rollback;
-- Test a bug that a two-phase subtransaction is considered as one-phase.
set optimizer = off; -- orca optimizes value scan so the output is different between orca and postgres optimizer.
truncate distxact1_4;
set test_print_direct_dispatch_info = true;
begin;
savepoint sp1;
insert into distxact1_4 values (2),(1);
release sp1;
end;
reset test_print_direct_dispatch_info;
reset optimizer;
select count(gp_segment_id) from distxact1_4 group by gp_segment_id; -- sanity check: tuples should be in > 1 segments
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册