Skip to content
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

storage: truncate log only between first index and truncate index #28126

Merged
merged 4 commits into from
Aug 19, 2018

Conversation

nvanbenschoten
Copy link
Member

@nvanbenschoten nvanbenschoten commented Aug 1, 2018

Question

Raft log truncations currently perform two steps (there may be
others, but for the sake of this discussion, let's consider only
these two):

  1. above raft, they compute the stats of all raft log entries up
    to the truncation entry.
  2. beneath raft, they use ClearIterRange to clear all raft log
    entries up to the truncation entry.

In both steps, operations are performed on all entries up to the
truncation entry, and in both steps these operations start from
entry 0. A comment added in #16993 gives some idea as to why:

// We start at index zero because it's always possible that a previous
// truncation did not clean up entries made obsolete by the previous
// truncation.

My current understanding is that this case where a Raft log has
been truncated but its entries not cleaned up is only possible if
a node crashes between applyRaftCommand and handleEvalResultRaftMuLocked.
This brings up the question: why don't we truncate raft entries
downstream of raft in applyRaftCommand? That way, the entries
could be deleted atomically with the update to the RaftTruncatedStateKey
and we wouldn't have to worry about them ever diverging or Raft entries
being leaked. That seems like a trivial change, and if that was the
case, would the approach here be safe? I don't see a reason why
not.

Motivation

For motivation on why we should explore this, I've found that when
running sysbench oltp_insert on a fresh cluster without pre-splits to
measure single range write through, raft log truncation accounts for
about 20% of CPU utilization.

truncate

If we switch the ClearIterRange to a ClearRange downstream of raft,
we improve throughput by 13% and reduce the amount of CPU that raft
log truncation uses to about 5%. It's obvious why this speeds up the
actual truncation itself downstream of raft. The reason why it speeds
up the stats computation is less clear, but it may be allowing a RocksDB
iterator to more easily skip over the deleted entry keys.

If we make the change proposed here, we improve throughput by 28% and
reduce the amount of CPU that raft log truncation uses to a negligible
amount (< 1%, hard to tell exactly). The reason this speeds both the
truncation and the stats computation is because it avoids iterating
over RocksDB tombstones for all Raft entries that have ever existed
on the range.

The throughput improvements are of course exaggerated because we are
isolating the throughput of a single range, but they're significant
enough to warrant exploration about whether we can make this approach
work.

Extension

Finally, the outsized impact of this small change naturally justifies
further exploration. If we could make the change here safe (i.e. if we
could depend on replica.FirstIndex() to always be a lower bound on raft
log entry keys), could we make similar changes elsewhere? Are there
other places where we iterate over an entire raft log keyspace and
inadvertently run into all of the deletion tombstones when we could
simply skip to the replica.FirstIndex()? At a minimum, I believe
that clearRangeData fits this description, so there may be room to
speed up snapshots and replica GC.

cc. @tschottdorf @petermattis @benesch

@nvanbenschoten nvanbenschoten requested a review from a team August 1, 2018 05:32
@cockroach-teamcity
Copy link
Member

This change is Reviewable

@nvanbenschoten nvanbenschoten force-pushed the nvanbenschoten/truncIndex branch from 0d684fc to e04f09c Compare August 1, 2018 05:33
@bdarnell
Copy link
Contributor

bdarnell commented Aug 1, 2018

This brings up the question: why don't we truncate raft entries
downstream of raft in applyRaftCommand?

I think it's just the way the code is organized. In the propEvalKV project we moved everything but applying the pre-evaluated batch to replica_proposal.go. This was supposed to be memory-only side effects so we didn't have any facility for hooking a side effect like this into the main batch.

If range tombstones were cheap, we wouldn't have to do anything special here and could just truncate the log in evalTruncateLog itself. But we're trying to minimize the size of the batch that gets passed through raft without creating too many range tombstones, so we (effectively) have invented a new kind of range tombstone that we put in the raft command and apply ourselves.

Making this atomic with the batch commit and using FirstIndex instead of zero sounds like a good idea. I don't think there are any other places where we read the log starting from zero; raft itself is careful to always use FirstIndex.

@benesch
Copy link
Contributor

benesch commented Aug 1, 2018

it may be allowing a RocksDB iterator to more easily skip over the deleted entry keys.

Oops, yes, I forgot Peter landed a change to our RocksDB fork (and maybe upstream too? I forget) that does literally exactly this.

@nvanbenschoten
Copy link
Member Author

