cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Matthew F. Dennis (JIRA)" <j...@apache.org>
Subject [jira] Created: (CASSANDRA-2315) mmap segment underflow when building secondary index after joining ring
Date Fri, 11 Mar 2011 22:03:59 GMT
mmap segment underflow when building secondary index after joining ring
-----------------------------------------------------------------------

                 Key: CASSANDRA-2315
                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2315
             Project: Cassandra
          Issue Type: Bug
    Affects Versions: 0.7.3
            Reporter: Matthew F. Dennis


there was a 7 node ring (N0-N6) that was upgraded to 0.7.3
all the data was scrubbed
7 additional nodes were added (N7-N13) with no problems
some compactions ran, some things were restarted (but there was no read/write load)
after a while 7 more nodes (N14-N20) were added
all joined successfully except one
that node was pulling data from one of the nodes in the N7-N13 set (i.e. a node that got all
it's data from the original nodes) 
the EC2 instance that was running was completely destroyed, a new instance created and the
bootstrap attempted again but still failed

{code}
INFO [Thread-65] 2011-03-10 17:16:59,448 ColumnFamilyStore.java (line 377) Index build of
service_id, complete
INFO [Thread-65] 2011-03-10 17:16:59,449 ColumnFamilyStore.java (line 359) Submitting index
build of service_id, for data in SSTableReader(path='/mnt/cassandra/data/BackupifyMetadata/GoogleDocsMetadata-f-1-Data.db')
INFO [Thread-75] 2011-03-10 17:17:45,757 ColumnFamilyStore.java (line 377) Index build of
service_id, complete
INFO [Thread-75] 2011-03-10 17:17:45,767 ColumnFamilyStore.java (line 359) Submitting index
build of service_id, for data in SSTableReader(path='/mnt/cassandra/data/BackupifyMetadata/GoogleDocsMetadata-f-2-Data.db'),
SSTableReader(path='/mnt/cassandra/data/BackupifyMetadata/GoogleDocsMetadata-f-3-Data.db'),
SSTableReader(path='/mnt/cassandra/data/BackupifyMetadata/GoogleDocsMetadata-f-4-Data.db'),
SSTableReader(path='/mnt/cassandra/data/BackupifyMetadata/GoogleDocsMetadata-f-5-Data.db')
INFO [ScheduledTasks:1] 2011-03-10 17:17:46,550 ColumnFamilyStore.java (line 1020) Enqueuing
flush of Memtable-GoogleDocsMetadata.736572766963655f6964@1078361799(353352 bytes, 4539 operations)
INFO [FlushWriter:1] 2011-03-10 17:17:46,552 Memtable.java (line 157) Writing Memtable-GoogleDocsMetadata.736572766963655f6964@1078361799(353352
bytes, 4539 operations)
INFO [FlushWriter:1] 2011-03-10 17:17:46,736 Memtable.java (line 164) Completed flushing /mnt/cassandra/data/BackupifyMetadata/GoogleDocsMetadata.736572766963655f6964-f-1-Data.db
(483139 bytes)
INFO [Thread-65] 2011-03-10 17:18:04,278 ColumnFamilyStore.java (line 377) Index build of
service_id, complete
INFO [Thread-65] 2011-03-10 17:18:04,279 ColumnFamilyStore.java (line 359) Submitting index
build of service_id, for data in SSTableReader(path='/mnt/cassandra/data/BackupifyMetadata/BloggerMetadata-f-1-Data.db')
ERROR [Thread-75] 2011-03-10 17:18:21,781 AbstractCassandraDaemon.java (line 114) Fatal exception
in thread Thread[Thread-75,5,main]
java.lang.RuntimeException: java.util.concurrent.ExecutionException: java.lang.AssertionError:
mmap segment underflow; remaining is 43239499 but 1768777055 requested
        at org.apache.cassandra.db.ColumnFamilyStore.buildSecondaryIndexes(ColumnFamilyStore.java:375)
        at org.apache.cassandra.streaming.StreamInSession.closeIfFinished(StreamInSession.java:159)
        at org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:63)
        at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:91)
Caused by: java.util.concurrent.ExecutionException: java.lang.AssertionError: mmap segment
underflow; remaining is 43239499 but 1768777055 requested
        at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:222)
        at java.util.concurrent.FutureTask.get(FutureTask.java:83)
        at org.apache.cassandra.db.ColumnFamilyStore.buildSecondaryIndexes(ColumnFamilyStore.java:365)
        ... 3 more
Caused by: java.lang.AssertionError: mmap segment underflow; remaining is 43239499 but 1768777055
requested
        at org.apache.cassandra.io.util.MappedFileDataInput.readBytes(MappedFileDataInput.java:119)
        at org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:310)
        at org.apache.cassandra.utils.ByteBufferUtil.readWithLength(ByteBufferUtil.java:267)
        at org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:94)
        at org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:35)
        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.anyHasNext(CollatingIterator.java:364)
        at org.apache.commons.collections.iterators.CollatingIterator.hasNext(CollatingIterator.java:217)
        at org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterator.java:55)
        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:118)
        at org.apache.cassandra.db.filter.QueryFilter.collectCollatedColumns(QueryFilter.java:142)
        at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1340)
        at org.apache.cassandra.db.ColumnFamilyStore.cacheRow(ColumnFamilyStore.java:1158)
        at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1224)
        at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1145)
        at org.apache.cassandra.db.Table.readCurrentIndexedColumns(Table.java:459)
        at org.apache.cassandra.db.Table.access$200(Table.java:56)
        at org.apache.cassandra.db.Table$IndexBuilder.build(Table.java:573)
        at org.apache.cassandra.db.CompactionManager$8.run(CompactionManager.java:892)
        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
        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 [CompactionExecutor:1] 2011-03-10 17:18:21,785 AbstractCassandraDaemon.java (line 114)
Fatal exception in thread Thread[CompactionExecutor:1,1,main]
java.lang.AssertionError: mmap segment underflow; remaining is 43239499 but 1768777055 requested
        at org.apache.cassandra.io.util.MappedFileDataInput.readBytes(MappedFileDataInput.java:119)
        at org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:310)
        at org.apache.cassandra.utils.ByteBufferUtil.readWithLength(ByteBufferUtil.java:267)
        at org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:94)
        at org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:35)
        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.anyHasNext(CollatingIterator.java:364)
        at org.apache.commons.collections.iterators.CollatingIterator.hasNext(CollatingIterator.java:217)
        at org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterator.java:55)
        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:118)
        at org.apache.cassandra.db.filter.QueryFilter.collectCollatedColumns(QueryFilter.java:142)
        at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1340)
        at org.apache.cassandra.db.ColumnFamilyStore.cacheRow(ColumnFamilyStore.java:1158)
        at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1224)
        at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1145)
        at org.apache.cassandra.db.Table.readCurrentIndexedColumns(Table.java:459)
        at org.apache.cassandra.db.Table.access$200(Table.java:56)
        at org.apache.cassandra.db.Table$IndexBuilder.build(Table.java:573)
        at org.apache.cassandra.db.CompactionManager$8.run(CompactionManager.java:892)
        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
        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)
{code}


--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

Mime
View raw message