[jira] [Comment Edited] (CASSANDRA-11117) ColUpdateTimeDeltaHistogram histogram overflow
[ https://issues.apache.org/jira/browse/CASSANDRA-7?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15569506#comment-15569506 ] Joel Knighton edited comment on CASSANDRA-7 at 10/12/16 6:32 PM: - I've updated the branch above to adopt the approach you described. I also slightly shortened the unit test since the minimum constraint unifies the two cases being tested. New CI runs have completed on the links above and look clean relative to upstream. EDIT: I should note that the 2.2 branch and 3.0 branch each need to be applied as their own patch. The 3.0 branch should merge forward cleanly. was (Author: jkni): I've updated the branch above to adopt the approach you described. I also slightly shortened the unit test since the minimum constraint unifies the two cases being tested. New CI runs have completed on the links above and look clean relative to upstream. > 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, 4.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] [Comment Edited] (CASSANDRA-11117) ColUpdateTimeDeltaHistogram histogram overflow
[ https://issues.apache.org/jira/browse/CASSANDRA-7?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15430271#comment-15430271 ] Clément Lardeur edited comment on CASSANDRA-7 at 8/22/16 8:06 AM: -- 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 Open Source community than previously. If I could help I can give you more details with our cluster settings. was (Author: clardeur): 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 pre
[jira] [Comment Edited] (CASSANDRA-11117) ColUpdateTimeDeltaHistogram histogram overflow
[ https://issues.apache.org/jira/browse/CASSANDRA-7?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15280152#comment-15280152 ] Jeff Griffith edited comment on CASSANDRA-7 at 5/11/16 1:59 PM: 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. was (Author: jeffery.griffith): 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)