On a similar note, applying log truncations atomically with the rest of its raft entry will fix an issue where SysBytes can go negative because the deletion of a log entry will be double counted.

@nvanbenschoten nvanbenschoten force-pushed the nvanbenschoten/truncIndex branch from e04f09c to b60a6ce Compare August 15, 2018 18:52
@nvanbenschoten nvanbenschoten changed the title [DNM] storage: truncate log only between first index and truncate index storage: truncate log only between first index and truncate index Aug 15, 2018
@nvanbenschoten
Copy link
Member Author

I just brought this into a mergeable state. In doing so, I found and fixed a bug where we ignored the last entry in a raft log truncation interval above Raft.

I confirmed that the change still has the same performance impact on sysbench. I also found that this has an almost identical effect on kv0 with no pre-splits. On my laptop it increased throughput by 18%, reduced average operational latency by 16.2%, and reduced 99th-percentile latency by 55%! Even better, it seemed to dramatically reduce the amount of throughput dropoff the workload observes over time. This makes sense because without this change the cost of a raft log truncation would incrementally increase over time as the number of entry tombstones that each log truncation had to skip over continued to grow.

Copy link
Contributor

@bdarnell bdarnell left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained


pkg/storage/batcheval/cmd_truncate_log.go, line 84 at r2 (raw file):

	}

	start := engine.MakeMVCCMetadataKey(keys.RaftLogKey(rangeID, firstIndex))

I think this needs to be conditional on a new cluster version. Otherwise, clusters with non-finalized upgrades could leak log entries (and, surprisingly, this could leave gaps)

Copy link
Member Author

@nvanbenschoten nvanbenschoten left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained


pkg/storage/batcheval/cmd_truncate_log.go, line 84 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

I think this needs to be conditional on a new cluster version. Otherwise, clusters with non-finalized upgrades could leak log entries (and, surprisingly, this could leave gaps)

Yeah sorry, I meant to make it explicit that the migration to this approach introduces a small window of opportunity to leak entries that will never be cleaned up. Specifically, a replica who was killed in between the applyRaftCommand and handleReplicatedEvalResult routines of its last log truncation before upgrading to this binary will temporarily leak those entries (but its stats will still be correct). However, these entries will eventually be cleaned up if the replica is ever rebalanced. You may feel differently, but my thought was that this is such a low-risk problem (because it sounds very rare and because the leaked entries are fairly harmless) that it didn't warrant the extra complication of making each replica guarantee an atomic log truncation starting from 0 before migrating to the new approach of truncating the log starting from lastTruncatedIndex.

Also, the change upstream of Raft is strictly a correctness fix and doesn't need a cluster version. Starting from 0 when computing the stats was double counting the stats delta for entries that we not cleaned up during a previous log truncation.

Copy link
Member Author

@nvanbenschoten nvanbenschoten left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained


pkg/storage/batcheval/cmd_truncate_log.go, line 84 at r2 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Yeah sorry, I meant to make it explicit that the migration to this approach introduces a small window of opportunity to leak entries that will never be cleaned up. Specifically, a replica who was killed in between the applyRaftCommand and handleReplicatedEvalResult routines of its last log truncation before upgrading to this binary will temporarily leak those entries (but its stats will still be correct). However, these entries will eventually be cleaned up if the replica is ever rebalanced. You may feel differently, but my thought was that this is such a low-risk problem (because it sounds very rare and because the leaked entries are fairly harmless) that it didn't warrant the extra complication of making each replica guarantee an atomic log truncation starting from 0 before migrating to the new approach of truncating the log starting from lastTruncatedIndex.

Also, the change upstream of Raft is strictly a correctness fix and doesn't need a cluster version. Starting from 0 when computing the stats was double counting the stats delta for entries that we not cleaned up during a previous log truncation.

s/never be cleaned up/not be cleaned up until a replica is destroyed/

Copy link
Contributor

@bdarnell bdarnell left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained


pkg/storage/batcheval/cmd_truncate_log.go, line 84 at r2 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

s/never be cleaned up/not be cleaned up until a replica is destroyed/

The problem is that in a 3-node cluster, rebalancing is impossible, so they'll stick around forever. And the fact that the log becomes non-contiguous seems like it might be a problem (although we store the truncation index so nothing is ever supposed to look at these lost entries).

But good point that we can't just solve it here, we'd have to do something that would ensure a single run of the truncation from zero (which implies extra state). If it's going to be that difficult to address, I think it's rare enough (and probably-harmless enough) not to bother.

Copy link
Collaborator

@petermattis petermattis left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained


