On Tue, Apr 13, 2010 at 10:48 AM, Time Less <timelessn...@gmail.com> wrote: > > >> > If I have 10B rows in my CF, and I can fit 10k rows per >> > SStable, and the SStables are spread across 5 nodes, and I have 1 bloom
The error you are making is in thinking the Memtable thresholds are the SSTable limits. They are not. >> > filter false positive and 1 tombstone and ask the wrong node for the >> > key, Why would I ask the wrong node for the key? I know the tokens for every node, so I know exactly which nodes have the replicas. If I am asking the wrong node for a key, there is a bug. >> > then: >> > >> > Mv = (((2B/10k)+1+1)*3)+1 == ((200,000)+2)*3+1 == 300,007 iops to read a >> > key. >> >> This is a nonsensical arrangement. Assuming each SSTable is the size >> of the default Memtable threshold (128MB), then each row is (128MB / >> 10k) == 12.8k and 10B rows == 128TB of raw data. A typical RF of 3 >> takes us to 384TB. The need for enough space for compactions takes us >> to 768TB. That's not 5 nodes, it's more like 100+, and almost 2 >> orders of magnitude off your estimate, > > You started off so well. You laid out a couple of useful points: > > (1) for a 10B-row dataset, 12.8KB rows, RF=3, Cassandra cluster requires > 768TB. If you have less, you'll run into severe administration problems. > This is not obvious, but is critical and extremely useful. > The 384TB implied by RF=3 and 128TB of raw data is obvious. The additional 384TB of space to permit worst case compaction (a compaction of SSTables with no tombstones) might not be immediately obvious, but does not meaningfully change the situation: even at 384TB, your 5 node assumption is way off. > (2) 12.8KB rowsize wants a >128MB memtable threshold. Is there a rule of > thumb for this? memTableThreshold = rowsize * 100,000? > How frequently do you want to write SSTables? How much memory do you want Memtables to consume? How long do you want to wait between Memtable flushes? There is an entire wiki page on Memtable tuning: http://wiki.apache.org/cassandra/MemtableThresholds . There is a thorough discussion on the various tuning parameters around buffering and writing here: http://wiki.apache.org/cassandra/StorageConfiguration . Do you understand you are assuming there have been no compactions, which would be extremely bad practice given this number of SSTables? A major compaction, as would be best practice given this volume, would result in 1 SSTable per CF per node. One. Similarly, you are assuming the update is only on the last replica checked, but the system is going to read and write the first replica (the node that actually has that range based on its token) first in almost all situations. Not worst case? If 'we' are coming up with arbitrarily bad situations, why not assume 1 row per SSTable, lots of tombstones, in addition to no compactions? Why not assume RF=100? Why not assume node failures right in the middle of your query? The interesting question is not 'how bad can this get if you configure and operate things really badly?', but 'how bad can this get if you configure and operate things according to best practices?'. b