[jira] [Commented] (CASSANDRA-11117) ColUpdateTimeDeltaHistogram histogram overflow
[ https://issues.apache.org/jira/browse/CASSANDRA-7?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15548934#comment-15548934 ] Joel Knighton commented on CASSANDRA-7: --- I considered that as well. My main concern was that it masks the difference between the two different behaviors of having a genuinely high timestamp delta and have the appearance of one due to counter behavior/empty LivenessInfo/etc, but the more I think about it, the more I think the approach you describe above is better. In the case where it is a particular storage engine path (as I described above), these wouldn't be recurring, so they would be quickly lost to histogram decay anyway. I'll update the patch - thanks! > ColUpdateTimeDeltaHistogram histogram overflow > -- > > Key: CASSANDRA-7 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7 > Project: Cassandra > Issue Type: Bug >Reporter: Chris Lohfink >Assignee: Joel Knighton >Priority: Minor > Fix For: 2.2.x, 3.0.x, 3.x > > > {code} > getting attribute Mean of > org.apache.cassandra.metrics:type=ColumnFamily,name=ColUpdateTimeDeltaHistogram > threw an exceptionjavax.management.RuntimeMBeanException: > java.lang.IllegalStateException: Unable to compute ceiling for max when > histogram overflowed > {code} > Although the fact that this histogram has 164 buckets already, I wonder if > there is something weird with the computation thats causing this to be so > large? It appears to be coming from updates to system.local > {code} > org.apache.cassandra.metrics:type=Table,keyspace=system,scope=local,name=ColUpdateTimeDeltaHistogram > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-11117) ColUpdateTimeDeltaHistogram histogram overflow
[ https://issues.apache.org/jira/browse/CASSANDRA-7?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15547998#comment-15547998 ] Marcus Eriksson commented on CASSANDRA-7: - I like the approach in the patch but should we perhaps do; {code} if (timeDelta < Long.MAX_VALUE) metric.colUpdateTimeDeltaHistogram.update(Math.min(18165375903306L, timeDelta)); {code} so that we at least record the fact that we had a big time difference? I would assume operators would be interested in these large deviations (we would probably need to document that we bunch together everything > 18165375903306L) > ColUpdateTimeDeltaHistogram histogram overflow > -- > > Key: CASSANDRA-7 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7 > Project: Cassandra > Issue Type: Bug >Reporter: Chris Lohfink >Assignee: Joel Knighton >Priority: Minor > Fix For: 2.2.x, 3.0.x, 3.x > > > {code} > getting attribute Mean of > org.apache.cassandra.metrics:type=ColumnFamily,name=ColUpdateTimeDeltaHistogram > threw an exceptionjavax.management.RuntimeMBeanException: > java.lang.IllegalStateException: Unable to compute ceiling for max when > histogram overflowed > {code} > Although the fact that this histogram has 164 buckets already, I wonder if > there is something weird with the computation thats causing this to be so > large? It appears to be coming from updates to system.local > {code} > org.apache.cassandra.metrics:type=Table,keyspace=system,scope=local,name=ColUpdateTimeDeltaHistogram > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-11117) ColUpdateTimeDeltaHistogram histogram overflow
[ https://issues.apache.org/jira/browse/CASSANDRA-7?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15545512#comment-15545512 ] Joshua McKenzie commented on CASSANDRA-7: - [~krummas] to review. > ColUpdateTimeDeltaHistogram histogram overflow > -- > > Key: CASSANDRA-7 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7 > Project: Cassandra > Issue Type: Bug >Reporter: Chris Lohfink >Assignee: Joel Knighton >Priority: Minor > Fix For: 2.2.x, 3.0.x, 3.x > > > {code} > getting attribute Mean of > org.apache.cassandra.metrics:type=ColumnFamily,name=ColUpdateTimeDeltaHistogram > threw an exceptionjavax.management.RuntimeMBeanException: > java.lang.IllegalStateException: Unable to compute ceiling for max when > histogram overflowed > {code} > Although the fact that this histogram has 164 buckets already, I wonder if > there is something weird with the computation thats causing this to be so > large? It appears to be coming from updates to system.local > {code} > org.apache.cassandra.metrics:type=Table,keyspace=system,scope=local,name=ColUpdateTimeDeltaHistogram > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-11117) ColUpdateTimeDeltaHistogram histogram overflow
[ https://issues.apache.org/jira/browse/CASSANDRA-7?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15542557#comment-15542557 ] Joel Knighton commented on CASSANDRA-7: --- ||branch||testall||dtest|| |[CASSANDRA-7-2.2|https://github.com/jkni/cassandra/tree/CASSANDRA-7-2.2]|[testall|http://cassci.datastax.com/view/Dev/view/jkni/job/jkni-CASSANDRA-7-2.2-testall]|[dtest|http://cassci.datastax.com/view/Dev/view/jkni/job/jkni-CASSANDRA-7-2.2-dtest]| |[CASSANDRA-7-3.0|https://github.com/jkni/cassandra/tree/CASSANDRA-7-3.0]|[testall|http://cassci.datastax.com/view/Dev/view/jkni/job/jkni-CASSANDRA-7-3.0-testall]|[dtest|http://cassci.datastax.com/view/Dev/view/jkni/job/jkni-CASSANDRA-7-3.0-dtest]| |[CASSANDRA-7-3.X|https://github.com/jkni/cassandra/tree/CASSANDRA-7-3.X]|[testall|http://cassci.datastax.com/view/Dev/view/jkni/job/jkni-CASSANDRA-7-3.X-testall]|[dtest|http://cassci.datastax.com/view/Dev/view/jkni/job/jkni-CASSANDRA-7-3.X-dtest]| |[CASSANDRA-7-trunk|https://github.com/jkni/cassandra/tree/CASSANDRA-7-trunk]|[testall|http://cassci.datastax.com/view/Dev/view/jkni/job/jkni-CASSANDRA-7-trunk-testall]|[dtest|http://cassci.datastax.com/view/Dev/view/jkni/job/jkni-CASSANDRA-7-trunk-dtest]| I've linked patches above. CI for the patches look clean relative to upstream. For 2.2, I found write paths in Thrift for accessing counters that reproduced this behavior. In 3.0+, I found instances where UPDATE -> INSERT patterns reproduced this behavior, when an update to a nonexistent row created an empty {{LivenessInfo}}. This behavior is also possible to reproduce just by using client-specified timestamps. This leads us to a few options to fix this issue; we could look through the read/write path and try special case handling of the first two issues (and any others we would have to discover), but that wouldn't solve the client-specified timestamp behavior. For that reason, I opted in to simply filtering to values that wouldn't overflow the histogram. This limits the ColUpdateTimeDelta histogram to reflecting updates under normal conditions that are under about 100 days. The risk of the first approach is introducing additional complexity in the read/write path for a fairly niche metric. The risk of the second approach is that it reduces the amount of the information in the ColUpdateTimeDelta histogram. I do not think this significantly reduces the utility of the metric as proposed in [CASSANDRA-7979], which is to somehow quantify how frequently columns are updated. A single decaying histogram measurement (as for the second access in the counter and update/insert examples above) has limited value, and samples above 100 days don't impose a reasonable time skew constraint. I'm not necessarily opposed to the first option. If someone does have a strong preference, I think I'll defer on a patch here to someone more experienced with the read/write path. I'd be happy to review. > ColUpdateTimeDeltaHistogram histogram overflow > -- > > Key: CASSANDRA-7 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7 > Project: Cassandra > Issue Type: Bug >Reporter: Chris Lohfink >Assignee: Joel Knighton >Priority: Minor > Fix For: 2.2.x, 3.0.x, 3.x > > > {code} > getting attribute Mean of > org.apache.cassandra.metrics:type=ColumnFamily,name=ColUpdateTimeDeltaHistogram > threw an exceptionjavax.management.RuntimeMBeanException: > java.lang.IllegalStateException: Unable to compute ceiling for max when > histogram overflowed > {code} > Although the fact that this histogram has 164 buckets already, I wonder if > there is something weird with the computation thats causing this to be so > large? It appears to be coming from updates to system.local > {code} > org.apache.cassandra.metrics:type=Table,keyspace=system,scope=local,name=ColUpdateTimeDeltaHistogram > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-11117) ColUpdateTimeDeltaHistogram histogram overflow
[ https://issues.apache.org/jira/browse/CASSANDRA-7?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15525530#comment-15525530 ] Vadim Tsesko commented on CASSANDRA-7: -- I tried to trace the problem using Cassandra 3.7. The problematic value overflowing the histogram is added in the following stack trace: {code:java} at org.apache.cassandra.utils.EstimatedHistogram.add(EstimatedHistogram.java:134) [apache-cassandra-3.7.jar:3.7] at org.apache.cassandra.metrics.EstimatedHistogramReservoir.update(EstimatedHistogramReservoir.java:56) [apache-cassandra-3.7.jar:3.7] at com.codahale.metrics.Histogram.update(Histogram.java:39) [metrics-core-3.1.0.jar:3.1.0] at org.apache.cassandra.metrics.TableMetrics$TableHistogram.update(TableMetrics.java:828) [apache-cassandra-3.7.jar:3.7] at org.apache.cassandra.db.ColumnFamilyStore.apply(ColumnFamilyStore.java:1331) [apache-cassandra-3.7.jar:3.7] at org.apache.cassandra.db.Keyspace.apply(Keyspace.java:529) [apache-cassandra-3.7.jar:3.7] at org.apache.cassandra.db.Keyspace.apply(Keyspace.java:399) [apache-cassandra-3.7.jar:3.7] at org.apache.cassandra.db.Mutation.applyFuture(Mutation.java:202) [apache-cassandra-3.7.jar:3.7] at org.apache.cassandra.db.Mutation.apply(Mutation.java:214) [apache-cassandra-3.7.jar:3.7] at org.apache.cassandra.db.Mutation.apply(Mutation.java:228) [apache-cassandra-3.7.jar:3.7] at org.apache.cassandra.service.StorageProxy$8.runMayThrow(StorageProxy.java:1343) ~[apache-cassandra-3.7.jar:3.7] at org.apache.cassandra.service.StorageProxy$LocalMutationRunnable.run(StorageProxy.java:2519) ~[apache-cassandra-3.7.jar:3.7] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[na:1.8.0_102] at org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:164) ~[apache-cassandra-3.7.jar:3.7] at org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:136) ~[apache-cassandra-3.7.jar:3.7] at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) ~[apache-cassandra-3.7.jar:3.7] at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_102] {code} The problem is caused by using time interval in {{ColumnFamilyStore.apply()}} returned by {{Memtable.put()}} which is calculated by {{Rows.merge()}} method. It seems that in case of an update for a nonexisting row an *absolute* time value is returned, not time delta. That leads to very large values being sent to the histogram (as large as {{Long.MAX_VALUE / 10}} judging by debugging on real data). We decided to patch Cassandra and to ignore the overflowed histogram values temporarily. > ColUpdateTimeDeltaHistogram histogram overflow > -- > > Key: CASSANDRA-7 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7 > Project: Cassandra > Issue Type: Bug >Reporter: Chris Lohfink >Assignee: Joel Knighton >Priority: Minor > Fix For: 2.2.x, 3.0.x, 3.x > > > {code} > getting attribute Mean of > org.apache.cassandra.metrics:type=ColumnFamily,name=ColUpdateTimeDeltaHistogram > threw an exceptionjavax.management.RuntimeMBeanException: > java.lang.IllegalStateException: Unable to compute ceiling for max when > histogram overflowed > {code} > Although the fact that this histogram has 164 buckets already, I wonder if > there is something weird with the computation thats causing this to be so > large? It appears to be coming from updates to system.local > {code} > org.apache.cassandra.metrics:type=Table,keyspace=system,scope=local,name=ColUpdateTimeDeltaHistogram > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-11117) ColUpdateTimeDeltaHistogram histogram overflow
[ https://issues.apache.org/jira/browse/CASSANDRA-7?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15430271#comment-15430271 ] Clément Lardeur commented on CASSANDRA-7: - Hi we have the same problem with the version 3.0.8, the graphite reporter stop working like CASSANDRA-12185. {noformat} ERROR [metrics-graphite-reporter-1-thread-1] 2016-08-22 09:40:42,618 ScheduledReporter.java:119 - RuntimeException thrown from GraphiteReporter#report. Exception was suppressed. java.lang.IllegalStateException: Unable to compute ceiling for max when histogram overflowed at org.apache.cassandra.utils.EstimatedHistogram.rawMean(EstimatedHistogram.java:231) ~[apache-cassandra-3.0.8.jar:3.0.8] at org.apache.cassandra.metrics.EstimatedHistogramReservoir$HistogramSnapshot.getMean(EstimatedHistogramReservoir.java:103) ~[apache-cassandra-3.0.8.jar:3.0.8] at com.codahale.metrics.graphite.GraphiteReporter.reportHistogram(GraphiteReporter.java:265) ~[metrics-graphite-3.1.2.jar:3.1.2] at com.codahale.metrics.graphite.GraphiteReporter.report(GraphiteReporter.java:179) ~[metrics-graphite-3.1.2.jar:3.1.2] at com.codahale.metrics.ScheduledReporter.report(ScheduledReporter.java:162) ~[metrics-core-3.1.0.jar:3.1.0] at com.codahale.metrics.ScheduledReporter$1.run(ScheduledReporter.java:117) ~[metrics-core-3.1.0.jar:3.1.0] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_101] at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) [na:1.8.0_101] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) [na:1.8.0_101] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) [na:1.8.0_101] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_101] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_101] at java.lang.Thread.run(Thread.java:745) [na:1.8.0_101] {noformat} I don't think that ticket should be flagged as Minor because it's impacting our production monitoring of the cluster. Since the version 3.x with the incompatibility of OpsCenter, many users switch to another monitoring solution so the graphite reporter is now more critical for the OOS community than previously. If I could help I can give you more details with our cluster settings. > ColUpdateTimeDeltaHistogram histogram overflow > -- > > Key: CASSANDRA-7 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7 > Project: Cassandra > Issue Type: Bug >Reporter: Chris Lohfink >Assignee: Joel Knighton >Priority: Minor > Fix For: 2.2.x, 3.0.x, 3.x > > > {code} > getting attribute Mean of > org.apache.cassandra.metrics:type=ColumnFamily,name=ColUpdateTimeDeltaHistogram > threw an exceptionjavax.management.RuntimeMBeanException: > java.lang.IllegalStateException: Unable to compute ceiling for max when > histogram overflowed > {code} > Although the fact that this histogram has 164 buckets already, I wonder if > there is something weird with the computation thats causing this to be so > large? It appears to be coming from updates to system.local > {code} > org.apache.cassandra.metrics:type=Table,keyspace=system,scope=local,name=ColUpdateTimeDeltaHistogram > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-11117) ColUpdateTimeDeltaHistogram histogram overflow
[ https://issues.apache.org/jira/browse/CASSANDRA-7?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15380172#comment-15380172 ] Jeff Griffith commented on CASSANDRA-7: --- Yes, same here. Upgraded from 2.1. > ColUpdateTimeDeltaHistogram histogram overflow > -- > > Key: CASSANDRA-7 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7 > Project: Cassandra > Issue Type: Bug >Reporter: Chris Lohfink >Assignee: Joel Knighton >Priority: Minor > Fix For: 2.2.x, 3.0.x, 3.x > > > {code} > getting attribute Mean of > org.apache.cassandra.metrics:type=ColumnFamily,name=ColUpdateTimeDeltaHistogram > threw an exceptionjavax.management.RuntimeMBeanException: > java.lang.IllegalStateException: Unable to compute ceiling for max when > histogram overflowed > {code} > Although the fact that this histogram has 164 buckets already, I wonder if > there is something weird with the computation thats causing this to be so > large? It appears to be coming from updates to system.local > {code} > org.apache.cassandra.metrics:type=Table,keyspace=system,scope=local,name=ColUpdateTimeDeltaHistogram > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-11117) ColUpdateTimeDeltaHistogram histogram overflow
[ https://issues.apache.org/jira/browse/CASSANDRA-7?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15380061#comment-15380061 ] Joel Knighton commented on CASSANDRA-7: --- The Long.MAX_VALUE in the code snippet you've provided checks for a Long.MAX_VALUE sentinel value we pass along when there are no relevant column update deltas in the reconciliation path of the write. This shouldn't be the problem. The problem is when certain cells/liveness infos in the reconciliation path carry very high timestamp values or possibly Long.MIN_VALUE as a timestamp value (in some of the 3.0 cases). I can reproduce this in 3.0+; the same methods for reproduction do not work in 2.2. [~urandom], [~jeffery.griffith], you reported this issue on 2.2 clusters. I see that your cluster was upgraded from an earlier version [~urandom]. Was yours as well [~jeffery.griffith]? To help write tests for and validate the fix I have in mind, any details either of you can share on - table schemas (in particular, are you using static columns? collections? counters? user-defined anything?) - access patterns (insert/updates/deletes) - ttls/specified timestamps might help me find a reproduction path for 2.2. Thanks! > ColUpdateTimeDeltaHistogram histogram overflow > -- > > Key: CASSANDRA-7 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7 > Project: Cassandra > Issue Type: Bug >Reporter: Chris Lohfink >Assignee: Joel Knighton >Priority: Minor > Fix For: 2.2.x, 3.0.x, 3.x > > > {code} > getting attribute Mean of > org.apache.cassandra.metrics:type=ColumnFamily,name=ColUpdateTimeDeltaHistogram > threw an exceptionjavax.management.RuntimeMBeanException: > java.lang.IllegalStateException: Unable to compute ceiling for max when > histogram overflowed > {code} > Although the fact that this histogram has 164 buckets already, I wonder if > there is something weird with the computation thats causing this to be so > large? It appears to be coming from updates to system.local > {code} > org.apache.cassandra.metrics:type=Table,keyspace=system,scope=local,name=ColUpdateTimeDeltaHistogram > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-11117) ColUpdateTimeDeltaHistogram histogram overflow
[ https://issues.apache.org/jira/browse/CASSANDRA-7?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15280185#comment-15280185 ] Jeff Griffith commented on CASSANDRA-7: --- the code that updates this is here in ColmnFamilyStore.java: {code} public void apply(DecoratedKey key, ColumnFamily columnFamily, SecondaryIndexManager.Updater indexer, OpOrder.Group opGroup, ReplayPosition replayPosition) { long start = System.nanoTime(); Memtable mt = data.getMemtableFor(opGroup, replayPosition); final long timeDelta = mt.put(key, columnFamily, indexer, opGroup); maybeUpdateRowCache(key); metric.samplers.get(Sampler.WRITES).addSample(key.getKey(), key.hashCode(), 1); metric.writeLatency.addNano(System.nanoTime() - start); if(timeDelta < Long.MAX_VALUE) metric.colUpdateTimeDeltaHistogram.update(timeDelta); } {code} That "if (timeDelta < Long.MAX_VALUE)" looks ill-conceived since there are no longs > max long, but i don't really know what exactly is overflowing in the histogram. > ColUpdateTimeDeltaHistogram histogram overflow > -- > > Key: CASSANDRA-7 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7 > Project: Cassandra > Issue Type: Bug >Reporter: Chris Lohfink >Assignee: Joel Knighton >Priority: Minor > Fix For: 2.2.x, 3.0.x, 3.x > > > {code} > getting attribute Mean of > org.apache.cassandra.metrics:type=ColumnFamily,name=ColUpdateTimeDeltaHistogram > threw an exceptionjavax.management.RuntimeMBeanException: > java.lang.IllegalStateException: Unable to compute ceiling for max when > histogram overflowed > {code} > Although the fact that this histogram has 164 buckets already, I wonder if > there is something weird with the computation thats causing this to be so > large? It appears to be coming from updates to system.local > {code} > org.apache.cassandra.metrics:type=Table,keyspace=system,scope=local,name=ColUpdateTimeDeltaHistogram > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-11117) ColUpdateTimeDeltaHistogram histogram overflow
[ https://issues.apache.org/jira/browse/CASSANDRA-7?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15280152#comment-15280152 ] Jeff Griffith commented on CASSANDRA-7: --- the code that updates this is here: {code} public void apply(DecoratedKey key, ColumnFamily columnFamily, SecondaryIndexManager.Updater indexer, OpOrder.Group opGroup, ReplayPosition replayPosition) { long start = System.nanoTime(); Memtable mt = data.getMemtableFor(opGroup, replayPosition); final long timeDelta = mt.put(key, columnFamily, indexer, opGroup); maybeUpdateRowCache(key); metric.samplers.get(Sampler.WRITES).addSample(key.getKey(), key.hashCode(), 1); metric.writeLatency.addNano(System.nanoTime() - start); if(timeDelta < Long.MAX_VALUE) metric.colUpdateTimeDeltaHistogram.update(timeDelta); } {code} That "if (timeDelta < Long.MAX_VALUE)" looks ill-conceived since there are no longs > max long, but i don't really know what exactly is overflowing in the histogram. > ColUpdateTimeDeltaHistogram histogram overflow > -- > > Key: CASSANDRA-7 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7 > Project: Cassandra > Issue Type: Bug >Reporter: Chris Lohfink >Assignee: Joel Knighton >Priority: Minor > Fix For: 2.2.x, 3.0.x, 3.x > > > {code} > getting attribute Mean of > org.apache.cassandra.metrics:type=ColumnFamily,name=ColUpdateTimeDeltaHistogram > threw an exceptionjavax.management.RuntimeMBeanException: > java.lang.IllegalStateException: Unable to compute ceiling for max when > histogram overflowed > {code} > Although the fact that this histogram has 164 buckets already, I wonder if > there is something weird with the computation thats causing this to be so > large? It appears to be coming from updates to system.local > {code} > org.apache.cassandra.metrics:type=Table,keyspace=system,scope=local,name=ColUpdateTimeDeltaHistogram > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)