cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Apache Wiki <wikidi...@apache.org>
Subject [Cassandra Wiki] Update of "ArchitectureInternals" by JonathanEllis
Date Tue, 24 Nov 2009 19:06:45 GMT
Dear Wiki user,

You have subscribed to a wiki page or wiki category on "Cassandra Wiki" for change notification.

The "ArchitectureInternals" page has been changed by JonathanEllis.
http://wiki.apache.org/cassandra/ArchitectureInternals?action=diff&rev1=2&rev2=3

--------------------------------------------------

   * !StorageProxy gets the nodes responsible for replicas of the keys from the !ReplicationStrategy,
then sends read messages to them
     * This may be a !SliceFromReadCommand, a !SliceByNamesReadCommand, or a !RangeSliceReadCommand,
depending
   * On the data node, !ReadVerbHandler gets the data from CFS.getColumnFamily or CFS.getRangeSlice
and sends it back as a !ReadResponse
+    * For single-row requests, we use a !QueryFilter subclass to pick the data from the Memtable
and SSTables that we are looking for.  The Memtable read is straightforward.  The SSTable
read is a little different depending on which kind of request it is:
+      * If we are reading a slice of columns, we use the row-level column index to find where
to start reading, and deserialize block-at-a-time (where "block" is the group of columns covered
by a single index entry) so we can handle the "reversed" case without reading vast amounts
into memory
+      * If we are reading a group of columns by name, we still use the column index to locate
each column, but first we check the row-level bloom filter to see if we need to do anything
at all
+    * The column readers provide an Iterator interface, so the filter can easily stop when
it's done, without reading more columns than necessary
+      * Since we need to potentially merge columns from multiple SSTable versions, the reader
iterators are combined through a !ReducingIterator, which takes an iterator of uncombined
columns as input, and yields combined versions as output
   * If a quorum read was requested, !StorageProxy waits for a majority of nodes to reply
and makes sure the answers match before returning.  Otherwise, it returns the data reply as
soon as it gets it, and checks the other replies for discrepancies in the background in !StorageService.doConsistencyCheck.
 This is called "read repair," and also helps achieve consistency sooner.
     * As an optimization, !StorageProxy only asks the closest replica for the actual data;
the other replicas are asked only to compute a hash of the data.
  

Mime
View raw message