[
https://issues.apache.org/jira/browse/CASSANDRA-492?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12768808#action_12768808
]
Jonathan Ellis commented on CASSANDRA-492:
------------------------------------------
EndPointSnitch, IEndPointSnitch, and CassandraServer hunks do not apply to trunk
we shouldn't need getMapReplicationFactor in IEndPointSnitch, and shouldn't
need insertDCBlocking, this is too much abstraction leakage. i suggest moving
determineBlockFor into AbstractReplicationStrategy, then the DC strategy
version can use the configuration to determine blockFor w/o making
EndPointSnitch/StorageProxy have to know about it. actually we probably need
more than that, maybe a getQuorumResponseHandler(consistency_level) method and
you can subclass QRH to know about the details of the DC layouts. Bottom line,
StorageProxy shouldn't need to care about the details of the
replicationstrategy.
don't include boilerplate like this
+ * @param rm
+ * @param blockFor
+ * @param endpointMap
+ * @param primaryNodes
+ * @throws InvalidRequestException
+ * @throws UnavailableException
+ * @throws IOException
+ * @throws DigestMismatchException
+ * @throws TimeoutException
real docstrings are fine but this doesn't add anything to what the method
signature tells us
there is still a lot of whitespace/formatting getting changed unnecessarily
e.g. in StorageProxy.
> 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.