Hmm, 500 sstables is definitely a degenerate case: did you disable compaction? 
By default, Cassandra strives to keep the sstable count below ~32, since 
accesses to separate sstables require seeks.

In this case, the query will seek 500 times to check the secondary index for 
each sstable: if it finds matches it will need to seek to find them in the 
primary index, and seek again for the data file.

-----Original Message-----
From: "dragos cernahoschi" <dragos.cernahos...@gmail.com>
Sent: Tuesday, November 9, 2010 5:33am
To: dev@cassandra.apache.org
Subject: Re: CASSANDRA-1472 (bitmap indexes)

There are about 500 SSTables (12GB of data including index data,
statistics...) The source data file had about 3GB/26 million rows.

I only test with EQ expressions for now.

Increasing the file limit resolved the problem, but now I'm getting
TimedOutException(s) from thrift when "querying" even with slice size of 1.
Is my machine too small (core 2 duo 2.93 2GB RAM Ubuntu 10.04) for such a
test?

I really have some interesting sets of data to test indexes with and I want
to make a comparison between ordinary indexes and bitmap indexes.

Thank you,
Dragos

On Mon, Nov 8, 2010 at 6:42 PM, Stu Hood <stu.h...@rackspace.com> wrote:

> Dragos,
>
> How many SSTables did you have on disk, and were any of your index
> expressions GT(E)/LT(E)?
>
> I expect that you are bumping into a limitation of the current
> implementation: it opens up to 128 file-handles per SSTable in the worst
> case for a GT/LT query (one per index bucket).
>
> A future version might remove that requirement, but for now, you should
> probably bump the file handle limit on your machine to at least 2^16.
>
> Thanks,
> Stu
>
>
> -----Original Message-----
> From: "dragos cernahoschi" <dragos.cernahos...@gmail.com>
> Sent: Monday, November 8, 2010 10:05am
> To: dev@cassandra.apache.org
> Subject: CASSANDRA-1472 (bitmap indexes)
>
> Hi,
>
> I've got an exception during the following test:
>
> test machine: core 2 duo 2.93 2GB RAM Ubuntu 10.04
>
> test scenario:
> - 1 column family
> - about 15 columns
> - 7 indexed columns (bitmap)
> - 26 million rows (insert operation went fine)
> - thrift "query" on 3 of the indexed columns with get_indexed_slices
> (count:
> 100)
> - got the following exception:
>
> 10/11/08 17:52:40 ERROR service.AbstractCassandraDaemon: Fatal exception in
> thread Thread[ReadStage:3,5,main]
> java.io.IOError: java.io.FileNotFoundException:
> /home/dragos/cassandra/data/keyspace/visit-e-814-4-Bitidx.db (Too many open
> files)
>    at
>
> org.apache.cassandra.io.sstable.bitidx.SegmentIterator.open(SegmentIterator.java:78)
>    at
>
> org.apache.cassandra.io.sstable.bitidx.BitmapIndexReader.openBin(BitmapIndexReader.java:226)
>    at
>
> org.apache.cassandra.io.sstable.bitidx.BitmapIndexReader.iterator(BitmapIndexReader.java:214)
>    at
> org.apache.cassandra.io.sstable.SSTableReader.scan(SSTableReader.java:523)
>    at
>
> org.apache.cassandra.db.secindex.KeysBitmapIndex.iterator(KeysBitmapIndex.java:103)
>    at
> org.apache.cassandra.db.ColumnFamilyStore.scan(ColumnFamilyStore.java:1371)
>    at
>
> org.apache.cassandra.service.IndexScanVerbHandler.doVerb(IndexScanVerbHandler.java:41)
>    at
>
> org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:51)
>    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.io.FileNotFoundException:
> /home/dragos/cassandra/data/keyspace/visit-e-814-4-Bitidx.db (Too many open
> files)
>    at java.io.FileInputStream.open(Native Method)
>    at java.io.FileInputStream.<init>(FileInputStream.java:106)
>    at
> org.apache.avro.file.SeekableFileInput.<init>(SeekableFileInput.java:29)
>    at org.apache.avro.file.DataFileReader.<init>(DataFileReader.java:38)
>    at
>
> org.apache.cassandra.io.sstable.bitidx.SegmentIterator.open(SegmentIterator.java:72)
>    ... 10 more
> 10/11/08 17:52:40 ERROR service.AbstractCassandraDaemon: Fatal exception in
> thread Thread[ReadStage:2,5,main]
> java.io.IOError: java.io.FileNotFoundException:
> /home/dragos/cassandra/data/keyspace/visit-e-1018-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:129)
>    at
>
> org.apache.cassandra.io.util.SegmentedFile$SegmentIterator.next(SegmentedFile.java:1)
>    at
>
> org.apache.cassandra.io.sstable.SSTableReader.getPosition(SSTableReader.java:455)
>    at
>
> org.apache.cassandra.io.sstable.SSTableReader.getFileDataInput(SSTableReader.java:572)
>    at
>
> org.apache.cassandra.db.columniterator.SSTableSliceIterator.<init>(SSTableSliceIterator.java:49)
>    at
>
> org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:72)
>    at
>
> org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:84)
>    at
>
> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1190)
>    at
>
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1082)
>    at
>
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1052)
>    at
> org.apache.cassandra.db.ColumnFamilyStore.scan(ColumnFamilyStore.java:1378)
>    at
>
> org.apache.cassandra.service.IndexScanVerbHandler.doVerb(IndexScanVerbHandler.java:41)
>    at
>
> org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:51)
>    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.io.FileNotFoundException:
> /home/dragos/cassandra/data/keyspace/visit-e-1018-Index.db (Too many open
> files)
>    at java.io.RandomAccessFile.open(Native Method)
>    at java.io.RandomAccessFile.<init>(RandomAccessFile.java:212)
>    at java.io.RandomAccessFile.<init>(RandomAccessFile.java:98)
>    at
>
> org.apache.cassandra.io.util.BufferedRandomAccessFile.<init>(BufferedRandomAccessFile.java:142)
>    at
>
> org.apache.cassandra.io.util.BufferedSegmentedFile.getSegment(BufferedSegmentedFile.java:62)
>    ... 16 more
>
> The same test worked fine with 1 million rows.
>
>
>


Reply via email to