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 2021/11/29 17:55:13 UTC

[kudu-CR] wip [tablet bootstrap] KUDU-38: skip replaying flushed ops

Andrew Wong has uploaded this change for review. ( http://gerrit.cloudera.org:8080/18056


Change subject: wip [tablet_bootstrap] KUDU-38: skip replaying flushed ops
......................................................................

wip [tablet_bootstrap] KUDU-38: skip replaying flushed ops

wip:
- needs a more complete commit message
- not fully functional yet
- I only did some manual testing via dense_node-itest

There are a few means with which we anchor WAL segments:
1 Based on Raft consensus, ensuring that we don't GC any ops that are
  not yet Raft committed, or may be required to catch up other peers
2 Based on the LogAnchorRegistry, ensuring that in-memory state that
  needs to be rebuilt upon startup keep the WALs around until they are
  flushed to disk
3 Based on the OpTracker, ensuring we don't GC any ops that are still
  being replicated

We need to ensure that ops from #1 and #3 don't rely on being
bootstrapped from the WAL, or account for them when persisting the
lowest anchored location.

Change-Id: I94cd2e5a95355b6cdb0e548cd8dcfae26da9e470
---
M src/kudu/consensus/log.cc
M src/kudu/consensus/log.h
M src/kudu/consensus/log_anchor_registry.cc
M src/kudu/consensus/log_anchor_registry.h
M src/kudu/consensus/log_index.h
M src/kudu/consensus/log_reader.cc
M src/kudu/consensus/log_reader.h
M src/kudu/consensus/log_util.cc
M src/kudu/consensus/log_util.h
M src/kudu/consensus/opid.proto
M src/kudu/integration-tests/dense_node-itest.cc
M src/kudu/tablet/delta_tracker.cc
M src/kudu/tablet/diskrowset.cc
M src/kudu/tablet/metadata-test.cc
M src/kudu/tablet/metadata.proto
M src/kudu/tablet/rowset_metadata.cc
M src/kudu/tablet/rowset_metadata.h
M src/kudu/tablet/tablet.cc
M src/kudu/tablet/tablet_bootstrap.cc
M src/kudu/tablet/tablet_metadata.cc
M src/kudu/tablet/tablet_metadata.h
M src/kudu/tablet/tablet_replica.cc
M src/kudu/tablet/tablet_replica.h
M src/kudu/tserver/ts_tablet_manager.cc
24 files changed, 315 insertions(+), 35 deletions(-)



  git pull ssh://gerrit.cloudera.org:29418/kudu refs/changes/56/18056/1
-- 
To view, visit http://gerrit.cloudera.org:8080/18056
To unsubscribe, visit http://gerrit.cloudera.org:8080/settings

Gerrit-Project: kudu
Gerrit-Branch: master
Gerrit-MessageType: newchange
Gerrit-Change-Id: I94cd2e5a95355b6cdb0e548cd8dcfae26da9e470
Gerrit-Change-Number: 18056
Gerrit-PatchSet: 1
Gerrit-Owner: Andrew Wong <aw...@cloudera.com>