You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@cassandra.apache.org by Alex Lourie <al...@instaclustr.com> on 2018/06/28 04:52:08 UTC

Feedback requested for CASSANDRA-11559 (enhancing node representation)

Hi all


I've been recently working on the ticket mentioned above (
https://issues.apache.org/jira/browse/CASSANDRA-11559). The ticket is
suggesting to improve the current representation of the node from IP
address and port to IP address, the port and the UUID. This would
potentially allow more convenient work with nodes in many operations, but
particularly the replacement ones come into mind.


I've done some work on this ticket and it's available at
https://github.com/apache/cassandra/compare/trunk...alourie:CASSANDRA-11559#files_bucket.
Few
pointers:




   1. I've refactored *InetAddressAndPort* class into *Endpoint* class
   across all the codebase (this will be responsible for the majority of the
   code changes).
   2. I've added a UUID field to hold the hostID value of the endpoint and
   added additional methods for working with it.
   3. I've reworked the *TokenMetadata* to hold structures other than maps
   for UUID-host references and they would no longer be needed, i.e. keeping
   just a set of endpoints is enough to hold both address data and the
   hostID data and to also look up hosts by IDs or the vice versa.
   4. I've reworked the *SystemKeyspace* to also acknowledge the hostIDs where
   significant (in local data/peer data storing/fetching), and also only
   create new local id if requested (in most cases only when the node is
   created for the first time, but also useful for tests that require
   initiating multiple "nodes" on the same machine)
   5. I've added a field in *DatabaseDescriptor* to mark that
   *SystemKeyspace* is ready to be read. This is required for many unit
   tests that set up clusters "on the fly" and for further endpoint
   information discovery during the test run.
   6. I've updated required unit tests to properly utilise the new object
   and initialise others as required.
   7. I've updated the code in some other locations to incorporate this
   change, which does make it simpler on many occasions.


The current state is everything *seems* to be working fine in ccm tests and
the unit tests pass (https://circleci.com/gh/alourie/cassandra/123)

The complication that comes out of this work is developing unit tests is a
bit more complex - the host ID would now have to be kept in
multiple structures:


   - An *Endpoint* object when instantiated.
   - *SystemKeyspace.localHost* (queries the DB)
   - *SystemKeyspace.peersInfo* (queries the DB)
   - *TokenMetadata* lists (such as allEndpoints, tokenMap, etc)
   - *Gossip.instance.endpointState* maps (the specific endpoint is added
   including the uuid)
   - *FBUtilities* also keeps local reference once fetched.

          As a result, when you're creating tests, you'd need to update or
clear the hostID-related information in all relevant places, otherwise,
tests would fail with really confusing messages (in most cases because in
some thread an endpoint comparison will happen and UUIDs won't match), such
as "no seeds found", "host cannot be contacted" or various kinds of
timeouts and NPEs. Additionally, when SystemKeyspace is ready to be read
within a test flow, a *DatabaseDescriptor.canReadSystemKeyspace* field will
need to be set to *true* so that the UUID would be fetched from
*SystemKeyspace*.



Additionally, at the moment we are keeping *EndpointState* separately from
this object (in Gossip). Considering that now Endpoint object can include
basically any information about the endpoint, it may as well incorporate
its own state, and then all handling of the network/state information about
an endpoint will be kept in one place. Supposedly this should simplify
things further and allow clearing a lot of code.

 Ariel Weisberg has done the previous move away from InetAddress
representation to InetAddressAndPort, which this current patch changes
considerably. I'd love your feedback on this.



Any and all feedback is very welcome.

Thanks.

Alex Lourie.


P.S. The longer this work stays without any feedback the harder it is to
update every time after considerable work is done on trunk, as many tests
are being developed not using this code. The faster it is reviewed and,
hopefully, accepted, the easier it will be.
-- 

*Alex Lourie*


*Senior Software Engineer*+61 423177059

<https://www.instaclustr.com/solutions/managed-apache-kafka/>

   <https://twitter.com/instaclustr>
<https://www.linkedin.com/company/instaclustr>

Read our latest technical blog posts here
<https://www.instaclustr.com/blog/>.

This email has been sent on behalf of Instaclustr Pty. Limited (Australia)
and Instaclustr Inc (USA).

This email and any attachments may contain confidential and legally
privileged information.  If you are not the intended recipient, do not copy
or disclose its content, but please reply to this email immediately and
highlight the error to the sender and then immediately delete the message.