You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Josh McKenzie (Jira)" <ji...@apache.org> on 2022/12/01 17:54:00 UTC

[jira] [Commented] (CASSANDRA-18084) Introduce tags to snitch for better decision making for replica placement in topology strategies

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

Josh McKenzie commented on CASSANDRA-18084:
-------------------------------------------

{quote}The main motivation behind this is that we have special requirements around node's characteristics based on which we want to make further decisions when it comes to replica placement in topology strategies. (New topology strategy would be mostly derived from NTS / would be extended)
{quote}
ISTM that the intersection of this and Transactional Metadata in the future w/range ownership rather than token ownership could become pretty interesting. If we tagged topology data with hardware resources in a heterogenous environment, for instance, we could preferentially serve hot ranges from beefier hardware etc.

I don't have any major concerns w/the idea from a perspective of the old arch, but curious what [~ifesdjeen] and/or [~samt] think as they're working CEP-21.

> Introduce tags to snitch for better decision making for replica placement in topology strategies
> ------------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-18084
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-18084
>             Project: Cassandra
>          Issue Type: New Feature
>          Components: Cluster/Gossip, Cluster/Schema, Legacy/Distributed Metadata
>            Reporter: Stefan Miklosovic
>            Assignee: Stefan Miklosovic
>            Priority: Normal
>
> We would like to have extra meta-information in cassandra-rackdc.properties which would further differentiate nodes in dc / racks.
> The main motivation behind this is that we have special requirements around node's characteristics based on which we want to make further decisions when it comes to replica placement in topology strategies. (New topology strategy would be mostly derived from NTS / would be extended)
> The most reasonable way to do that is to introduce a new property into cassandra-rackdc.properties called "tags"
> {code:java}
> # Arbitrary tag to assign to this node, they serve as additional identificator of a node based on which operators might act. 
> # Value of this property is meant to be a comma-separated list of strings.
> #tags=tag1,tag2 
> {code}
> We also want to introduce new application state called TAGS. On startup of a node, this node would advertise its tags to cluster and vice versa, all nodes would tell to that respective node what tags they have so everybody would see the same state of tags across the cluster based on which topology strategies would do same decisions.
> These tags are not meant to be changed during whole runtime of a node, similarly as datacenter and rack is not.
> For performance reasons, we might limit the maximum size of tags (sum of their lenght), to be, for example, 64 characters and anything bigger would be either shortened or the start would fail.
> Once we have tags for all nodes, we can have access to them, cluster-wide, from TokenMetadata which is used quite heavily in topology strategies and it exposes other relevant topology information (dc's, racks ...). We would just add a new way to look at nodes.
> Tags would be a set.
> This would be persisted to the system.local to see what tags a local node has and it would be persisted to system.peers_v2 to see what tags all other nodes have. Column would be called "tags".
> {code:java}
> admin@cqlsh> select * from system.local ;
> @ Row 1
>  key                     | local
>  bootstrapped            | COMPLETED
>  broadcast_address       | 172.19.0.5
>  broadcast_port          | 7000
>  cluster_name            | Test Cluster
>  cql_version             | 3.4.6
>  data_center             | dc1
>  gossip_generation       | 1669739177
>  host_id                 | 54f8c6ea-a6ba-40c5-8fa5-484b2b4184c9
>  listen_address          | 172.19.0.5
>  listen_port             | 7000
>  native_protocol_version | 5
>  partitioner             | org.apache.cassandra.dht.Murmur3Partitioner
>  rack                    | rack1
>  release_version         | 4.2-SNAPSHOT
>  rpc_address             | 172.19.0.5
>  rpc_port                | 9042
>  schema_version          | ef865449-2491-33b8-95b0-47c09cb14ea9
>  tags                    | {'tag1', 'tag2'}
>  tokens                  | {'6504358681601109713'} 
> {code}
> for system.peers_v2:
> {code:java}
> admin@cqlsh> select peer,tags from system.peers_v2 ;
> @ Row 1
> ------+-----------------
>  peer | 172.19.0.15
>  tags | {'tag2', 'tag3'}
> @ Row 2
> ------+-----------------
>  peer | 172.19.0.11
>  tags | null
> {code}
> the POC implementation doing exactly that is here:
> We do not want to provide our custom topology strategies which are using this feature as that will be the most probably a proprietary solution. This might indeed change in the future. For now, we just want to implement hooks we can base our in-house implementation on. All other people can benefit from this as well if they choose so as this feature enables them to do that.
> Adding tags is not only about custom topology strategies. Operators could tag their nodes if they wish to make further distinctions on topology level for their operational needs.
> [https://github.com/instaclustr/cassandra/commit/eddd4681d8678515454dabb926d5f56b0c225eea]



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org