You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@kudu.apache.org by "Andrew Wong (Code Review)" <ge...@cloudera.org> on 2018/10/03 20:05:39 UTC

[kudu-CR] KUDU-2463 pt 2: bump MVCC safe time on Raft no-op

Hello Tidy Bot, Mike Percy, Kudu Jenkins, 

I'd like you to reexamine a change. Please visit

    http://gerrit.cloudera.org:8080/11427

to look at the new patch set (#11).

Change subject: KUDU-2463 pt 2: bump MVCC safe time on Raft no-op
......................................................................

KUDU-2463 pt 2: bump MVCC safe time on Raft no-op

Based on the same rationale as Part 1 of this patch series, this patch
updates MVCC's safe and clean time using the no-op timestamp provided by
the leader following a successful Raft election.

There isn't an obvious reference to the tablet (to get to the MVCC
module) in Raft consensus, but there is a ReplicaTransactionFactory,
that the TabletReplica implements. I've extended this to be a more
general ConsensusRoundHandler that can be used to create transactions or
finish transactions as needed.

An invariant we are trying to uphold is that once MVCC's safe time is
adjusted, all further transactions registered with MVCC will have higher
timestamps than the safe time. With this in mind, it is critical that
the adjustment of safe time be serialized with respect to transactions.
This is the case today because safe time is only advanced by writes in
the prepare thread, on which transactions are started. To echo this,
Raft no-ops will also adjust the safe time on the prepare thread.

The following test changes are included:
- to ensure nothing terrible happens when there is a lot of election
  churn (and hence, a lot of new timestamp advancement), I've tweaked
  exactly_once_writes-itest to more explicitly churn elections.
  Previously it attempted this with just a low timeout. I injected some
  latency to make it churn a bit harder and looped the test 1000 times
  in both TSAN and debug mode.
- since MvccManager::StartTransaction() will hit a CHECK failure if it
  starts a transaction at a timestamp that was previously marked safe, I
  added a configurable sleep at the beginning of the function to widen
  the window during which safe time can be advanced, encouraging the
  CHECK failure. I configured this in raft_consensus_election-itest and
  looped it 1000 times in TSAN and debug mode. If no-ops _didn't_ use
  the prepare thread to advance safe time, the added delay would lead to
  CHECK failures.
- added a test that ensures that, on its own, a tablet will bump its
  MVCC timestamps, with just its elections
- tweaked raft_consensus-itest to use more realistic timestamps, now
  that MVCC's clean and safe time gets updated with the leadership no-op

This patch alone doesn't fix KUDU-2463. Rather, a later patch will
prevent scans from occuring if the MVCC safe time hasn't been advanced,
at which point this patch will reduce the window of scan unavailability.

Change-Id: Icbf812e2cbeeee7c322fd980245cfe40c886a15a
---
M src/kudu/consensus/consensus-test-util.h
M src/kudu/consensus/raft_consensus.cc
M src/kudu/consensus/raft_consensus.h
M src/kudu/integration-tests/exactly_once_writes-itest.cc
M src/kudu/integration-tests/raft_consensus-itest.cc
M src/kudu/integration-tests/raft_consensus_election-itest.cc
M src/kudu/integration-tests/timestamp_advancement-itest.cc
M src/kudu/tablet/mvcc.cc
M src/kudu/tablet/mvcc.h
M src/kudu/tablet/tablet_replica.cc
M src/kudu/tablet/tablet_replica.h
11 files changed, 198 insertions(+), 81 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/27/11427/11
-- 
To view, visit http://gerrit.cloudera.org:8080/11427
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Icbf812e2cbeeee7c322fd980245cfe40c886a15a
Gerrit-Change-Number: 11427
Gerrit-PatchSet: 11
Gerrit-Owner: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy <mp...@apache.org>
Gerrit-Reviewer: Tidy Bot