pkg/storage/replica.go, line 5707 at r2 (raw file):

				log.Errorf(ctx, "unable to clear truncated Raft entries for %+v: %s", rResult.State.TruncatedState, err)
			}
			iter.Close()

One oddity about performing the truncation above-Raft is that the Raft log truncation op contains the keys of the Raft log entries being deleted. I suppose this is necessary to ensure that the update of the TruncatedState and the clearing of the keys happens atomically.

An alternative idea which wouldn't involve a new cluster version, would be to leave the truncation below Raft but use an unreplicated local key to store the first entry in the Raft log. This would only need to be read when replica state is loaded, but would be written on every truncation. Might not have the same benefit as the approach here. I haven't thought through the pros and cons of this approach.

Copy link
Member Author

@nvanbenschoten nvanbenschoten left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained


pkg/storage/replica.go, line 5707 at r2 (raw file):

would be to leave the truncation below Raft

Just to be clear, this change isn't moving the log truncation back above Raft, it's just making its application atomic with the rest of the write batch.

I think I'm probably misunderstanding your point. Are you saying that the extra unreplicated state could be used to solve the leaked entries problem Ben and I are discussing below? That's kind of what I had in mind for the migration, if we decided one was necessary.

Copy link
Collaborator

@petermattis petermattis left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained


pkg/storage/replica.go, line 5707 at r2 (raw file):
Ah, I think I mistook applyRaftCommand for processRaftCommand. You're right this is still happening below Raft.

Are you saying that the extra unreplicated state could be used to solve the leaked entries problem Ben and I are discussing below?

Yes.

Copy link
Collaborator

@petermattis petermattis left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained


pkg/storage/batcheval/cmd_truncate_log.go, line 84 at r2 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

The problem is that in a 3-node cluster, rebalancing is impossible, so they'll stick around forever. And the fact that the log becomes non-contiguous seems like it might be a problem (although we store the truncation index so nothing is ever supposed to look at these lost entries).

But good point that we can't just solve it here, we'd have to do something that would ensure a single run of the truncation from zero (which implies extra state). If it's going to be that difficult to address, I think it's rare enough (and probably-harmless enough) not to bother.

Perhaps we truncate the Raft log from 0 to firstIndex when a Replica is loaded or some other infrequent time.

Copy link
Member Author

@nvanbenschoten nvanbenschoten left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained


pkg/storage/batcheval/cmd_truncate_log.go, line 84 at r2 (raw file):

Perhaps we truncate the Raft log from 0 to firstIndex when a Replica is loaded or some other infrequent time.

I'd hate to have to perform writes for every replica when loading them, especially in light of @spencerkimball's work to speed up startup.

We could do this write during consistency checks, although if we did that we'd never have any guarantee that all ranges were touched before upgrading to future versions, so we wouldn't be able to ever remove the code with 100% confidence that it cleared out all leaked entries.

Copy link
Collaborator

@petermattis petermattis left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained


pkg/storage/batcheval/cmd_truncate_log.go, line 84 at r2 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Perhaps we truncate the Raft log from 0 to firstIndex when a Replica is loaded or some other infrequent time.

I'd hate to have to perform writes for every replica when loading them, especially in light of @spencerkimball's work to speed up startup.

We could do this write during consistency checks, although if we did that we'd never have any guarantee that all ranges were touched before upgrading to future versions, so we wouldn't be able to ever remove the code with 100% confidence that it cleared out all leaked entries.

Ack. Don't let me comments hold up this PR. You and Ben have thought about this more deeply than I have. I'm just driving by.

Copy link
Member

@tbg tbg left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Pretty exciting stuff.

Reviewed 3 of 3 files at r2.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained


pkg/storage/batcheval/cmd_truncate_log.go, line 84 at r2 (raw file):

Previously, petermattis (Peter Mattis) wrote…

Ack. Don't let me comments hold up this PR. You and Ben have thought about this more deeply than I have. I'm just driving by.

We could make the writes conditional on there being something to delete, so it would be a single seek (two if you count loading the truncated state, but that needs to happen anyway). Not sure if that would still be too slow (the seek hits a tombstone minefield) to be done, for instance, at server boot time. If it still is, we can write some migration checkpoint key once it's been done once (with the new cluster version active).

I agree that leftover entries hopefully wouldn't ever be a problem, but I don't think we should risk it.

@nvanbenschoten
Copy link
Member Author

I added a third commit that removes leaked Raft entries during the first time that a Store starts with the new binary. It includes a few optimizations to avoid slowing startup down too much. For instance, the one-time migration takes about 550ms for a dataset with 30,000 ranges on my laptop. PTAL.

