cockroachdb / cockroach

CockroachDB — the cloud native, distributed SQL database designed for high availability, effortless scale, and control over data placement.
https://www.cockroachlabs.com
Other
30.11k stars 3.81k forks source link

storage: MVCC stats after initial splits have negative values #7290

Closed tbg closed 8 years ago

tbg commented 8 years ago
{RaftAppliedIndex:42 LeaseAppliedIndex:25 Desc:range_id:1 start_key:"" end_key:"\223" replicas:<node_id:1 store_id:1 replica_id:1 > next_replica_id:2  Lease:replica {1 1 1} 1970-01-01 00:00:00 +0000 UTC 407251h36m37.60286253s TruncatedState:index:0 term:0  GCThreshold:0.000000000,0 Stats:{LastUpdateNanos:1466105792859562525 IntentAge:0 GCBytesAge:2276 LiveBytes:36283 LiveCount:204 KeyBytes:7697 KeyCount:204 ValBytes:28829 ValCount:212 IntentBytes:0 IntentCount:0 SysBytes:455 SysCount:6} Frozen:false}

{RaftAppliedIndex:22 LeaseAppliedIndex:9 Desc:range_id:2 start_key:"\223" end_key:"\224" replicas:<node_id:1 store_id:1 replica_id:1 > next_replica_id:2  Lease:replica {1 1 1} 1970-01-01 00:00:00 +0000 UTC 407251h36m42.592591058s TruncatedState:index:0 term:0  GCThreshold:0.000000000,0 Stats:{LastUpdateNanos:1466105783090649289 IntentAge:0 GCBytesAge:0 LiveBytes:0 LiveCount:0 KeyBytes:0 KeyCount:0 ValBytes:0 ValCount:0 IntentBytes:0 IntentCount:0 SysBytes:503 SysCount:7} Frozen:false}

{RaftAppliedIndex:20 LeaseAppliedIndex:8 Desc:range_id:3 start_key:"\224" end_key:"\225" replicas:<node_id:1 store_id:1 replica_id:1 > next_replica_id:2  Lease:replica {1 1 1} 1970-01-01 00:00:00 +0000 UTC 407251h36m27.840649289s TruncatedState:index:0 term:0  GCThreshold:0.000000000,0 Stats:{LastUpdateNanos:1466105783090649289 IntentAge:0 GCBytesAge:0 LiveBytes:430 LiveCount:5 KeyBytes:204 KeyCount:5 ValBytes:226 ValCount:5 IntentBytes:0 IntentCount:0 SysBytes:742 SysCount:8} Frozen:false}

{RaftAppliedIndex:18 LeaseAppliedIndex:6 Desc:range_id:4 start_key:"\225" end_key:"\226" replicas:<node_id:1 store_id:1 replica_id:1 > next_replica_id:2  Lease:replica {1 1 1} 1970-01-01 00:00:00 +0000 UTC 407251h36m27.840649289s TruncatedState:index:0 term:0  GCThreshold:0.000000000,0 Stats:{LastUpdateNanos:1466105783090649289 IntentAge:0 GCBytesAge:0 LiveBytes:2125 LiveCount:24 KeyBytes:884 KeyCount:24 ValBytes:1241 ValCount:24 IntentBytes:0 IntentCount:0 SysBytes:714 SysCount:8} Frozen:false}

{RaftAppliedIndex:10 LeaseAppliedIndex:0 Desc:range_id:5 start_key:"\226" end_key:"\377\377" replicas:<node_id:1 store_id:1 replica_id:1 > next_replica_id:2  Lease:replica {0 0 0} 1970-01-01 00:00:00 +0000 UTC 0 TruncatedState:index:0 term:0  GCThreshold:0.000000000,0 Stats:{LastUpdateNanos:1466105783090649289 IntentAge:0 GCBytesAge:0 LiveBytes:-7600 LiveCount:-172 KeyBytes:-6912 KeyCount:-172 ValBytes:-688 ValCount:-172 IntentBytes:0 IntentCount:0 SysBytes:487 SysCount:4} Frozen:false}

