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

John Roesler commented on KAFKA-6718:
-------------------------------------

Hey [~lkokhreidze]. I know Matthias pinged me a while back... I've just been 
re-queuing the task to look at this ticket every day to the next day. Sorry 
about that.

I actually don't think that there's anything specific to worry about with 
respect to KIP-441. In my opinion, you can just design your feature against the 
current state of Streams, and whoever loses the race would just have to deal 
with adjusting the implementation to take the other feature into account.

For what it's worth, though, I don't think there's too much semantic overlap, 
just the practical overlap that both efforts affect the same protocol and 
module. That's why I don't think we need to wait for each other _or_ try to 
unify the efforts.

I don't know if you have any specific ideas, and I haven't had a chance to look 
into the details of the old work on this ticket. (This is what I haven't been 
able to find time to do). But sharing my immediate thoughts, I've used the 
"rack awareness" feature in Elasticsearch, and felt that it was a pretty 
straightforward and reasonable way to implement it. See 
https://www.elastic.co/guide/en/elasticsearch/reference/current/allocation-awareness.html
 for more information.

Basically, they let you add some arbitrary "tags"/"attrs" to each instance (in 
the config), and then there's a "rack awareness" config that takes a list of 
"tags" to consider. It's a pretty simple, but also powerful, arrangement. You 
might also want to consider the "forced awareness" section, which is in 
response to a concern that also applies to us.

Thanks for picking it up!

> Rack Aware Stand-by Task Assignment for Kafka Streams
> -----------------------------------------------------
>
>                 Key: KAFKA-6718
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6718
>             Project: Kafka
>          Issue Type: New Feature
>          Components: streams
>            Reporter: Deepak Goyal
>            Priority: Major
>              Labels: needs-kip
>
> |Machines in data centre are sometimes grouped in racks. Racks provide 
> isolation as each rack may be in a different physical location and has its 
> own power source. When tasks are properly replicated across racks, it 
> provides fault tolerance in that if a rack goes down, the remaining racks can 
> continue to serve traffic.
>   
>  This feature is already implemented at Kafka 
> [KIP-36|https://cwiki.apache.org/confluence/display/KAFKA/KIP-36+Rack+aware+replica+assignment]
>  but we needed similar for task assignments at Kafka Streams Application 
> layer. 
>   
>  This features enables replica tasks to be assigned on different racks for 
> fault-tolerance.
>  NUM_STANDBY_REPLICAS = x
>  totalTasks = x+1 (replica + active)
>  # If there are no rackID provided: Cluster will behave rack-unaware
>  # If same rackId is given to all the nodes: Cluster will behave rack-unaware
>  # If (totalTasks <= number of racks), then Cluster will be rack aware i.e. 
> each replica task is each assigned to a different rack.
>  # Id (totalTasks > number of racks), then it will first assign tasks on 
> different racks, further tasks will be assigned to least loaded node, cluster 
> wide.|
> We have added another config in StreamsConfig called "RACK_ID_CONFIG" which 
> helps StickyPartitionAssignor to assign tasks in such a way that no two 
> replica tasks are on same rack if possible.
>  Post that it also helps to maintain stickyness with-in the rack.|



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to