-
Notifications
You must be signed in to change notification settings - Fork 3.8k
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
kvserver: investigate raft entry cache sizing #98666
Comments
cc @cockroachdb/replication |
I'd also expect this to still be in OS disk caches. |
This tracks the cases where we fall back to reading log entries from storage (i.e. pebble) in `(raft.Storage.Entries)` Ideally this is zero, as everything ought to be served from the raft entry cache. We know that this cache is not configured well[^1] and so we can't really expect this to work, but you can't improve what you don't measure. Additionally, this metric has been useful in investigations related to raft overload[^2]. [^1]: cockroachdb#98666 [^2]: cockroachdb#98576 Epic: none Release note: None
This tracks the cases where we fall back to reading log entries from storage (i.e. pebble) in `(raft.Storage.Entries)` Ideally this is zero, as everything ought to be served from the raft entry cache. We know that this cache is not configured well[^1] and so we can't really expect this to work, but you can't improve what you don't measure. Additionally, this metric has been useful in investigations related to raft overload[^2]. [^1]: cockroachdb#98666 [^2]: cockroachdb#98576 Epic: none Release note: None
This tracks the cases where we fall back to reading log entries from storage (i.e. pebble) in `(raft.Storage.Entries)` Ideally this is zero, as everything ought to be served from the raft entry cache. We know that this cache is not configured well[^1] and so we can't really expect this to work, but you can't improve what you don't measure. Additionally, this metric has been useful in investigations related to raft overload[^2]. [^1]: #98666 [^2]: #98576 Epic: none Release note: None
Informs cockroachdb#98666. This commit introduces a new `COCKROACH_RAFT_ENTRY_CACHE_SIZE` which can be used to configure the size of the raft entry cache. The default value is 16 MiB. Release note: None
In a write-heavy workload running This demonstrates that for write-heavy workloads, a 16mb raft cache entry cache is insufficient. We should explore dynamically sizing the cache based on available memory capacity in the system. In the meantime, #105799 makes it configurable through an environment variable. p99 raft scheduler latency: notice node 2 and 3, compared to the other nodes raft entry cache accesses and hits same data, tabular form
cpu profile from n3 and n4: notice n3's time reading the raft log from pebble |
Interesting, thanks for these results. There are other actors here too -- we've seen previously (#78412) that performance tanks when recent Raft log entries are flushed from the memtable into L0 before application and truncation, and the Pebble block cache and OS caches also come into play. I suspect the memtable flushing primarily affects performance by increasing compaction disk IO, competing with foreground writes. However, it'd be interesting to see if it would be sufficient/better to try to have the log entries stick around in the memtable for as long as possible, instead of relying on a separate entry cache, since that would solve two problems at once. |
That said, let's see if we can do something tactical for 23.2 -- 16 MB is miniscule. |
I'm unsure how much keeping the entries in the memtable will help with these reads, because any read other than a point |
Good point, I guess that answers that. |
104350: sql/schemachanger: support for create sequence inside the declarative schema changer r=fqazi a=fqazi This patch implements CREATE SEQUENCE in the declarative schema changer. The first three commits can be ignored and are included in (#104348, which should be reviewed and merged first). This patch will: - Skip validation/backfill for descriptors in an added state - Fixes prefix resolution creating new objects, where two-part names were not handled properly before - Adds support for CREATE sequence in the declarative schema changer Fixes: #104351 105454: roachtest: update `mixedversion` to always use secure clusters r=srosenberg a=renatolabs Secure clusters are closer to production deployments and also allow us to tests features that we couldn't before, like creating new users with passwords during a test, and then performing SQL operations with those users. In the process of getting this to work, there were a few bugs that needed to be fixed (first commit), and the `cluster` interface needed a small update as well (second commit). Epic: CRDB-19321 Release note: None 105765: ui: fix db page Node/Regions column rendering r=xinhaoz a=xinhaoz Previously, the db page was not updating its columns if the `showNodeRegionsColumn` prop changed. The db details page was also not filtering out the regions column when desired. Epic: none Release note (bug fix): node/regions columns in db and db details page should properly render. This column is hidden for tenants and otherwise is shown for clusters with > 1 node. 105770: persistedsqlstats: skip TestSQLStatsPersistedLimitReached under stress r=zachlite a=zachlite TestSQLStatsPersistedLimitReached succeeds under 'normal' testing conditions. We can and should get this test coverage when we can. Informs #97488 Epic: none Release note: None 105799: kv: expose env var to configure raft entry cache size r=erikgrinaker a=nvanbenschoten Informs #98666. This commit introduces a new `COCKROACH_RAFT_ENTRY_CACHE_SIZE` which can be used to configure the size of the raft entry cache. The default value is 16 MiB. Release note: None Co-authored-by: Faizan Qazi <[email protected]> Co-authored-by: Renato Costa <[email protected]> Co-authored-by: Xin Hao Zhang <[email protected]> Co-authored-by: zachlite <[email protected]> Co-authored-by: Nathan VanBenschoten <[email protected]>
Informs cockroachdb#98666. This commit introduces a new `COCKROACH_RAFT_ENTRY_CACHE_SIZE` which can be used to configure the size of the raft entry cache. The default value is 16 MiB. Release note: None
Informs cockroachdb#98666. This commit introduces a new `COCKROACH_RAFT_ENTRY_CACHE_SIZE` which can be used to configure the size of the raft entry cache. The default value is 16 MiB. Release note: None
Informs cockroachdb#98666. This commit introduces a new `COCKROACH_RAFT_ENTRY_CACHE_SIZE` which can be used to configure the size of the raft entry cache. The default value is 16 MiB. Release note: None
Informs cockroachdb#98666. This commit introduces a new `COCKROACH_RAFT_ENTRY_CACHE_SIZE` which can be used to configure the size of the raft entry cache. The default value is 16 MiB. Release note: None
I sampled CentMon hit rates:
|
Some quick benchmarks, recording hit rates for all 3 nodes:
Will run some experiments with the small-node variants. |
3 runs of
While the cache hit rate is significantly improved compared to #98666 (comment), it unfortunately doesn't seem to have a significant impact on throughput for this specific workload. The results in #98666 (comment) were on AWS though, which I believe tends to have lower IOPS capacity. I'll rerun that experiment with a 128 MB entry cache. |
I re-ran the experiments from #98666 (comment), using 16 MB and 128 MB entry cache sizes respectively. I saw similar cluster behavior as the original results, including the throughput degradation and node bimodality. The 128 MB entry cache basically eliminated cache misses. Furthermore, these are 256 GB RAM nodes, so if we had used the linear 1/256 memory scaling from #107424 the entry cache would have been 1 GB, which almost certainly would have ensured a 100% hit rate. Per-node hit rates:
Unfortunately, as with other results, this did not translate to a significant throughput improvement. In fact, the 16 MB cluster had higher throughput (40127 w/s vs. 38865 w/s) which is most likely down to VM variation. Furthermore, the 128 MB cluster also showed Raft scheduler bimodality that did not correlate strongly with entry cache misses: The CPU profile from n3 (one with high scheduler latencies) confirms that we're not reading entries from disk: n1 with low Raft scheduler latency appears very similar: For the purposes of the entry cache, I think we can say that the linear 1/256 scaling proposed in #107424 appears sufficient to avoid cache misses, but the impact on throughput is unclear. |
107265: liveness: allow registering callbacks after start r=erikgrinaker a=tbg I discovered[^1] a deadlock scenario when multiple nodes in the cluster restart with additional stores that need to be bootstrapped. In that case, liveness must be running when the StoreIDs are allocated, but it is not. Trying to address this problem, I realized that when an auxiliary Store is bootstrapped, it will create a new replicateQueue, which will register a new callback into NodeLiveness. But if liveness must be started at this point to fix #106706, we'll run into the assertion that checks that we don't register callbacks on a started node liveness. Something's got to give: we will allow registering callbacks at any given point in time, and they'll get an initial set of notifications synchronously. I audited the few users of RegisterCallback and this seems OK with all of them. [^1]: #106706 (comment) Epic: None Release note: None 107417: kvserver: ignore RPC conn when deciding to campaign/vote r=erikgrinaker a=erikgrinaker **kvserver: remove stale mayCampaignOnWake comment** The comment is about a parameter that no longer exists. **kvserver: revamp shouldCampaign/Forget tests** **kvserver: ignore RPC conn in `shouldCampaignOnWake`** Previously, `shouldCampaignOnWake()` used `IsLiveMapEntry.IsLive` to determine whether the leader was dead. However, this not only depends on the node's liveness, but also its RPC connectivity. This can prevent an unquiescing replica from acquiring Raft leadership if the leader is still alive but unable to heartbeat liveness, and the leader will be unable to acquire epoch leases in this case. This patch ignores the RPC connection state when deciding whether to campaign, using only on the liveness state. **kvserver: ignore RPC conn in `shouldForgetLeaderOnVoteRequest`** Previously, `shouldForgetLeaderOnVoteRequest()` used `IsLiveMapEntry.IsLive` to determine whether the leader was dead. However, this not only depends on the node's liveness, but also its RPC connectivity. This can prevent granting votes to a new leader that may be attempting to acquire a epoch lease (which the current leader can't). This patch ignores the RPC connection state when deciding whether to campaign, using only on the liveness state. Resolves #107060. Epic: none Release note: None **kvserver: remove `StoreTestingKnobs.DisableLivenessMapConnHealth`** 107424: kvserver: scale Raft entry cache size with system memory r=erikgrinaker a=erikgrinaker The Raft entry cache size defaulted to 16 MB, which is rather small. This has been seen to cause tail latency and throughput degradation with high write volume on large nodes, correlating with a reduction in the entry cache hit rate. This patch linearly scales the Raft entry cache size as 1/256 of total system/cgroup memory, shared evenly between all stores, with a minimum 32 MB. For example, a 32 GB 8-vCPU node will have a 128 MB entry cache. This is a conservative default, since this memory is not accounted for in existing memory budgets nor by the `--cache` flag. We rarely see cache misses in production clusters anyway, and have seen significantly improved hit rates with this scaling (e.g. a 64 KB kv0 workload on 8-vCPU nodes increased from 87% to 99% hit rate). Resolves #98666. Epic: none Release note (performance improvement): The default Raft entry cache size has been increased from 16 MB to 1/256 of system memory with a minimum of 32 MB, divided evenly between all stores. This can be configured via `COCKROACH_RAFT_ENTRY_CACHE_SIZE`. 107442: kvserver: deflake TestRequestsOnFollowerWithNonLiveLeaseholder r=erikgrinaker a=tbg The test previously relied on aggressive liveness heartbeat expirations to avoid running for too long. As a result, it was flaky since liveness wasn't reliably pinned in the way the test wanted. The hybrid manual clock allows time to jump forward at an opportune moment. Use it here to avoid running with a tight lease interval. On my gceworker, previously flaked within a few minutes. As of this commit, I ran it for double-digit minutes without issue. Fixes #107200. Epic: None Release note: None 107526: kvserver: fail gracefully in TestLeaseTransferRejectedIfTargetNeedsSnapshot r=erikgrinaker a=tbg We saw this test hang in CI. What likely happened (according to the stacks) is that a lease transfer that was supposed to be caught by an interceptor never showed up in the interceptor. The most likely explanation is that it errored out before it got to evaluation. It then signaled a channel the test was only prepared to check later, so the test hung (waiting for a channel that was now never to be touched). This test is hard to maintain. It would be great (though, for now, out of reach) to write tests like it in a deterministic framework[^1] [^1]: see #105177. For now, fix the test so that when the (so far unknown) error rears its head again, it will fail properly, so we get to see the error and can take another pass at fixing the test (separately). Stressing this commit[^2], we get: > transferErrC unexpectedly signaled: /Table/Max: transfer lease unexpected > error: refusing to transfer lease to (n3,s3):3 because target may need a Raft > snapshot: replica in StateProbe This makes sense. The test wants to exercise the below-raft mechanism, but the above-raft mechanism also exists and while we didn't want to interact with it, we sometimes do[^1] The second commit introduces a testing knob that disables the above-raft mechanism selectively. I've stressed the test for 15 minutes without issues after this change. [^1]: somewhat related to #107524 [^2]: `./dev test --filter TestLeaseTransferRejectedIfTargetNeedsSnapshot --stress ./pkg/kv/kvserver/` on gceworker, 285s Fixes #106383. Epic: None Release note: None 107531: kvserver: disable replicate queue and lease transfers in closedts tests r=erikgrinaker a=tbg For a more holistic suggestion on how to fix this for the likely many other tests susceptible to similar issues, see: #107528 > 1171 runs so far, 0 failures, over 15m55s Fixes #101824. Release note: None Epic: none Co-authored-by: Tobias Grieger <[email protected]> Co-authored-by: Erik Grinaker <[email protected]>
The raft entry cache is sized to 16mb per store. This seems very small:
cockroach/pkg/kv/kvserver/store.go
Lines 103 to 105 in e005c0f
The
raft.entrycache.{accesses,hits}
metrics can be used to arrive at a cache hit rate. This is something we should be paying attention to when optimizing write performance.x-ref #98576 where this was noticed.
Jira issue: CRDB-25431
The text was updated successfully, but these errors were encountered: