You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Jeff Griffith (JIRA)" <ji...@apache.org> on 2016/05/11 13:59:12 UTC

[jira] [Commented] (CASSANDRA-11117) ColUpdateTimeDeltaHistogram histogram overflow

    [ https://issues.apache.org/jira/browse/CASSANDRA-11117?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15280152#comment-15280152 ] 

Jeff Griffith commented on CASSANDRA-11117:
-------------------------------------------

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-11117
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-11117
>             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)