[
https://issues.apache.org/jira/browse/CASSANDRA-20333?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17929133#comment-17929133
]
Benedict Elliott Smith commented on CASSANDRA-20333:
----------------------------------------------------
With a thread local buffer, there are plenty of options for avoiding losing
recent updates without blocking readers or writers.
A ThreadLocal buffer could be flushed using an asynchronous lock; that is, if
the lock is already taken we simply move to a new buffer to continue writing,
retaining the prior buffer to be flushed later - either by the current owner of
the lock, or the next scheduled update or next reader of the histogram. This
way, updates are never lost.
With an exclusive writer to the global snapshot histogram, we can avoid any
issue with inconsistent data. There is only a single writer, so there is no
problem of another non-rebase thread witnessing the new decay weight and using
it on non-rebased data. The rebase writer thread can also publish two counters
indicating its progress, and a reader can consult these counters to detect a
race, or else determine which side of the rebase operation their read occurred
on. There is only a narrow race window where they would have to wait in this
scenario, which should be well tolerated with e.g. a parkNanos loop.
We can easily tune the size of any thread local buffer to minimise competition
while ensuring we do not use an unreasonable amount of memory. Today, we use
more than 1GiB for metrics iirc, and with the other improvements Dmitry is
making (plus possible follow-up improvements to reduce the per-thread overhead)
I expect we will be able to dramatically _reduce_ the metric overhead, freeing
up plenty of headroom for any modest additional memory required here.
> Reduce DecayingEstimatedHistogramReservoir update cost
> ------------------------------------------------------
>
> Key: CASSANDRA-20333
> URL: https://issues.apache.org/jira/browse/CASSANDRA-20333
> Project: Apache Cassandra
> Issue Type: Improvement
> Components: Observability/Metrics
> Reporter: Dmitry Konstantinov
> Assignee: Dmitry Konstantinov
> Priority: Normal
>
> Based on the discussions in CASSANDRA-20250
> [~benedict]:
> {quote}We can probably improve our reservoir performance if we want to,
> perhaps in a follow-up patch? For instance, we could have a small
> thread-local buffer of (time, latency) pairs that we periodically flush
> together, so that we amortise the memory latency costs. Or we could explore
> maintaining a per-thread HdrHistogram, that we periodically flush. This would
> be a good time to explore fully migrating to HdrHistogram, as it has built-in
> merge semantics iirc. I am not sure what the decayed version would look like
> there, but I am certain we could maintain a separate decayed HdrHistogram.
> Having a thread-local buffer of updates we intend to flush to the histograms
> would amortise the latency penalties without fundamentally redesigning
> anything (as well as reducing contention).
> Other possibilities might include e.g. changing the bucket distribution so we
> don't need a LUT for computing lg2, although the above would gracefully
> handle any contribution this has as well.
> {quote}
>
> Other ideas about squeezing extra bits from the current design:
> * bucket id can be calculated once (currently we do it 2 times for decaying
> and current buckets), like:
> {code:java}
> int stripe = (int) (Thread.currentThread().getId() & (nStripes - 1));
> int bucket = stripedIndex(index, stripe);
> rescaledDecayingBuckets.update(bucket, now);
> updateBucket(buckets, bucket, 1); {code}
> * for histograms on highly loaded paths we can use another number of stripes
> (by default it is 2, we can set for example 4 for them)
> * I noticed some variation in performance for a micro-benchmark (existing
> one: DecayingEstimatedHistogramBench) depending on what exact value for
> distributionPrime is used (but I need to double check it)
> * forwardDecayWeight function depends on SampledClock value, so we can try
> to recalculate the weight only when time is changed
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]