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

Sam Overton commented on CASSANDRA-3881:
----------------------------------------

Thanks Jonathan. I've addressed the above in the following commits:
7eab101 [incremental 
patch|https://github.com/acunu/cassandra/commit/7eab101fd1649737682d4ce30004a15d0c6c2343]
 ([raw 
diff|https://github.com/acunu/cassandra/commit/7eab101fd1649737682d4ce30004a15d0c6c2343.diff])
1474cf0 [incremental 
patch|https://github.com/acunu/cassandra/commit/1474cf090f5b2c80bbe573d16041458f1782ecbb]
 ([raw 
diff|https://github.com/acunu/cassandra/commit/1474cf090f5b2c80bbe573d16041458f1782ecbb.diff])

(patch links above updated)

except for the following:

{quote}
* Topology syncronization: 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)
{quote}

I was trying to avoid any copying, as calculateNaturalEndpoints will be called 
thousands of times with vnodes in some code paths. I prefer the latter solution 
of cloning TMD before using it in any method which will use the Topology. The 
only places where cloning will be necessary to avoid concurrent updates are 
those where StorageService.instance.tokenMetadata is used directly. I'll update 
the patches shortly.

{quote}
* 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.
{quote}
I think this case is already handled: the subsequent endpoints for that 
duplicate rack will hit this line first:
{noformat}
    // can we skip checking the rack?
    if (seenRacks.get(dc).size() == racks.get(dc).keySet().size())
{noformat}
and they get added as a replica immediately because we know we have exhausted 
the racks for that DC. Did I miss something?


                
> 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

        

Reply via email to