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

HB commented on CASSANDRA-2195:
-------------------------------

Ok here's an update: I tried to sstable2json a few of my .db files, and this is 
what happens: sstable2json creates file handles until it eventually exits with 
Exception in thread "main" java.io.IOError: java.io.FileNotFoundException: 
/var/lib/cassandra/data/<ks>/Search-1265-Index.db (Too many open files)
        at 
org.apache.cassandra.io.util.BufferedSegmentedFile.getSegment(BufferedSegmentedFile.java:68)
        at 
org.apache.cassandra.io.util.SegmentedFile$SegmentIterator.next(SegmentedFile.java:130)
        at 
org.apache.cassandra.io.util.SegmentedFile$SegmentIterator.next(SegmentedFile.java:109)
        at 
org.apache.cassandra.io.sstable.SSTableReader.getPosition(SSTableReader.java:472)
        at 
org.apache.cassandra.io.sstable.SSTableReader.getFileDataInput(SSTableReader.java:563)
        at 
org.apache.cassandra.db.columniterator.SSTableSliceIterator.<init>(SSTableSliceIterator.java:49)
        at 
org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:68)
        at 
org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:80)
        at 
org.apache.cassandra.tools.SSTableExport.serializeRow(SSTableExport.java:175)
        at 
org.apache.cassandra.tools.SSTableExport.export(SSTableExport.java:353)
        at 
org.apache.cassandra.tools.SSTableExport.export(SSTableExport.java:375)
        at 
org.apache.cassandra.tools.SSTableExport.export(SSTableExport.java:388)
        at org.apache.cassandra.tools.SSTableExport.main(SSTableExport.java:446)
Caused by: java.io.FileNotFoundException: 
/var/lib/cassandra/data/<ks>/Search-1265-Index.db (Too many open files)
        at java.io.RandomAccessFile.open(Native Method)
        at java.io.RandomAccessFile.<init>(RandomAccessFile.java:212)
        at 
org.apache.cassandra.io.util.BufferedRandomAccessFile.<init>(BufferedRandomAccessFile.java:116)
        at 
org.apache.cassandra.io.util.BufferedRandomAccessFile.<init>(BufferedRandomAccessFile.java:111)
        at 
org.apache.cassandra.io.util.BufferedRandomAccessFile.<init>(BufferedRandomAccessFile.java:96)
        at 
org.apache.cassandra.io.util.BufferedSegmentedFile.getSegment(BufferedSegmentedFile.java:62)
        ... 12 more
Checked with lsof -n, doing a line count shows it gets near 65535 handles 
before it gives up, which is the limit shown by ulimit -H -a for open files). 
Sometimes it manages to write some actual json (in some cases 40-50MB), which 
at first glance looks ok (though I did notice the keys are encoded in hex, 
whereas they used to be plain text -- dehexing them does show expected values, 
though). I should also note that I'm successfully running sstable2json on an 
older version of the dataset, taken when it was still running on 0.6.

I think I failed to mention this dataset used to be owned by a 0.6.x instance, 
I moved over the files to a differen server, converted the original config file 
and made 0.7 load it. I also updated the CF by adding an UTF8Type extra column 
with index_type: KEYS. Also, in addition to Search-xxx-* I now have a number of 
Search-f-xxxx-* and Search.64617465-f-xx.* which I didn't use to have, is this 
ok?

Something definitely seems to be up with my sstables. Since this is a test 
node, I can afford to lose this dataset, but of course I'd like to find out 
what went wrong so it doesn't happen again (to me or others), so I hope there's 
anything you can extract from this information that is helpful.


> java.lang.RuntimeException: java.lang.NegativeArraySizeException
> ----------------------------------------------------------------
>
>                 Key: CASSANDRA-2195
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2195
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>    Affects Versions: 0.7.2
>         Environment: Debian Lenny, Pelops-based servlet doing lots of 
> List<Column> columns = selector.getColumnsFromRow(columnFamily, key, false, 
> ConsistencyLevel.ONE); and mutator.writeColumns(columnFamily, key, 
> mutator.newColumnList(...); mutator.execute(ConsistencyLevel.ANY); operations.
>            Reporter: HB
>            Assignee: Stu Hood
>            Priority: Blocker
>             Fix For: 0.7.3
>
>
> When putting my 0.7.2 node under load, I get a large amount of these: 
> ERROR 15:33:25,075 Fatal exception in thread Thread[MutationStage:290,5,main]
> java.lang.RuntimeException: java.lang.NegativeArraySizeException
>         at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34)
>         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.lang.NegativeArraySizeException
>         at 
> org.apache.cassandra.utils.BloomFilterSerializer.deserialize(BloomFilterSerializer.java:49)
>         at 
> org.apache.cassandra.utils.BloomFilterSerializer.deserialize(BloomFilterSerializer.java:30)
>         at 
> org.apache.cassandra.io.sstable.IndexHelper.defreezeBloomFilter(IndexHelper.java:108)
>         at 
> org.apache.cassandra.db.columniterator.SSTableNamesIterator.read(SSTableNamesIterator.java:106)
>         at 
> org.apache.cassandra.db.columniterator.SSTableNamesIterator.<init>(SSTableNamesIterator.java:71)
>         at 
> org.apache.cassandra.db.filter.NamesQueryFilter.getSSTableColumnIterator(NamesQueryFilter.java:59)
>         at 
> org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:80)
>         at 
> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1275)
>         at 
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1167)
>         at 
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1095)
>         at 
> org.apache.cassandra.db.Table.readCurrentIndexedColumns(Table.java:510)
>         at org.apache.cassandra.db.Table.apply(Table.java:445)
>         at org.apache.cassandra.db.RowMutation.apply(RowMutation.java:190)
>         at 
> org.apache.cassandra.service.StorageProxy$2.runMayThrow(StorageProxy.java:283)
>         at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
>         ... 3 more
> On recommendation of driftx I forced a compaction, which finished. After a 
> restart, the -Compacted files where removed and the node seemed to start up, 
> querying some random rows seemed to go alright but after a few minutes I 
> started getting the above messages again. I'm grabbing single rows, not 
> slices.

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

        

Reply via email to