Re: Can Cassandra make real use of several DataFileDirectories?
Please refer the code: org.apache.cassandra.db.ColumnFamilyStore public String getFlushPath() { long guessedSize = 2 * DatabaseDescriptor.getMemtableThroughput() * 1024*1024; // 2* adds room for keys, column indexes String location = DatabaseDescriptor.getDataFileLocationForTable(table_, guessedSize); if (location == null) throw new RuntimeException(Insufficient disk space to flush); return new File(location, getTempSSTableFileName()).getAbsolutePath(); } and we can go through org.apache.cassandra.config.DatabaseDescriptor: public static String getDataFileLocationForTable(String table, long expectedCompactedFileSize) { long maxFreeDisk = 0; int maxDiskIndex = 0; String dataFileDirectory = null; String[] dataDirectoryForTable = getAllDataFileLocationsForTable(table); for ( int i = 0 ; i dataDirectoryForTable.length ; i++ ) { File f = new File(dataDirectoryForTable[i]); if( maxFreeDisk f.getUsableSpace()) { maxFreeDisk = f.getUsableSpace(); maxDiskIndex = i; } } // Load factor of 0.9 we do not want to use the entire disk that is too risky. maxFreeDisk = (long)(0.9 * maxFreeDisk); if( expectedCompactedFileSize maxFreeDisk ) { dataFileDirectory = dataDirectoryForTable[maxDiskIndex]; currentIndex = (maxDiskIndex + 1 )%dataDirectoryForTable.length ; } else { currentIndex = maxDiskIndex; } return dataFileDirectory; } So, DataFileDirectories means multiple disks or disk-partitions. I think your storage01, storage02 and storage03 are in same disk or disk partition. 2010/4/26 Roland Hänel rol...@haenel.me I have a configuration like this: DataFileDirectories DataFileDirectory/storage01/cassandra/data/DataFileDirectory DataFileDirectory/storage02/cassandra/data/DataFileDirectory DataFileDirectory/storage03/cassandra/data/DataFileDirectory /DataFileDirectories After loading a big chunk of data into cassandra, I end up wich some 70GB in the first directory, and only about 10GB in the second and third one. All rows are quite small, so it's not just some big rows that contain the majority of data. Does Cassandra have the ability to 'see' the maximum available space in these directory? I'm asking myself this question since my limit is 100GB, and the first directory is approaching this limit... And, wouldn't it be better if Cassandra tried to 'load-balance' the files inside the directories because this will result in better (read) performance if the directories are on different disks (which is the case for me)? Any help is appreciated. Roland
Re: Can Cassandra make real use of several DataFileDirectories?
Thanks very much. Precisely answers my questions. :-) 2010/4/26 Schubert Zhang zson...@gmail.com Please refer the code: org.apache.cassandra.db.ColumnFamilyStore public String getFlushPath() { long guessedSize = 2 * DatabaseDescriptor.getMemtableThroughput() * 1024*1024; // 2* adds room for keys, column indexes String location = DatabaseDescriptor.getDataFileLocationForTable(table_, guessedSize); if (location == null) throw new RuntimeException(Insufficient disk space to flush); return new File(location, getTempSSTableFileName()).getAbsolutePath(); } and we can go through org.apache.cassandra.config.DatabaseDescriptor: public static String getDataFileLocationForTable(String table, long expectedCompactedFileSize) { long maxFreeDisk = 0; int maxDiskIndex = 0; String dataFileDirectory = null; String[] dataDirectoryForTable = getAllDataFileLocationsForTable(table); for ( int i = 0 ; i dataDirectoryForTable.length ; i++ ) { File f = new File(dataDirectoryForTable[i]); if( maxFreeDisk f.getUsableSpace()) { maxFreeDisk = f.getUsableSpace(); maxDiskIndex = i; } } // Load factor of 0.9 we do not want to use the entire disk that is too risky. maxFreeDisk = (long)(0.9 * maxFreeDisk); if( expectedCompactedFileSize maxFreeDisk ) { dataFileDirectory = dataDirectoryForTable[maxDiskIndex]; currentIndex = (maxDiskIndex + 1 )%dataDirectoryForTable.length ; } else { currentIndex = maxDiskIndex; } return dataFileDirectory; } So, DataFileDirectories means multiple disks or disk-partitions. I think your storage01, storage02 and storage03 are in same disk or disk partition. 2010/4/26 Roland Hänel rol...@haenel.me I have a configuration like this: DataFileDirectories DataFileDirectory/storage01/cassandra/data/DataFileDirectory DataFileDirectory/storage02/cassandra/data/DataFileDirectory DataFileDirectory/storage03/cassandra/data/DataFileDirectory /DataFileDirectories After loading a big chunk of data into cassandra, I end up wich some 70GB in the first directory, and only about 10GB in the second and third one. All rows are quite small, so it's not just some big rows that contain the majority of data. Does Cassandra have the ability to 'see' the maximum available space in these directory? I'm asking myself this question since my limit is 100GB, and the first directory is approaching this limit... And, wouldn't it be better if Cassandra tried to 'load-balance' the files inside the directories because this will result in better (read) performance if the directories are on different disks (which is the case for me)? Any help is appreciated. Roland
Re: Can Cassandra make real use of several DataFileDirectories?
I would recommend using RAID-0 rather that multiple data directories. -ryan 2010/4/26 Roland Hänel rol...@haenel.me: I have a configuration like this: DataFileDirectories DataFileDirectory/storage01/cassandra/data/DataFileDirectory DataFileDirectory/storage02/cassandra/data/DataFileDirectory DataFileDirectory/storage03/cassandra/data/DataFileDirectory /DataFileDirectories After loading a big chunk of data into cassandra, I end up wich some 70GB in the first directory, and only about 10GB in the second and third one. All rows are quite small, so it's not just some big rows that contain the majority of data. Does Cassandra have the ability to 'see' the maximum available space in these directory? I'm asking myself this question since my limit is 100GB, and the first directory is approaching this limit... And, wouldn't it be better if Cassandra tried to 'load-balance' the files inside the directories because this will result in better (read) performance if the directories are on different disks (which is the case for me)? Any help is appreciated. Roland
Re: Can Cassandra make real use of several DataFileDirectories?
Hm... I understand that RAID0 would help to create a bigger pool for compactions. However, it might impact read performance: if I have several CF's (with their SSTables), random read requests for the CF files that are on separate disks will behave nicely - however if it's RAID0 then a random read on any file will create a random read on all of the hard disks. Correct? -Roland 2010/4/26 Jonathan Ellis jbel...@gmail.com http://wiki.apache.org/cassandra/CassandraHardware On Mon, Apr 26, 2010 at 1:06 PM, Edmond Lau edm...@ooyala.com wrote: Ryan - You (or maybe someone else) mentioned using RAID-0 instead of multiple data directories at the Cassandra hackathon as well. Could you explain the motivation behind that? Thanks, Edmond On Mon, Apr 26, 2010 at 9:53 AM, Ryan King r...@twitter.com wrote: I would recommend using RAID-0 rather that multiple data directories. -ryan 2010/4/26 Roland Hänel rol...@haenel.me: I have a configuration like this: DataFileDirectories DataFileDirectory/storage01/cassandra/data/DataFileDirectory DataFileDirectory/storage02/cassandra/data/DataFileDirectory DataFileDirectory/storage03/cassandra/data/DataFileDirectory /DataFileDirectories After loading a big chunk of data into cassandra, I end up wich some 70GB in the first directory, and only about 10GB in the second and third one. All rows are quite small, so it's not just some big rows that contain the majority of data. Does Cassandra have the ability to 'see' the maximum available space in these directory? I'm asking myself this question since my limit is 100GB, and the first directory is approaching this limit... And, wouldn't it be better if Cassandra tried to 'load-balance' the files inside the directories because this will result in better (read) performance if the directories are on different disks (which is the case for me)? Any help is appreciated. Roland
Re: Can Cassandra make real use of several DataFileDirectories?
2010/4/26 Roland Hänel rol...@haenel.me: Hm... I understand that RAID0 would help to create a bigger pool for compactions. However, it might impact read performance: if I have several CF's (with their SSTables), random read requests for the CF files that are on separate disks will behave nicely - however if it's RAID0 then a random read on any file will create a random read on all of the hard disks. Correct? Without RAID0 you will end up with host spots (a compaction could end up putting a large SSTable on one disk, while the others have smaller SSTables). If you have many CFs this might average out, but it might not and there are no guarantees here. I'd reccomend RAID0 unless you have reason to do something else. -ryan
Re: Can Cassandra make real use of several DataFileDirectories?
Ryan, I agree with you on the hot spots, however for the physical disk performance, even the worst case hot spot is not worse than RAID0: in a hot spot scenario, it might be that 90% of your reads go to one hard drive. But with RAID0, 100% of your reads will go to *all* hard drives. But you're right, individual disks might waste up to 50% of your total disk space... I came to consider this idea because Hadoop DFS explicitely recommends different disks. But the design is not exactly the same, they don't have to deal with very big files on the native FS layer. -Roland 2010/4/26 Ryan King r...@twitter.com 2010/4/26 Roland Hänel rol...@haenel.me: Hm... I understand that RAID0 would help to create a bigger pool for compactions. However, it might impact read performance: if I have several CF's (with their SSTables), random read requests for the CF files that are on separate disks will behave nicely - however if it's RAID0 then a random read on any file will create a random read on all of the hard disks. Correct? Without RAID0 you will end up with host spots (a compaction could end up putting a large SSTable on one disk, while the others have smaller SSTables). If you have many CFs this might average out, but it might not and there are no guarantees here. I'd reccomend RAID0 unless you have reason to do something else. -ryan
Re: Can Cassandra make real use of several DataFileDirectories?
2010/4/26 Roland Hänel rol...@haenel.me: Ryan, I agree with you on the hot spots, however for the physical disk performance, even the worst case hot spot is not worse than RAID0: in a hot spot scenario, it might be that 90% of your reads go to one hard drive. But with RAID0, 100% of your reads will go to *all* hard drives. RAID0 is designed specifically to improve performance (both latency and bandwidth). I'm unclear about why you think it would decrease importance. Perhaps you're thinking of another RAID type? Paul Prescod
Re: Can Cassandra make real use of several DataFileDirectories?
RAID0 decreases the performance of muliple, concurrent random reads because for each read request (I assume that at least a couple of stripe sizes are read), all hard disks are involved in that read. Consider the following example: you want to read 1MB out of each of two files a) both files are on the same RAID0 of two disks. For the first 1MB read request, both disks contain some stripes of this request, both disks have to move their heads to the correct location and do the read. The second read request has to wait until the first one finishes, because it is served from the same disks and depends on the same disk heads. b) files are on seperate disks. Both reads can be done at the same time, because disk heads can move independently. Or look at it this way: if you issue a read request on a RAID0, and your disks have 8ms access time, then after the read request, the whole RAID0 is completely blocked for 8ms. If you handle the disks independently, only the disk containing the file is blocked. RAID0 has its advantages of course. Streaming reads/writes (e.g. during a compaction) will be extremely fast. -Roland 2010/4/26 Paul Prescod p...@prescod.net 2010/4/26 Roland Hänel rol...@haenel.me: Ryan, I agree with you on the hot spots, however for the physical disk performance, even the worst case hot spot is not worse than RAID0: in a hot spot scenario, it might be that 90% of your reads go to one hard drive. But with RAID0, 100% of your reads will go to *all* hard drives. RAID0 is designed specifically to improve performance (both latency and bandwidth). I'm unclear about why you think it would decrease importance. Perhaps you're thinking of another RAID type? Paul Prescod