Mmuzaf commented on code in PR #3102:
URL: https://github.com/apache/cassandra/pull/3102#discussion_r1713929010
##########
src/java/org/apache/cassandra/metrics/DecayingEstimatedHistogramReservoir.java:
##########
@@ -371,42 +363,50 @@ private void rescaleIfNeeded(long now)
{
try
{
- rescale(now);
+ decayingBuckets = rescale(now);
}
finally
{
- decayLandmark = now;
rescaling.set(false);
}
}
}
}
- private void rescale(long now)
+ /**
+ * rescale races with update;
+ * If rescale is called at the same time as an update, and an update falls
into an already
+ * processed bucket this update will be lost. We accept that for the sake
of update performance.
+ * OTOH, if the update falls into a bucket that is yet to be processed, it
will be added
+ * with a weight that is consistent with the rescaleFactor used by
rescale, so the new rescaled
+ * buckets will all have values consistent with the new decay landmark.
+ */
+ private DecayingBuckets rescale(long now)
{
- // despite striping its safe to rescale each bucket individually
- final double rescaleFactor = forwardDecayWeight(now);
- for (int i = 0; i < decayingBuckets.length(); i++)
+ final double rescaleFactor = decayingBuckets.forwardDecayWeight(now);
+ DecayingBuckets newBuckets = new DecayingBuckets(now);
+ for (int i = 0; i < decayingBuckets.buckets.length(); i++)
{
- long newValue = Math.round(decayingBuckets.get(i) / rescaleFactor);
- decayingBuckets.set(i, newValue);
+ long newValue = Math.round(decayingBuckets.buckets.get(i) /
rescaleFactor);
+ newBuckets.buckets.set(i, newValue);
}
+ return newBuckets;
}
private boolean needRescale(long now)
{
- return (now - decayLandmark) > LANDMARK_RESET_INTERVAL_IN_NS;
+ return (now - decayingBuckets.decayLandmark) >
LANDMARK_RESET_INTERVAL_IN_NS;
}
@VisibleForTesting
public void clear()
{
- final int bucketCount = decayingBuckets.length();
+ final int bucketCount = buckets.length();
for (int i = 0; i < bucketCount; i++)
{
- decayingBuckets.set(i, 0L);
buckets.set(i, 0L);
}
+ decayingBuckets = new DecayingBuckets(decayingBuckets.decayLandmark);
Review Comment:
Let's use the `AtomicReferenceFieldUpdater` to update the `decayingBuckets`
(instead of the `rescaling` field) for both the clear and rescaleIfNeeded
methods. Even if the former is only used in tests now, the changes should be
consistent making any further changes safe without worrying about concurrency.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]