[ 
https://issues.apache.org/jira/browse/CASSANDRA-492?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12769402#action_12769402
 ] 

Jonathan Ellis commented on CASSANDRA-492:
------------------------------------------

Much better!  The basic foundation is right now, just a little cleanup to do:

the code in DataCenterShardSingleton belongs in DatacenterShardStategy

DatacenterShardStategy should extend ARS and implement the abstract method, 
rather than overriding a non-abstract method for RUS.  (otherwise, you break 
bootstrap.)

DSS/DCSS shouldn't care about the bootstrap tokens, only the write methods in 
ARS need that and you shouldn't have to mess with those

rename DCQuorumResponseHandlerSync to DCQuorumSyncResponseHandler for 
consistency

change getNewQuorumRespone to
public ResponseHandler getQuorumResponseHandler(IResponseResolver<Boolean> 
responseResolver, int consistency_level)

"This Class has the dependency of DataCenterEndpointSnitch.class to be used." 
you should verify that with an assert

use a 120 column width in formatting, not 80, and read through the code and fix 
obvious ugliness like
+     * @param endPoint
+     *            the endPoint to process
or
+               InetAddress endPointOfIntrest = tokenToEndPointMap.get(tokens
+                       .get(i));

goffinet, I think you had some comments about reloadConfiguration using 
properties files instead of xml?

> Data Center Quorum
> ------------------
>
>                 Key: CASSANDRA-492
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-492
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Core
>    Affects Versions: 0.5
>         Environment: Cassandra Core Code
>            Reporter: Vijay
>            Priority: Minor
>             Fix For: 0.5
>
>         Attachments: cassandra-DCAWARE.patch, DC-Config.xml
>
>   Original Estimate: 336h
>  Remaining Estimate: 336h
>
> Need a Cassandra Datacenter Quorum Read and Datacenter Quorum Write.
> Add 1 new enum DC_Quorum. basically reads with this will not span across the 
> datacenter it will use the existing nodes in the Datacenter which has this 
> data and read from it.
> For writes - All the data centers need to get this data, (datac enters will 
> be configured in the storage-config.xml and number of replicas in it). Once 
> configured write will basically write to all the nodes in all the datacenter 
> but will wait only for the write in the current datacenter. 
> Example: We have 3 Datacenter A,B,C A has a replication factor of 3, B has 2 
> and C has 2. DC_Quorum write will make sure to write on 2 of 3 nodes in A.... 
> B and C (total 4 +1) nodes will be eventually consistent.
> Changes will be in RackAware, storage, read and write classes.
> Thanks
> Vijay

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.

Reply via email to