New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
txn: support 2pc async commit protocol #18622
Conversation
Binlog is not compatible with async commit. Should we ignore the binlog part when async commit is used? |
I think so, seems no perfect solution by now. And it did reduce the usability a lot. |
store/tikv/2pc.go
Outdated
// checkAsyncCommit checks if async commit protocol is available for current transaction commit, true is returned if possible. | ||
func (c *twoPhaseCommitter) checkAsyncCommit(ctx context.Context) bool { | ||
const asyncCommitKeysLimit = 256 | ||
if c.connID > 0 && config.GetGlobalConfig().TiKVClient.EnableAsyncCommit && len(c.mutations.keys) <= asyncCommitKeysLimit { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
when will c.ConnID
will be 0~?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The system session running background works.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why do we need to bypass the internal SQLs?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
To make the tests easier by now, maybe we could remove this constraint if it's stable.
@@ -666,44 +669,60 @@ func sendTxnHeartBeat(bo *Backoffer, store *tikvStore, primary []byte, startTS, | |||
} | |||
} | |||
|
|||
// checkAsyncCommit checks if async commit protocol is available for current transaction commit, true is returned if possible. | |||
func (c *twoPhaseCommitter) checkAsyncCommit(ctx context.Context) bool { | |||
const asyncCommitKeysLimit = 256 |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
do we need to check the total size for "key size is large but key count is less" situation?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think it should be considered. But I haven't think carefully about how to set this limit, maybe some tests are needed.
failpoint.Inject("asyncCommitDoNothing", func() { | ||
failpoint.Return() | ||
}) | ||
commitBo := NewBackofferWithVars(ctx, CommitMaxBackoff, c.txn.vars) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
maybe there is a tiny optimization we should take care at here.
it'd better transfer ttlManager's ownership to this forked goroutine and keep the heartbeat even if commit result has be given to end-user(close ttlManager after this goroutine done?).
other txn should wait origin txn commit's fork goroutine if it's alive, this seems make the origin txn has more opportunity to commit success and reduce duplicate resolve when txn size is larger
store/tikv/prewrite.go
Outdated
@@ -128,6 +128,11 @@ func (action actionPrewrite) handleSingleBatch(c *twoPhaseCommitter, bo *Backoff | |||
c.run(c, nil) | |||
} | |||
} | |||
c.mu.Lock() | |||
if prewriteResp.MinCommitTs > c.minCommitTS { | |||
c.minCommitTS = prewriteResp.MinCommitTs |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
And if MinCommitTS
is 0, it means async commit cannot proceed due to some reason. We should also set useAysncCommit
to false.
Codecov Report
@@ Coverage Diff @@
## master #18622 +/- ##
===========================================
Coverage 79.5004% 79.5004%
===========================================
Files 542 542
Lines 147618 147618
===========================================
Hits 117357 117357
Misses 20939 20939
Partials 9322 9322 |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The rest LGTM.
store/tikv/2pc.go
Outdated
if binlogSkipped { | ||
binloginfo.RemoveOneSkippedCommitter() | ||
} else { | ||
c.writeFinishBinlog(ctx, binlog.BinlogType_Commit, int64(c.commitTS)) | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Are these lines still necessary?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Removed.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM
LGTM |
LGTM |
/merge |
/run-all-tests |
@cfzjywxk merge failed. |
/merge |
/run-all-tests |
@cfzjywxk merge failed. |
/merge |
/run-all-tests |
@cfzjywxk merge failed. |
What problem does this PR solve?
Issue Number: async commit project
Problem Summary:
Change the 2pc commit logic to support async commit protocol.
What is changed and how it works?
What's Changed:
Return success to the client if all prewrites are successful, and get the commit work done asynchronously.
How it Works:
Related changes
Check List
Tests
Side effects
Release note