TL;DR: last two lines have

LiveBytes:-7600 LiveCount:-172 KeyBytes:-6912 KeyCount:-172 ValBytes:-688 ValCount:-172

The in-memory and on-disk stats are identical. We must be messing up the computations, but at least we do so in a consistent way.

Pushed a repro to tschottdorf/repro-7290. Check it out, run

rm -rf cockroach-data; make build && ./cockroach start --alsologtostderr=INFO

and hit http://localhost:8080/_status/ranges/local to see the output above in the logs.

petermattis commented 8 years ago

@nvanbenschoten You touched the MVCC stats code a little while back. Can you take a look at this? The easy reproduction makes me suspect we're doing something stupid.

nvanbenschoten commented 8 years ago

I'm still investigating, but it looks like the negative values are a result of live bytes not being accounted for in the stats somewhere. When a split happens and MVCCStats for the left half of a range are recomputed, this recomputation then accounts for the new bytes, which means that when this left stats object is subtracted from the total stats (+ current delta) to get the right stats, we go negative. I'm seeing this issue materialize about 50% of the time, and it happens even on the first split of a newly bootstrapped cluster.

nvanbenschoten commented 8 years ago

It looks like the issue has something to do with MergeRequests being sent from ts.DB.StoreData. This races with initial splits and would explain why the issue seems to be nondeterministic. If I disable the time series poller, the negative stats issue seems to go away. I'm wondering if we correctly handle stats computations for all MergeRequests.

Looping @mrtracy in as well. You touched the stats code in relation to time series data a little while ago, correct? Do you happen to have any insight into this?

nvanbenschoten commented 8 years ago

It looks pretty promising that https://github.com/cockroachdb/cockroach/blob/master/storage/engine/mvcc.go#L199 is causing this issue. Specifically

we're unable to keep accurate stats on merge as the actual details of the merge play out asynchronously during compaction ... These errors are corrected during splits and merges

nvanbenschoten commented 8 years ago

MVCCMerge seems to be somewhat incompatible with our current handling of MVCCStats. If we're unable to keep accurate stats during MVCCMerges because the details play out asynchronously during RocksDB compaction, then a lot of the assumptions we make about accurate stats break down. Specifically, the optimization made in https://github.com/cockroachdb/cockroach/pull/4697 no longer works in all cases, as it conflicts with the assumption that merge "errors are corrected during splits".

I'm curious if anyone more familiar with the semantics of the Merge operator has ideas about ways that we can get more insight into the impact of the operation on stats (ideally synchronously). Alternative solutions might be to actively correct errors in stats due to the operator on compactions, but this presents a whole slew of other challenges.

tbg commented 8 years ago

Yuck. I wish we could simply get rid of merges.

petermattis commented 8 years ago

MVCCComputeStats accurately tracks stats for merges while updateStatsOnMerge only performs an estimate. In order to keep the optimization in #4697 valid we have to make these routines compute the same thing. We can't make updateStatsOnMerge accurate without making it slower, but is there any difficulty in making MVCCComputeStats compute the same stats for merges as updateStatsOnMerge? The end result is that we will have mildly inaccurate stats for ranges containing merges, but merges are only present in system ranges so that doesn't seem too bad.

Getting rid of merges seems like a whole additional headache I'd prefer not to explore.

nvanbenschoten commented 8 years ago

I'm starting to question if there is any way that we can estimate MVCCStats during the initialization of a Merge (in updateStatsOnMerge) in such a way that we can adjust MVCCComputeStats to produce identical results. The issue I'm seeing is that a given Merge for Timestamp values, which is currently the only situation where we use it, may not add any new information to an existing kv pair. For instance, if a new InternalTimeSeriesData contains only samples with already seen offsets (perhaps from a replayed raft command), the current value will not be changed. This contradicts the current comment for updateStatsOnMerge, which claims to "undercount", and I think makes it impossible to adjust MVCCComputeStats to agree with updateStatsOnMerge in all cases (perhaps unless we keep a "merge count" on the data and do some fancy calculations).

