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. The comment on this change is: clean up read repair treatment. http://wiki.apache.org/cassandra/ArchitectureInternals?action=diff&rev1=8&rev2=9 -------------------------------------------------- = Read path = * !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 + * The request for actual data is sent to the closest node as determined by the !AbstractEndpointSnitch implementation; the other nodes are sent "digest" requests, i.e., a request for an md5 of the data. This allows comparing consistency of the replicas, without actually sending the full data more than once. * 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. + * Back on the coordinating (originating) node, replies are processed by ResponseVerbHandler. 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 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. = Gossip = * based on "Efficient reconciliation and flow control for anti-entropy protocols:" http://www.cs.cornell.edu/home/rvr/papers/flowgossip.pdf
