cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Allan C <alla...@gmail.com>
Subject HintedHandoff Exception and node holding hints to random tokens
Date Mon, 20 Jan 2014 20:18:36 GMT
Hi ,

I’m hitting a very odd issue with HintedHandoff on 1 node in my 12 node cluster running
1.2.13. Somehow it’s holding a large amount of hints for tokens that have never been part
of the cluster. Pretty sure this is causing a bunch of memory pressure somehow that’s causing
the node to go down.

I’d like to find out if I can just reset by deleting the hints CF or if there’s actually
important data in there. I’m tempted to clear the CF and hope that fixes it, but a few nodes
have been up and down (especially this one) since my last repair and I worry that I won’t
be able to get through a full repair given the problems with the node currently.

Here’s what I see so far:


* listEndpointsPendingHints returns a list of about 20 tokens that are not part of the ring
and have never been part of it. I’m not using vnodes, fwiw. deleteHintsForEndpoint doesn’t
work. It tells me that the there’s no host for the token.


* The hints CF is oddly large:

     Column Family: hints
SSTable count: 260
Space used (live): 124904685
Space used (total): 124904685
SSTable Compression Ratio: 0.394676439667606
Number of Keys (estimate): 66560
Memtable Columns Count: 0
Memtable Data Size: 0
Memtable Switch Count: 14
Read Count: 113
Read Latency: 757.123 ms.
Write Count: 987
Write Latency: 0.044 ms.
Pending Tasks: 0
Bloom Filter False Positives: 10
Bloom Filter False Ratio: 0.00209
Bloom Filter Space Used: 6528
Compacted row minimum size: 36
Compacted row maximum size: 107964792
Compacted row mean size: 787505
Average live cells per slice (last five minutes): 0.0


* I get this assertion in the logs often:

ERROR [CompactionExecutor:81] 2014-01-20 12:31:22,652 CassandraDaemon.java (line 191) Exception
in thread Thread[CompactionExecutor:81,1,main]
java.lang.AssertionError: originally calculated column size of 71868452 but now it is 71869026
        at org.apache.cassandra.db.compaction.LazilyCompactedRow.write(LazilyCompactedRow.java:135)
        at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:160)
        at org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:162)
        at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
        at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
        at org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:58)
        at org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:60)
        at org.apache.cassandra.db.compaction.CompactionManager$7.runMayThrow(CompactionManager.java:442)
        at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:439)
        at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
        at java.util.concurrent.FutureTask.run(FutureTask.java:138)
        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
        at java.lang.Thread.run(Thread.java:662)
ERROR [HintedHandoff:52] 2014-01-20 12:31:22,652 CassandraDaemon.java (line 191) Exception
in thread Thread[HintedHandoff:52,1,main]
java.lang.RuntimeException: java.util.concurrent.ExecutionException: java.lang.AssertionError:
originally calculated column size of 71868452 but now it is 71869026
        at org.apache.cassandra.db.HintedHandOffManager.doDeliverHintsToEndpoint(HintedHandOffManager.java:436)
        at org.apache.cassandra.db.HintedHandOffManager.deliverHintsToEndpoint(HintedHandOffManager.java:282)
        at org.apache.cassandra.db.HintedHandOffManager.access$300(HintedHandOffManager.java:90)
        at org.apache.cassandra.db.HintedHandOffManager$4.run(HintedHandOffManager.java:502)
        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
        at java.lang.Thread.run(Thread.java:662)
Caused by: java.util.concurrent.ExecutionException: java.lang.AssertionError: originally calculated
column size of 71868452 but now it is 71869026
        at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:222)
        at java.util.concurrent.FutureTask.get(FutureTask.java:83)
        at org.apache.cassandra.db.HintedHandOffManager.doDeliverHintsToEndpoint(HintedHandOffManager.java:432)
        ... 6 more
Caused by: java.lang.AssertionError: originally calculated column size of 71868452 but now
it is 71869026
        at org.apache.cassandra.db.compaction.LazilyCompactedRow.write(LazilyCompactedRow.java:135)
        at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:160)
        at org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:162)
        at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
        at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
        at org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:58)
        at org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:60)
        at org.apache.cassandra.db.compaction.CompactionManager$7.runMayThrow(CompactionManager.java:442)
        at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:439)
        at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
        at java.util.concurrent.FutureTask.run(FutureTask.java:138)
        ... 3 more


* I see a similar error when I try to compact the hints CF, even when I set in_memory_compaction_limit_in_mb
as high as 1024.

This started after I had brought up a few new nodes last week and then decommissioned them
a few days later. The adding and decommissioning appeared to go uneventfully.


If anyone has seen anything like this or can give me some hints on how to determine if the
hints can be deleted, I’d greatly appreciate it.

-Allan
Mime
View raw message