If we're willing to compromise on accuracy for ranges containing merges, maybe the easiest thing to do is to ignore merge value's effects on the stats completely. To do this, we could just remove updateStatsOnMerge, and ignore merge values (determined using IsTimeSeriesData) in MVCCComputeStats.

tbg commented 8 years ago

That would be a bit of a bummer since we don't actually GC timeseries data and so its size is pretty relevant. Before we do that, could we at least count timeseries kv pairs? After all, a merge never deletes one but will always create one if there isn't one, and when you delete it's not a merge so you know whether you deleted or not.

On Thu, Jul 7, 2016 at 5:11 PM Nathan VanBenschoten < notifications@github.com> wrote:

I'm starting to question if there is any way that we can estimate MVCCStats during the initialization of a Merge (in updateStatsOnMerge) in such a way that we can adjust MVCCComputeStats to produce identical results. The issue I'm seeing is that a given Merge for Timestamp values, which is currently the only situation where we use it, may not add any new information to an existing kv pair. For instance, if a new InternalTimeSeriesData contains only samples with already seen offsets (perhaps from a replayed raft command), the current value will not be changed. This contradicts the current comment for updateStatsOnMerge, which claims to "undercount", and I think makes it impossible to adjust MVCCComputeStats to agree with updateStatsOnMerge in all cases (perhaps unless we keep a "merge count" on the data and do some fancy calculations).

If we're willing to compromise on accuracy for ranges containing merges, maybe the easiest thing to do is to ignore merge value's effects on the stats completely. To do this, we could just remove updateStatsOnMerge, and ignore merge values (determined using IsTimeSeriesData) in MVCCComputeStats.

— You are receiving this because you authored the thread. Reply to this email directly, view it on GitHub https://github.com/cockroachdb/cockroach/issues/7290#issuecomment-231208725, or mute the thread https://github.com/notifications/unsubscribe/AE135F0tn-GXYiWfjgkrmZy7UvROyoubks5qTWuTgaJpZM4I3u7n .

-- Tobias

bdarnell commented 8 years ago

We can't just ignore time series data completely. We have ranges that consist solely of time series data, and those ranges need stats so we can split them. (We'd still have the keys, so these ranges wouldn't have size zero, but the ratio of value size to key size in the TS ranges is pretty low IIRC so we'd have a hard time managing the size of these ranges)

nvanbenschoten commented 8 years ago

@tschottdorf Do you mean with our current KeyCount field in stats? If so, I'm not sure I see an easy way to do that (which is why we currently don't). When we issue a Merge, we have no way of synchronously knowing if we're creating a new key or merging into an existing key, so we won't know whether to add to the count or not. Am I missing what you meant?

Also, you bring up a good point that as of now we will not add to LiveCount/KeyCount/ValCount when Merging, however we will subtract from them when Deleting merged keys. This could lead to negative counts.

@bdarnell Good point, we can't harmlessly ignore the time series data completely.

tbg commented 8 years ago

@nvanbenschoten no, I just didn't think this through.

petermattis commented 8 years ago

I'm not seeing any way to fix this either. So perhaps we give up on accurate stats when the merge operator is used. We want to keep the optimization to only compute the stats for the left side of a split in the common case. What if we added a field to MVCCStats that indicates that a merge operation has been applied? During a split, if we see that a merge operation has occurred on the range, we recompute the stats for both sides of the split and clear that field. I'm sure there are pitfalls to this approach. Please tear it apart.

bdarnell commented 8 years ago

If the stats are too low, the split will never happen. We need to maintain approximately valid stats in order to trigger splits.