@nvanbenschoten
Copy link
Member Author

A follow-on thought I'm now having is that we shouldn't even need to use ClearIterRange to clear Raft entries during log truncations. All Raft entries are at discrete keys and we know the start and end bounds ahead of time, so we should be able to lay down a series of blind deletes without the need for iteration at all. We also already computed the stats upstream of Raft so there's really nothing to look at in the truncation interval. I'm not going to make that change here, but I'm curious if anyone sees any holes in that approach.

@nvanbenschoten nvanbenschoten force-pushed the nvanbenschoten/truncIndex branch from baf1ab4 to f5d8cf5 Compare August 17, 2018 22:00
@nvanbenschoten
Copy link
Member Author

I'm not going to make that change here

Never mind, I just did because it was a trivial change that made the code cleaner. Over a series of 4 trials before and after, the approach improved throughput of single-range sysbench oltp_insert by 5.94% (p=0.029).

This change fixes a bug where TruncateLog requests would miss the
last raft entry when computing its impact on the range stats. The
bug was even worse when VersionRaftLogTruncationBelowRaft is not
active. In that case, the last entry in the Raft log truncation
interval was simply not deleted at all.

Release note: None
Raft log truncations currently perform two steps (there may be
others, but for the sake of this discussion, let's consider only
these two):
1. above raft, they compute the stats of all raft log entries up
   to the truncation entry.
2. beneath raft, they use ClearIterRange to clear all raft log
   entries up to the truncation entry.

In both steps, operations are performed on all entries up to the
truncation entry, and in both steps these operations start from
entry 0. A comment added in cockroachdb#16993 gives some idea as to why:

> // We start at index zero because it's always possible that a previous
> // truncation did not clean up entries made obsolete by the previous
> // truncation.

My current understanding is that this case where a Raft log has
been truncated but its entries not cleaned up is only possible if
a node crashes between `applyRaftCommand` and `handleEvalResultRaftMuLocked`.
This brings up the question: why don't we truncate raft entries
downstream of raft in `applyRaftCommand`? That way, the entries
could be deleted atomically with the update to the `RaftTruncatedStateKey`
and we wouldn't have to worry about them ever diverging or Raft entries
being leaked. That seems like a trivial change, and if that was the
case, would the approach here be safe? I don't see a reason why
not.

For motivation on why we should explore this, I've found that when
running `sysbench oltp_insert` on a fresh cluster without pre-splits to
measure single range write through, raft log truncation accounts for
about 20% of CPU utilization.

If we switch the ClearIterRange to a ClearRange downstream of raft,
we improve throughput by 13% and reduce the amount of CPU that raft
log truncation uses to about 5%. It's obvious why this speeds up the
actual truncation itself downstream of raft. The reason why it speeds
up the stats computation is less clear, but it may be allowing a RocksDB
iterator to more easily skip over the deleted entry keys.

If we make the change proposed here, we improve throughput by 28% and
reduce the amount of CPU that raft log truncation uses to a negligible
amount (< 1%, hard to tell exactly). The reason this speeds both the
truncation and the stats computation is because it avoids iterating
over RocksDB tombstones for all Raft entries that have ever existed
on the range.

The throughput improvements are of course exaggerated because we are
isolating the throughput of a single range, but they're significant
enough to warrant exploration about whether we can make this approach
work.

Finally, the outsized impact of this small change naturally justifies
further exploration. If we could make the change here safe (i.e. if we
could depend on replica.FirstIndex() to always be a lower bound on raft
log entry keys), could we make similar changes elsewhere? Are there
other places where we iterate over an entire raft log keyspace and
inadvertently run into all of the deletion tombstones when we could
simply skip to the `replica.FirstIndex()`? At a minimum, I believe
that `clearRangeData` fits this description, so there may be room to
speed up snapshots and replica GC.

Release note (performance improvement): Reduce the cost of Raft log
truncations and increase single-range throughput.
This change prevents any entries that were previously leaked in
non-atomic raft log truncations from living on until a replica
is eventually removed from its store. It does so by adding a new
migration that checks all replicas on a store for evidence of
leaked raft entries and removes them if they exist.

I tested this with a 30,000 range cluster on my laptop that had
been subjected to 20 minutes of `kv0`. The migration took about
550ms. Because of the fast path that quickly determines whether
any entries were leaked, this time will scale linearly with the
number of replicas on a store, but not with the number of Raft
entry tombstones (except on replicas that actually did leak
entries). This one-time cost seems reasonable, and because this
is done on startup, we can remove it in 2.2.

