cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Jeremy Hanna (JIRA)" <j...@apache.org>
Subject [jira] Commented: (CASSANDRA-952) DC Quorum broken @ trunk
Date Thu, 13 May 2010 16:50:44 GMT

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

Jeremy Hanna commented on CASSANDRA-952:
----------------------------------------

Reviewing part of it and I know this wasn't part of the change - but in RackAwareStrategy
couldn't we save some instruction execution by moving the check for whether we already have
found another datacenter/rack outside of the checks?  Like so:

<code>
Token t = iter.next();

// First try to find one in a different data center
// If we have already found something in a diff datacenter no need to find another
if (!bDataCenter)
{
    if (!snitch.getDatacenter(metadata.getEndpoint(primaryToken)).equals(snitch.getDatacenter(metadata.getEndpoint(t))))
    {
        endpoints.add(metadata.getEndpoint(t));
        bDataCenter = true;
        continue;
    }
}

// Now  try to find one on a different rack
// If we have already found something in a diff rack no need to find another
if (!bOtherRack)
{
    if (!snitch.getRack(metadata.getEndpoint(primaryToken)).equals(snitch.getRack(metadata.getEndpoint(t)))
&&
        snitch.getDatacenter(metadata.getEndpoint(primaryToken)).equals(snitch.getDatacenter(metadata.getEndpoint(t))))
    {
        endpoints.add(metadata.getEndpoint(t));
        bOtherRack = true;
        continue;
    }
}
</code>

not that big of a deal, but since it's done on every write, might be helpful.  Maybe I'm missing
something there though...

> DC Quorum broken @ trunk
> ------------------------
>
>                 Key: CASSANDRA-952
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-952
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>    Affects Versions: 0.6
>         Environment: Linux, Cassandra
>            Reporter: Vijay
>            Assignee: Vijay
>            Priority: Minor
>             Fix For: 0.7, 0.8
>
>         Attachments: 0001-clean-out-callback-purging-from-truncate-writerh.txt, 0002-dcquorum-fixes.txt,
952-Canges_To_Stategy_V002.txt, 952-Change_BlockFor.txt, 952-Changes_To_ResponseHandler_v002.txt,
952-Fix_Refactor_DCStatergy.txt, 952-v3.txt, 952-v4.txt, 952-v5.txt, DC-Config.xml
>
>
> Currently DCQuorum is broken in trunk, Suggesting the following fix... 
> Write to DC's
> 1) Move determineBlockFor(int expandedTargets, ConsistencyLevel consistency_level) to
AbstractEndpointSnitch
> 2) Add the same to support DC Quorum in DatacenterShardStategy
> Read to DC's
> 1) find suitable nodes was a list which was returning a list of local DC's earlier but
now it is just one node and MD is been sent by other nodes. Need to have an option to even
avoid MD from other DC's?

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


Mime
View raw message