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/09/18 07:29:15 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 (#3).

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.

One thing to note is that in some cases (e.g. brand new tablets), the
first election would replicate the no-op with a timestamp of 1 (the
timestamp we're trying to avoid). I tracked the cause of this to be the
fact that sometimes the hybrid clock doesn't get updated before sending
out the first no-op, and this will result in the first assigned
timestamp being 1. To work around this, I updated the clock to the
initial clean time, which seems in line with other updates to the hybrid
clock in the time manager.

I tested this in the following ways:
- to ensure nothing terrible happens when there is a lot of election
  churn (and hence, a lot of timestamp advancement), I've tweaked
  exactly_once_writes-itest to actually churn elections. Previously it
  attempted this with just a low timeout; I injected some latency to
  make it churn a bit harder.
- I added a test that ensured that, on its own, a tablet would bump its
  MVCC timestamps, by virtue of its elections
- I tested the above with single-replica tablets as well
- a few other tests needed to be tweaked given the extra bump to the
  hybrid clock

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/consensus/time_manager.cc
M src/kudu/integration-tests/alter_table-test.cc
M src/kudu/integration-tests/exactly_once_writes-itest.cc
M src/kudu/integration-tests/raft_consensus-itest.cc
M src/kudu/integration-tests/timestamp_advancement-itest.cc
M src/kudu/tablet/tablet_replica.cc
M src/kudu/tablet/tablet_replica.h
10 files changed, 180 insertions(+), 78 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/27/11427/3
-- 
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: 3
Gerrit-Owner: Andrew Wong <aw...@cloudera.com>
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy <mp...@apache.org>
Gerrit-Reviewer: Tidy Bot