[
https://issues.apache.org/jira/browse/CASSANDRA-7278?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14045328#comment-14045328
]
sankalp kohli commented on CASSANDRA-7278:
------------------------------------------
I think my fix will work. Here is why
while ((reader = cachedInstances.poll()) != null)
Here cachedInstances is a ConcurrentLinkedQueue. This will enforce that poll
only gives a reader to one caller irrespective of GC pause. So its either the
thread doing the get() will get it or the removalListener.
So if FileCacheService.get retrieves the reader from the queue, the
removalListener won't get it.
And if removalListener gets it, the get() method won't get it.
> NPE in StorageProxy.java:1920
> -----------------------------
>
> Key: CASSANDRA-7278
> URL: https://issues.apache.org/jira/browse/CASSANDRA-7278
> Project: Cassandra
> Issue Type: Bug
> Components: Core
> Environment: Cassandra 2.07, x86-64 ubuntu 12.04
> Reporter: Duncan Sands
> Assignee: sankalp kohli
> Priority: Minor
> Attachments: sl
>
>
> Got this this morning under heavy load:
> ERROR [ReadStage:128] 2014-05-21 07:59:03,274 CassandraDaemon.java (line 198)
> Exception in thread Thread[ReadStage:128,5,main]
> java.lang.RuntimeException: java.lang.NullPointerException
> at
> org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1920)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> at java.lang.Thread.run(Thread.java:744)
> Caused by: java.lang.NullPointerException
> at
> org.apache.cassandra.io.util.RandomAccessReader.getTotalBufferSize(RandomAccessReader.java:157)
> at
> org.apache.cassandra.io.compress.CompressedRandomAccessReader.getTotalBufferSize(CompressedRandomAccessReader.java:159)
> at
> org.apache.cassandra.service.FileCacheService.get(FileCacheService.java:96)
> at
> org.apache.cassandra.io.util.PoolingSegmentedFile.getSegment(PoolingSegmentedFile.java:36)
> at
> org.apache.cassandra.io.sstable.SSTableReader.getFileDataInput(SSTableReader.java:1195)
> at
> org.apache.cassandra.db.columniterator.SimpleSliceReader.<init>(SimpleSliceReader.java:57)
> at
> org.apache.cassandra.db.columniterator.SSTableSliceIterator.createReader(SSTableSliceIterator.java:65)
> at
> org.apache.cassandra.db.columniterator.SSTableSliceIterator.<init>(SSTableSliceIterator.java:42)
> at
> org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:167)
> at
> org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:62)
> at
> org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:250)
> at
> org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:53)
> at
> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1540)
> at
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1369)
> at org.apache.cassandra.db.Keyspace.getRow(Keyspace.java:327)
> at
> org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:65)
> at
> org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1352)
> at
> org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1916)
> ... 3 more
> There had just been a 20 second GC pause, and the system was dropping
> messages like mad, see attached log snippet.
--
This message was sent by Atlassian JIRA
(v6.2#6252)