Hi,

We periodically get the following exception in our Cassandra cluster. What I 
mean by periodically is once or twice a week. It always affects the same column 
family ( containing super columns).  The frequency of the problems seems to 
center on the same couple nodes, but not always. I've been getting past the 
problem by performing a scrub on the column family, which corrects the problem 
by eliminating one unreadable row.

 WARN [CompactionExecutor:1] 2011-08-27 04:08:42,916 CompactionManager.java 
(line 654) Unable to recover 1 rows that were skipped.  You can attempt manual 
recovery from the pre-scrub snapshot.  You can also run nodetool repair to 
transfer the data from a healthy replica, if any

Next time I have the opportunity, I'll throw the node into debug mode before 
scrubbing, perhaps more information will be available.

I have a copy of the SSTables if anybody has an interest in dissecting it. I'm 
not a Cassandra developer, but currently I've been able to open the SSTable 
using the SSTableReader class, but not sure where to go from there. Maybe I 
should be using the CompactionManager.scrubOne method or maybe use the 
SSTableScanner. Any tips from developers on the cassandra list in this regard 
would also be much appreciated. Specifically, I'd like to dissect the sstable 
and to figure out what the key is to the bad row and what is wrong with the 
columns/supercolumns in that row.

The only issue I've found WRT to mmap segment underflow is CASSANDRA-2315, 
which doesn't seem to apply in this case.

Thanks for your help.

Chad

ERROR [ReadStage:2772] 2011-08-27 03:42:08,606 AbstractCassandraDaemon.java 
(line 113) Fatal exception in thread Thread[ReadStage:2772,5,main]
java.io.IOError: java.io.IOException: mmap segment underflow; remaining is 
2612010 but 692929875 requested
        at 
org.apache.cassandra.io.util.ColumnIterator.deserializeNext(ColumnSortedMap.java:252)
        at 
org.apache.cassandra.io.util.ColumnIterator.next(ColumnSortedMap.java:268)
        at 
org.apache.cassandra.io.util.ColumnIterator.next(ColumnSortedMap.java:227)
        at 
java.util.concurrent.ConcurrentSkipListMap.buildFromSorted(ConcurrentSkipListMap.java:1521)
        at 
java.util.concurrent.ConcurrentSkipListMap.<init>(ConcurrentSkipListMap.java:1471)
        at 
org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:379)
        at 
org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:362)
        at 
org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:322)
        at 
org.apache.cassandra.db.columniterator.SimpleSliceReader.computeNext(SimpleSliceReader.java:79)
        at 
org.apache.cassandra.db.columniterator.SimpleSliceReader.computeNext(SimpleSliceReader.java:40)
        at 
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:136)
        at 
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:131)
        at 
org.apache.cassandra.db.columniterator.SSTableSliceIterator.hasNext(SSTableSliceIterator.java:108)
        at 
org.apache.commons.collections.iterators.CollatingIterator.set(CollatingIterator.java:283)
        at 
org.apache.commons.collections.iterators.CollatingIterator.least(CollatingIterator.java:326)
        at 
org.apache.commons.collections.iterators.CollatingIterator.next(CollatingIterator.java:230)
        at 
org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterator.java:69)
        at 
com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:136)
        at 
com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:131)
        at 
org.apache.cassandra.db.filter.SliceQueryFilter.collectReducedColumns(SliceQueryFilter.java:116)
        at 
org.apache.cassandra.db.filter.QueryFilter.collectCollatedColumns(QueryFilter.java:130)
        at 
org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1390)
        at 
org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1267)
        at 
org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1195)
        at org.apache.cassandra.db.Table.getRow(Table.java:324)
        at 
org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:63)
        at 
org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:451)
        at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
        at java.lang.Thread.run(Thread.java:636)
Caused by: java.io.IOException: mmap segment underflow; remaining is 2612010 
but 692929875 requested
        at 
org.apache.cassandra.io.util.MappedFileDataInput.readBytes(MappedFileDataInput.java:118)
        at 
org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:356)
        at 
org.apache.cassandra.utils.ByteBufferUtil.readWithLength(ByteBufferUtil.java:318)
        at 
org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:81)
        at 
org.apache.cassandra.io.util.ColumnIterator.deserializeNext(ColumnSortedMap.java:248)
        ... 30 more

Reply via email to