You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@kudu.apache.org by "Todd Lipcon (Code Review)" <ge...@cloudera.org> on 2016/09/07 23:28:50 UTC

[kudu-CR] Tie log retention to consensus watermarks

Hello Kudu Jenkins,

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

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

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

Change subject: Tie log retention to consensus watermarks
......................................................................

Tie log retention to consensus watermarks

This changes the calculation of log retention to consult consensus.
Consensus returns a struct which indicates the watermark necessary for
durability (the committed index) as well as the watermark necessary to
catch up other peers. This replaces the old single "must be retained"
watermark that the log GC code used before.

The new struct is passed down into the log, and we use the following
policy:

- we always maintain any logs necessary for durability
- beyond that, we try to retain logs to catch up lagging peers, however
  we never maintain more than --log_max_segments_to_retain (a new
  configuration)

I removed the old flag --log_min_seconds_to_retain, since its main
purpose was for dealing with lagging peers, and that is now handled by
directly consulting consensus.

The one tricky bit of the policy is that, even though the peer catch-up
figures into log retention, we do _not_ want it to impact the
calculation of flush priority. In other words, even if the user is OK
retaining 10GB of logs to catch up trailing peers, they probably still
want to flush more aggressively than that so they can avoid very long
startup times. So, the peer-based watermark is not used during the
mapping of log anchors to retention amounts.

Note that the above is only relevant once we have implemented KUDU-38:
we currently will replay all of the retained logs even though we are
aggressively flushing to keep the durability-related retention bounded.

In practice, even without KUDU-38, this patch shouldn't have a large
negative effect on restart times. In fact, in many cases it can
_improve_ startup times, because in most steady workloads we don't have
peers that are extremely far behind. Our log retention only increases in
those cases, and only on those tablets which have a lagging follower.
For other tablets, the new retention policies actually serve to reduce
the number of retained segments, so if there are no laggy peers, we'll
start up faster.

Manually tested for now as follows:
- started a three-node cluster (locally), set to roll logs at 1MB
  segments, but otherwise default
- started an insert workload against a single-tablet table
- I could see that the three servers were maintaining 2 WAL segments in
  their WAL directory.
- I kill -STOPped a random server while continuing to insert. I saw that
  the WALs in this tablet server's directory froze as is (obviously),
  and the other two kept rolling. However, because of this change, the
  other servers started retaining wals starting from the point where I
  had stopped the follower.
- If I let the insert workload continue, the live servers kept rolling
  up until they had 10 segments (default --log_max_segments_to_retain)
  at which point they dropped the oldest log.
- I verified that, during this period while the extra segments were
  retained, the servers continued to flush frequently so that their
  recovery time would be bounded.
- I also verified that, if I un-paused the follower before the others
  had evicted it, it was able to catch up, at which point the other
  servers GCed those extra logs they had been retaining.

The above scenario is also tested through modifications to
RaftConsensusITest.TestCatchupAfterOpsEvicted and various log-test test
cases.

Change-Id: Icfc071270510f3dc3c65f88d615e93c6ffb26b12
---
M src/kudu/consensus/consensus.h
M src/kudu/consensus/consensus.proto
M src/kudu/consensus/consensus_peers-test.cc
M src/kudu/consensus/consensus_queue-test.cc
M src/kudu/consensus/consensus_queue.cc
M src/kudu/consensus/consensus_queue.h
M src/kudu/consensus/log-test-base.h
M src/kudu/consensus/log-test.cc
M src/kudu/consensus/log.cc
M src/kudu/consensus/log.h
M src/kudu/consensus/log_reader.cc
M src/kudu/consensus/log_reader.h
M src/kudu/consensus/raft_consensus-test.cc
M src/kudu/consensus/raft_consensus.cc
M src/kudu/consensus/raft_consensus.h
M src/kudu/consensus/raft_consensus_quorum-test.cc
M src/kudu/integration-tests/delete_table-test.cc
M src/kudu/integration-tests/external_mini_cluster_fs_inspector.cc
M src/kudu/integration-tests/external_mini_cluster_fs_inspector.h
M src/kudu/integration-tests/raft_consensus-itest.cc
M src/kudu/tablet/tablet_peer-test.cc
M src/kudu/tablet/tablet_peer.cc
M src/kudu/tablet/tablet_peer.h
23 files changed, 327 insertions(+), 336 deletions(-)


  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/77/4177/5
-- 
To view, visit http://gerrit.cloudera.org:8080/4177
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Icfc071270510f3dc3c65f88d615e93c6ffb26b12
Gerrit-PatchSet: 5
Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-Owner: Todd Lipcon <to...@apache.org>
Gerrit-Reviewer: David Ribeiro Alves <dr...@apache.org>
Gerrit-Reviewer: Kudu Jenkins
Gerrit-Reviewer: Mike Percy <mp...@apache.org>
Gerrit-Reviewer: Todd Lipcon <to...@apache.org>