Release note: None
Copy link
Collaborator

@petermattis petermattis left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I didn't review the migration code. Hopefully someone else can scrutinize that.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained


pkg/keys/keys.go, line 349 at r6 (raw file):

// AppendRaftLogIndexToRaftLogPrefix returns a system-local key
// for a Raft log entry using the provided Raft log prefix.
func AppendRaftLogIndexToRaftLogPrefix(prefix roachpb.Key, logIndex uint64) roachpb.Key {

Holy-long-function-name, Batman!


pkg/storage/replica.go, line 5729 at r6 (raw file):

				keyPrefixBuf = make([]byte, len(keyPrefix), len(keyPrefix)+extraCap)
				copy(keyPrefixBuf, keyPrefix)
			}

This seems overly complicated. I'm wondering if you can do something like:

prefixBuf := &r.raftMu.stateLoader.RangeIDPrefixBuf
for idx := oldTruncatedState.Index + 1; idx <= newTruncatedState.Index; idx++ {
  unsafeKey := prefixBuf.RaftLogKey(idx)
  ...
}

Copy link
Member Author

@nvanbenschoten nvanbenschoten left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained


pkg/keys/keys.go, line 349 at r6 (raw file):

Previously, petermattis (Peter Mattis) wrote…

Holy-long-function-name, Batman!

Renamed to AppendRaftLogIndexToPrefix.


pkg/storage/replica.go, line 5707 at r2 (raw file):

Previously, petermattis (Peter Mattis) wrote…

Ah, I think I mistook applyRaftCommand for processRaftCommand. You're right this is still happening below Raft.

Are you saying that the extra unreplicated state could be used to solve the leaked entries problem Ben and I are discussing below?

Yes.

Done.


pkg/storage/replica.go, line 5729 at r6 (raw file):

Previously, petermattis (Peter Mattis) wrote…

This seems overly complicated. I'm wondering if you can do something like:

prefixBuf := &r.raftMu.stateLoader.RangeIDPrefixBuf
for idx := oldTruncatedState.Index + 1; idx <= newTruncatedState.Index; idx++ {
  unsafeKey := prefixBuf.RaftLogKey(idx)
  ...
}

The risk I'm trying to avoid here is RangeIDPrefixBuf not having sufficient capacity and every call to RaftLogKey forcing an allocation. Do you see a cleaner way to accomplish this?


pkg/storage/batcheval/cmd_truncate_log.go, line 84 at r2 (raw file):

Previously, tschottdorf (Tobias Schottdorf) wrote…

We could make the writes conditional on there being something to delete, so it would be a single seek (two if you count loading the truncated state, but that needs to happen anyway). Not sure if that would still be too slow (the seek hits a tombstone minefield) to be done, for instance, at server boot time. If it still is, we can write some migration checkpoint key once it's been done once (with the new cluster version active).

I agree that leftover entries hopefully wouldn't ever be a problem, but I don't think we should risk it.

Done.

@nvanbenschoten nvanbenschoten force-pushed the nvanbenschoten/truncIndex branch from 1d09a32 to b78f32a Compare August 18, 2018 18:48
Copy link
Collaborator

@petermattis petermattis left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained


pkg/keys/keys.go, line 349 at r6 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Renamed to AppendRaftLogIndexToPrefix.

Ack, though I think this can go away with my other comment.


pkg/storage/replica.go, line 5729 at r6 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

The risk I'm trying to avoid here is RangeIDPrefixBuf not having sufficient capacity and every call to RaftLogKey forcing an allocation. Do you see a cleaner way to accomplish this?

Sure. RangeIDPrefixBuf is simply type RangeIDPrefixBuf roachpb.Key. So just make a larger one:

  prefixBuf := make(keys.RangeIDPrefixBuf, 0, 128)
  prefixBuf = append(prefixBuf, r.raftMu.stateLoader.RangeIDPrefixBuf...)  

I thought I had calculated that the default RangeIDPrefixBuf size (32) was sufficient for Raft log entries, though.

This changes Raft log truncation to perform a series of
blind deletes without the need to iterate over the truncation
interval and search for each key to delete. This is possible
because all Raft entries are at discrete keys and we know the
start and end bounds ahead of time.

Release note: None
@nvanbenschoten nvanbenschoten force-pushed the nvanbenschoten/truncIndex branch from b78f32a to 6e4e57f Compare August 18, 2018 20:19
Copy link
Member Author

