incubator-cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Julian Simon <jsi...@jules.com.au>
Subject Re: Large data files and no "edit in place"?
Date Tue, 30 Mar 2010 21:22:43 GMT
Thanks for the detailed explanation David.  I had a feeling it was to
do with random vs sequential IO, and now I'm comfortable with that
concept w.r.t Cassandra.



On Tue, Mar 30, 2010 at 11:59 PM, David Strauss <david@fourkitchens.com> wrote:
> On 2010-03-30 05:54, Julian Simon wrote:
>> My understanding is that Cassandra never updates data "in place" on
>> disk - instead it completely re-creates the data files during a
>> "flush".  Stop me if I'm wrong already ;-)
>
> You're correct that existing SSTables are immutable; they are retired
> following compaction rather than modified.
>
>> So imagine we have a large data set in our ColumnFamily and we're
>> constantly adding data to it.
>
> Sounds like a good reason to consider Cassandra.
>
>> Every [x] minutes or [y] bytes, the compaction process is triggered,
>> and the entire data set is written to disk.
>
> The compaction process takes several stages:
> http://wiki.apache.org/cassandra/MemtableSSTable
>
>> So as our data set grows over time, the compaction process will result
>> in an increasingly large IO operation to write all that data to disk
>> each time.
>
> I'll interpret that to mean "an increasingly large IO operation [for
> each node] to write all that data to disk each time."
>
> That is not entirely correct from an operational standpoint. In a
> cluster where the node count exceeds ReplicationFactor, a single server
> only handles a fraction of the rows in each CF. If a cluster ever
> reached the point where, on some boxes, compaction required too much IO
> to sustain, you would simply expand the cluster (keeping the same
> ReplicationFactor). This would distribute the load over more nodes.
>
>> We could easily be talking about single data files in the
>> many-gigabyte size range, no?  Or is there a file size limit that I'm
>> not aware of?
>
> It's certainly possible to reach a multi-GB size for the SSTable files,
> but that should not be a problem.
>
>> If not, is this an efficient approach to take for large data sets?
>> Seems like we would become awfully IO bound, writing the entire thing
>> from scratch each time.
>>
>> Do let me know if I've gotten it all wrong ;-)
>
> The mistake you're making is assuming that IO capability is equivalent
> for sequential and random activity. A system that replaces items on disk
> (when possible) may write fewer bytes to disk, but it is at the cost of
> many seeks. On a standard hard disk, those seeks are expensive.
> Cassandra is optimized around avoiding seeks on write, even if it has to
> write much more data sequentially over the long term.
>
> Imagine I was asking you to fetch water from around the city. For the
> first round, I have you fetch 100 thimbles of water randomly scattered
> over the city. It's not very much water, but it takes you a very long time.
>
> On the second round, I have you fetch 20x the water in volume (say, 20
> buckets), but it's all on one street corner. Despite the considerable
> increase in volume, you're done much faster on the second round.
>
> Then, on an ongoing basis, you have the daily choice of fetching 30
> thimbles (30% of the original thimble set) all over town or 20 buckets
> (100% of the original set) on a designated street corner.
>
> You'd want to pick the bucket option -- despite the higher volume -- and
> your hard disk would agree.
>
> --
> David Strauss
>   | david@fourkitchens.com
> Four Kitchens
>   | http://fourkitchens.com
>   | +1 512 454 6659 [office]
>   | +1 512 870 8453 [direct]
>
>

Mime
View raw message