You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Jonathan Ellis (JIRA)" <ji...@apache.org> on 2012/06/25 19:03:44 UTC

[jira] [Commented] (CASSANDRA-3881) reduce computational complexity of processing topology changes

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

Jonathan Ellis commented on CASSANDRA-3881:
-------------------------------------------

This looks reasonable.  I have two concerns:

- Topology syncnization: a mix between "Topology synchronizes internally" and "caller must synchronize externally" is a recipe for trouble.  Maybe just synchronizing getDatacenterEndpoints/getDatacenterRacks and returning copies, would be enough.  Alternatively, we could just say "you must clone TMD before calling calculateNaturalEndpoints" and possibly get rid of all the Topology synchronization (relying on TMD's on the update path)
- I think there is a hole in the rack-handling logic in cNE: we only check skippedDcEndpoints when a new rack is found.  So if there is (for instance) a single rack in a DC w/ RF=3, we'll add the first endpoint in that rack, then the rest will get added to the skipped list, but never added to replicas.

00 nits:
- would like to see javadoc for Topology
- type specification is not necessary for HMM.create calls (this is why guava prefers factories to constructors)
- I recognize that you were following precedent here, but I would prefer the param-less TMD constructor to call new TMD(HashBimap.create(), new Topology()) instead of (null, null) + special casing in the 2-param version

01 nits:
- @Override is redundant for calculateNaturalEndpoints since parent declares it abstract
- some "} else" formatting issues in cNE
- "!skippedDcEndpoints.get(dc).isEmpty()" is redundant since the empty iterator case will just be a no-op in the following loop
- perhaps dcReplicas would be a better name than replicaEndpoints, for symmetry w/ "replicas"
- would be cleaner to move the "replicaEndpoints.get(dc).size() >= Math.min(allEndpoints.get(dc).size(), getReplicationFactor)" check into "have we already found all replicas for this dc", instead of playing games w/ mutating replicaEndpoints as an (unimportant) optimization.  (Note that "datacenters.containsKey(dc)" remains a sufficient check for "is this a dc we care about at all.")
                
> reduce computational complexity of processing topology changes
> --------------------------------------------------------------
>
>                 Key: CASSANDRA-3881
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3881
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>            Reporter: Peter Schuller
>            Assignee: Sam Overton
>              Labels: vnodes
>
> This constitutes follow-up work from CASSANDRA-3831 where a partial improvement was committed, but the fundamental issue was not fixed. The maximum "practical" cluster size was significantly improved, but further work is expected to be necessary as cluster sizes grow.
> _Edit0: Appended patch information._
> h3. Patches
> ||Compare||Raw diff||Description||
> |[00_snitch_topology|https://github.com/acunu/cassandra/compare/refs/top-bases/p/3881/00_snitch_topology...p/3881/00_snitch_topology]|[00_snitch_topology.patch|https://github.com/acunu/cassandra/compare/refs/top-bases/p/3881/00_snitch_topology...p/3881/00_snitch_topology.diff]|Adds some functionality to TokenMetadata to track which endpoints and racks exist in a DC.|
> |[01_calc_natural_endpoints|https://github.com/acunu/cassandra/compare/refs/top-bases/p/3881/01_calc_natural_endpoints...p/3881/01_calc_natural_endpoints]|[01_calc_natural_endpoints.patch|https://github.com/acunu/cassandra/compare/refs/top-bases/p/3881/01_calc_natural_endpoints...p/3881/01_calc_natural_endpoints.diff]|Rewritten O(logN) implementation of calculateNaturalEndpoints using the topology information from the tokenMetadata.|
> ----
> _Note: These are branches managed with TopGit. If you are applying the patch output manually, you will either need to filter the TopGit metadata files (i.e. {{wget -O - <url> | filterdiff -x*.topdeps -x*.topmsg | patch -p1}}), or remove them afterward ({{rm .topmsg .topdeps}})._

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira