If you're interested in this idea, you should read up about Spinnaker: http://www.vldb.org/pvldb/vol4/p243-rao.pdf
-ryan On Mon, Jul 11, 2011 at 2:48 PM, Yang <teddyyyy...@gmail.com> wrote: > I'm not proposing any changes to be done, but this looks like a very > interesting topic for thought/hack/learning, so the following are only > for thought exercises .... > > > HBase enforces a single write/read entry point, so you can achieve > strong consistency by writing/reading only one node. but just writing > to one node exposes you to loss of data if that node fails. so the > region server HLog is replicated to 3 HDFS data nodes. the > interesting thing here is that each replica sees a complete *prefix* > of the HLog: it won't miss a record, if a record sync() to a data node > fails, all the existing bytes in the block are replicated to a new > data node. > > if we employ a similar "leader" node among the N replicas of > cassandra (coordinator always waits for the reply from leader, but > leader does not do further replication like in HBase or counters), the > leader sees all writes onto the key range, but the other replicas > could miss some writes, as a result, each of the non-leader replicas' > write history has some "holes", so when the leader dies, and when we > elect a new one, no one is going to have a complete history. so you'd > have to do a repair amongst all the replicas to reconstruct the full > history, which is slow. > > it seems possible that we could utilize the FIFO property of the > InComingTCPConnection to simplify history reconstruction, just like > Zookeeper. if the IncomingTcpConnection of a replica fails, that means > that it may have missed some edits, then when it reconnects, we force > it to talk to the active leader first, to catch up to date. when the > leader dies, the next leader is elected to be the replica with the > most recent history. by maintaining the property that each node has a > complete prefix of history, we only need to catch up on the tail of > history, and avoid doing a complete repair on the entire > memtable+SStable. but one issue is that the history at the leader has > to be kept really long ----- if a non-leader replica goes off for 2 > days, the leader has to keep all the history for 2 days to feed them > to the replica when it comes back online. but possibly this could be > limited to some max length so that over that length, the woken replica > simply does a complete bootstrap. > > > thanks > yang > On Sun, Jul 3, 2011 at 8:25 PM, AJ <a...@dude.podzone.net> wrote: >> We seem to be having a fundamental misunderstanding. Thanks for your >> comments. aj >> >> On 7/3/2011 8:28 PM, William Oberman wrote: >> >> I'm using cassandra as a tool, like a black box with a certain contract to >> the world. Without modifying the "core", C* will send the updates to all >> replicas, so your plan would cause the extra write (for the placeholder). I >> wasn't assuming a modification to how C* fundamentally works. >> Sounds like you are hacking (or at least looking) at the source, so all the >> power to you if/when you try these kind of changes. >> will >> On Sun, Jul 3, 2011 at 8:45 PM, AJ <a...@dude.podzone.net> wrote: >>> >>> On 7/3/2011 6:32 PM, William Oberman wrote: >>> >>> Was just going off of: " Send the value to the primary replica and send >>> placeholder values to the other replicas". Sounded like you wanted to write >>> the value to one, and write the placeholder to N-1 to me. >>> >>> Yes, that is what I was suggesting. The point of the placeholders is to >>> handle the crash case that I talked about... "like" a WAL does. >>> >>> But, C* will propagate the value to N-1 eventually anyways, 'cause that's >>> just what it does anyways :-) >>> will >>> >>> On Sun, Jul 3, 2011 at 7:47 PM, AJ <a...@dude.podzone.net> wrote: >>>> >>>> On 7/3/2011 3:49 PM, Will Oberman wrote: >>>> >>>> Why not send the value itself instead of a placeholder? Now it takes 2x >>>> writes on a random node to do a single update (write placeholder, write >>>> update) and N*x writes from the client (write value, write placeholder to >>>> N-1). Where N is replication factor. Seems like extra network and IO >>>> instead of less... >>>> >>>> To send the value to each node is 1.) unnecessary, 2.) will only cause a >>>> large burst of network traffic. Think about if it's a large data value, >>>> such as a document. Just let C* do it's thing. The extra messages are >>>> tiny >>>> and doesn't significantly increase latency since they are all sent >>>> asynchronously. >>>> >>>> >>>> Of course, I still think this sounds like reimplementing Cassandra >>>> internals in a Cassandra client (just guessing, I'm not a cassandra dev) >>>> >>>> I don't see how. Maybe you should take a peek at the source. >>>> >>>> >>>> On Jul 3, 2011, at 5:20 PM, AJ <a...@dude.podzone.net> wrote: >>>> >>>> Yang, >>>> >>>> How would you deal with the problem when the 1st node responds success >>>> but then crashes before completely forwarding any replicas? Then, after >>>> switching to the next primary, a read would return stale data. >>>> >>>> Here's a quick-n-dirty way: Send the value to the primary replica and >>>> send placeholder values to the other replicas. The placeholder value is >>>> something like, "PENDING_UPDATE". The placeholder values are sent with >>>> timestamps 1 less than the timestamp for the actual value that went to the >>>> primary. Later, when the changes propagate, the actual values will >>>> overwrite the placeholders. In event of a crash before the placeholder >>>> gets >>>> overwritten, the next read value will tell the client so. The client will >>>> report to the user that the key/column is unavailable. The downside is >>>> you've overwritten your data and maybe would like to know what the old data >>>> was! But, maybe there's another way using other columns or with MVCC. The >>>> client would want a success from the primary and the secondary replicas to >>>> be certain of future read consistency in case the primary goes down >>>> immediately as I said above. The ability to set an "update_pending" flag >>>> on >>>> any column value would probably make this work. But, I'll think more on >>>> this later. >>>> >>>> aj >>>> >>>> On 7/2/2011 10:55 AM, Yang wrote: >>>> >>>> there is a JIRA completed in 0.7.x that "Prefers" a certain node in >>>> snitch, so this does roughly what you want MOST of the time >>>> >>>> but the problem is that it does not GUARANTEE that the same node will >>>> always be read. I recently read into the HBase vs Cassandra comparison >>>> thread that started after Facebook dropped Cassandra for their messaging >>>> system, and understood some of the differences. what you want is >>>> essentially >>>> what HBase does. the fundamental difference there is really due to the >>>> gossip protocol: it's a probablistic, or eventually consistent failure >>>> detector while HBase/Google Bigtable use Zookeeper/Chubby to provide a >>>> strong failure detector (a distributed lock). so in HBase, if a tablet >>>> server goes down, it really goes down, it can not re-grab the tablet from >>>> the new tablet server without going through a start up protocol (notifying >>>> the master, which would notify the clients etc), in other words it is >>>> guaranteed that one tablet is served by only one tablet server at any given >>>> time. in comparison the above JIRA only TRYIES to serve that key from one >>>> particular replica. HBase can have that guarantee because the group >>>> membership is maintained by the strong failure detector. >>>> just for hacking curiosity, a strong failure detector + Cassandra >>>> replicas is not impossible (actually seems not difficult), although the >>>> performance is not clear. what would such a strong failure detector bring >>>> to >>>> Cassandra besides this ONE-ONE strong consistency ? that is an interesting >>>> question I think. >>>> considering that HBase has been deployed on big clusters, it is probably >>>> OK with the performance of the strong Zookeeper failure detector. then a >>>> further question was: why did Dynamo originally choose to use the >>>> probablistic failure detector? yes Dynamo's main theme is "eventually >>>> consistent", so the Phi-detector is **enough**, but if a strong detector >>>> buys us more with little cost, wouldn't that be great? >>>> >>>> >>>> On Fri, Jul 1, 2011 at 6:53 PM, AJ <a...@dude.podzone.net> wrote: >>>>> >>>>> Is this possible? >>>>> >>>>> All reads and writes for a given key will always go to the same node >>>>> from a client. It seems the only thing needed is to allow the clients to >>>>> compute which node is the closes replica for the given key using the same >>>>> algorithm C* uses. When the first replica receives the write request, it >>>>> will write to itself which should complete before any of the other >>>>> replicas >>>>> and then return. The loads should still stay balanced if using random >>>>> partitioner. If the first replica becomes unavailable (however that is >>>>> defined), then the clients can send to the next repilca in the ring and >>>>> switch from ONE write/reads to QUORUM write/reads temporarily until the >>>>> first replica becomes available again. QUORUM is required since there >>>>> could >>>>> be some replicas that were not updated after the first replica went down. >>>>> >>>>> Will this work? The goal is to have strong consistency with a >>>>> read/write consistency level as low as possible while secondarily a >>>>> network >>>>> performance boost. >>>> >>>> >>>> >>> >>> >>> >>> -- >>> Will Oberman >>> Civic Science, Inc. >>> 3030 Penn Avenue., First Floor >>> Pittsburgh, PA 15201 >>> (M) 412-480-7835 >>> (E) ober...@civicscience.com >>> >> >> >> >> -- >> Will Oberman >> Civic Science, Inc. >> 3030 Penn Avenue., First Floor >> Pittsburgh, PA 15201 >> (M) 412-480-7835 >> (E) ober...@civicscience.com >> >> >