Recomputing stats on splits will still leave windows of inconsistency that will be reported by the consistency checker unless we make it specially aware of MVCCStats and this merge flag.

tbg commented 8 years ago

if and only if https://github.com/cockroachdb/cockroach/issues/7611 (leaseholder-collected mostly accurate replicated stats) becomes a thing, this would actually fit into that category relatively nicely. Instead of *MVCCStats, we would pass around a larger object which has more counters, but which aren't necessarily authoritative. If we put timeseries in that category and then "ignored them" for the strict replicated stats (i.e. the current *MVCCStats), we'd be ok as long as we make sure that things which look at the "approximate real size" of the range use some method which sits on the top-level struct and sums up the corresponding contributions. Like @petermattis' approach, surely with pitfalls and asking to be torn apart.

petermattis commented 8 years ago

@bdarnell We do maintain approximate stats for merges. I'm not proposing we remove those approximations. Currently, they under-estimate the size of merges. We could change them to over-estimate so we split ranges containing merges too early.

Can you elaborate on how recomputing stats on splits leaves a window of inconsistency. I think the consistency checker computation was performed as a raft command. So long as the both replicas have performed the split or not performed the split we should be safe. Very likely I'm missing something here.

tbg commented 8 years ago

During a split, if we see that a merge operation has occurred on the range, we recompute the stats for both sides of the split and clear that field.

Mostly this just leaves a bad taste because I'd rather have a clearer separation of "true" and "approximate" stats, and because adding two ways in which a split can process is one too many.

Another potentially silly idea: Can we experiment with just reading the key after the merge to get the real stats? Sure, that isn't very performant, but with an iterator we should be able to use unsafe{Key,Value} to at least avoid pulling that memory on the Go heap. And then we have the precise numbers (we don't need to unmarshal, right?) and if it's not too horrible performance-wise, we should be ok?

vivekmenezes commented 8 years ago

inconsistency in issue #7693 might be related to this

On Fri, Jul 8, 2016, 8:06 AM Tobias Schottdorf notifications@github.com wrote:

During a split, if we see that a merge operation has occurred on the range, we recompute the stats for both sides of the split and clear that field.

Mostly this just leaves a bad taste because I'd rather have a clearer separation of "true" and "approximate" stats, and because adding two ways in which a split can process is one way too many.

Another potentially silly idea: Can we experiment with just reading the key after the merge to get the real stats? Sure, that isn't very performant, but with an iterator we should be able to use unsafe{Key,Value} to at least avoid pulling that memory on the Go heap. And then we have the precise numbers (we don't need to unmarshal, right?) and if it's not too horrible performance-wise, we should be ok?

— You are receiving this because you are subscribed to this thread. Reply to this email directly, view it on GitHub https://github.com/cockroachdb/cockroach/issues/7290#issuecomment-231343204, or mute the thread https://github.com/notifications/unsubscribe/ALOpBO80Uh5-OqtkB1Z-k_bV-e1eMWNBks5qTj0-gaJpZM4I3u7n .

petermattis commented 8 years ago

How is reading after the merge operation better performance than reading before the merge operation? If it isn't better performance then we'd be better off reading before the merge and essentially transforming merge into read-modify-write.

tbg commented 8 years ago

I think it would be less efficient because what you're suggesting is doing the merge in Go, right (at which point we remove Merge)? The C++ side is likely going to be better about memory, though I don't know what the overhead of doing a Merge when you're going to always read is.

tbg commented 8 years ago

Oh, you're probably suggesting doing the RMW in C++? That sounds good.

petermattis commented 8 years ago

C++ vs Go wasn't on my mind. I was wondering why doing the read after the merge would be better than getting rid of merge and replacing it with RMW. One benefit of the merge-then-read approach is that it keeps the write amplification as it is now. If we do a RMW then we'll be writing all of the timeseries data on every update. That sounds bad.