@nvanbenschoten nvanbenschoten left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained


pkg/keys/keys.go, line 349 at r6 (raw file):

Previously, petermattis (Peter Mattis) wrote…

Ack, though I think this can go away with my other comment.

Done.


pkg/storage/replica.go, line 5729 at r6 (raw file):

Previously, petermattis (Peter Mattis) wrote…

Sure. RangeIDPrefixBuf is simply type RangeIDPrefixBuf roachpb.Key. So just make a larger one:

  prefixBuf := make(keys.RangeIDPrefixBuf, 0, 128)
  prefixBuf = append(prefixBuf, r.raftMu.stateLoader.RangeIDPrefixBuf...)  

I thought I had calculated that the default RangeIDPrefixBuf size (32) was sufficient for Raft log entries, though.

Oh nice, TIL! Raft log keys are only 16 bytes long, so your first suggestion was spot on. Done.

Copy link
Collaborator

@petermattis petermattis left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

:lgtm:

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale)

@nvanbenschoten
Copy link
Member Author

bors r+

craig bot pushed a commit that referenced this pull request Aug 19, 2018
28126: storage: truncate log only between first index and truncate index r=nvanbenschoten a=nvanbenschoten

### Question

Raft log truncations currently perform two steps (there may be
others, but for the sake of this discussion, let's consider only
these two):
1. above raft, they compute the stats of all raft log entries up
   to the truncation entry.
2. beneath raft, they use ClearIterRange to clear all raft log
   entries up to the truncation entry.

In both steps, operations are performed on all entries up to the
truncation entry, and in both steps these operations start from
entry 0. A comment added in #16993 gives some idea as to why:

> // We start at index zero because it's always possible that a previous
> // truncation did not clean up entries made obsolete by the previous
> // truncation.

My current understanding is that this case where a Raft log has
been truncated but its entries not cleaned up is only possible if
a node crashes between `applyRaftCommand` and `handleEvalResultRaftMuLocked`.
This brings up the question: why don't we truncate raft entries
downstream of raft in `applyRaftCommand`? That way, the entries
could be deleted atomically with the update to the `RaftTruncatedStateKey`
and we wouldn't have to worry about them ever diverging or Raft entries
being leaked. That seems like a trivial change, and if that was the
case, would the approach here be safe? I don't see a reason why
not.

### Motivation

For motivation on why we should explore this, I've found that when
running `sysbench oltp_insert` on a fresh cluster without pre-splits to
measure single range write through, raft log truncation accounts for
about **20%** of CPU utilization.

<img width="1272" alt="truncate" src="https://user-images.githubusercontent.com/5438456/43502846-bb7a98d2-952a-11e8-9ba0-0b886d3e3ad9.png">

If we switch the ClearIterRange to a ClearRange downstream of raft,
we improve throughput by **13%** and reduce the amount of CPU that raft
log truncation uses to about **5%**. It's obvious why this speeds up the
actual truncation itself downstream of raft. The reason why it speeds
up the stats computation is less clear, but it may be allowing a RocksDB
iterator to more easily skip over the deleted entry keys.

If we make the change proposed here, we improve throughput by **28%** and
reduce the amount of CPU that raft log truncation uses to a negligible
amount (**< 1%**, hard to tell exactly). The reason this speeds both the
truncation and the stats computation is because it avoids iterating
over RocksDB tombstones for all Raft entries that have ever existed
on the range.

The throughput improvements are of course exaggerated because we are
isolating the throughput of a single range, but they're significant
enough to warrant exploration about whether we can make this approach
work.

### Extension

Finally, the outsized impact of this small change naturally justifies
further exploration. If we could make the change here safe (i.e. if we
could depend on replica.FirstIndex() to always be a lower bound on raft
log entry keys), could we make similar changes elsewhere? Are there
other places where we iterate over an entire raft log keyspace and
inadvertently run into all of the deletion tombstones when we could
simply skip to the `replica.FirstIndex()`? At a minimum, I believe
that `clearRangeData` fits this description, so there may be room to
speed up snapshots and replica GC.

cc. @tschottdorf @petermattis @benesch

Co-authored-by: Nathan VanBenschoten <[email protected]>
@craig
Copy link
Contributor

craig bot commented Aug 19, 2018

Build succeeded

@craig craig bot merged commit 6e4e57f into cockroachdb:master Aug 19, 2018
@nvanbenschoten nvanbenschoten deleted the nvanbenschoten/truncIndex branch August 19, 2018 22:09
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

6 participants