Actually, I think we'll need a read-then-merge approach because in order to update the stats we need to know the previous size of the timeseries data at a key. So we read, get the previous size, merge and also do a local merge with the previous data, get the new size. Phew. Let me explore what the performance overhead of this is going to be.

mrtracy commented 8 years ago

Merge-then-read is probably fine; that's essentially the performance that already exists when the Admin UI is open. Really, we have no established requirements for time series performance; I'm not quite sure how to go about benchmarking it.

I have another suggestion, as I do not understand RocksDB performance that well: what if we stored every sample in its own key? Currently, we group multiple samples over a set time period into a "slab" of samples stored in a single key (one sample every ten seconds over an hour, ~360 samples to a key). A non-merge alternative would be to write every sample to its own key. I don't know the relative performance profile of using more keys; intuitively it seems that it would be worse, but maybe it's not?

tbg commented 8 years ago

Even if that works, that seems more invasive (migrations?) and a lot more code changes. If what Pete suggests isn't horrible (read-modify-write), it's probably the way to go to get this issue off our plates.

-- Tobias

bdarnell commented 8 years ago

Can you elaborate on how recomputing stats on splits leaves a window of inconsistency.

Sorry, I thought the inconsistency was already there and the proposal was to fix it up during splits. But the data is already consistent thanks to our previous work in this area. Recomputing stats on split seems fine to me then, as long as the pre-split stats are close enough to accurate that the splits happen.

Can we experiment with just reading the key after the merge to get the real stats?

I believe merge operations are only worthwhile when the ratio of reads to writes is less than 1. If we read every time there's little (if any) benefit to using merge.

petermattis commented 8 years ago

I believe merge operations are only worthwhile when the ratio of reads to writes is less than 1. If we read every time there's little (if any) benefit to using merge.

Even if we read on every merge operation, there is still a benefit in using merge instead of write: we only write the new data. Consider the time series which are ~5KB when containing a full hour of data. Using merge to add a new sample means writing ~15 bytes vs a full write which would be ~5KB.

petermattis commented 8 years ago

If we're ok about losing historical timeseries data, then migration can be handled by changing the timeseries prefix and deleting keys containing the old prefix during compactions.

petermattis commented 8 years ago

Reading the previous timeseries before merging or reading the new timeseries after merging causes a pretty horrific performance decrease:

name                         old time/op  new time/op    delta
MVCCMergeTimeSeries_RocksDB  6.17µs ±10%  129.29µs ± 6%  +1995.69%  (p=0.000 n=10+10)

For the "new" I inserted a call to read the previous timeseries (without decoding it) in DBMerge. This performance decrease is so horrible that it feels like I must have done something wrong, though the change is only 5 lines of code:

~/Development/go/src/github.com/cockroachdb/cockroach/storage/engine master git diff
diff --git a/storage/engine/rocksdb/db.cc b/storage/engine/rocksdb/db.cc
index 8002968..bc66744 100644
--- a/storage/engine/rocksdb/db.cc
+++ b/storage/engine/rocksdb/db.cc
@@ -1528,6 +1528,11 @@ DBStatus DBSnapshot::Merge(DBKey key, DBSlice value) {
 }

 DBStatus DBMerge(DBEngine* db, DBKey key, DBSlice value) {
+  DBString prev_value;
+  DBStatus status = db->Get(key, &prev_value);
+  if (prev_value.data != nullptr) {
+    free(prev_value.data);
+  }
   return db->Merge(key, value);
 }
nvanbenschoten commented 8 years ago

I implemented a fix for this in #7721 using the strategy from @petermattis:

What if we added a field to MVCCStats that indicates that a merge operation has been applied? During a split, if we see that a merge operation has occurred on the range, we recompute the stats for both sides of the split and clear that field.

I haven't been able to find any pitfalls with this approach (besides the extra logic in splitTrigger, which isn't too bad), and it seems to at least fix the immediate issue.