[
https://issues.apache.org/jira/browse/RATIS-556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16914643#comment-16914643
]
Josh Elser commented on RATIS-556:
----------------------------------
{quote}can't we use the heartbeat mechanism like every peer will be sending
heartbeat request regularly to meta quorum using a separate thread?
{quote}
I was looking at this (also with a mind to RATIS-651), but I'm not seeing good
extension points in LeaderState/RaftServerImpl/LogAppender to actually hook
into the heartbeat messages that the Leader gets back (aka the
AppendEntriesReply that don't have any entries in them). I agree that
piggy-backing on the existing heartbeat for cluster membership is good, but I
don't think it's ready for us to use it well (i think that would have to be
something we design first, and then try to use in the LogService).
{quote}If we just create an inverted index then a log will be served by 3 peers
and when any of the peers goes down we need to close the log
{quote}
But, I thought that was the plan – to close the log when we lose a peer.
{quote}if we take HBase use case a log will be created by a server and from the
logname, we can detect the server but such functionality cannot be done in this
generic log service
{quote}
In the HBase case, HBase would be acting as a client to the LogService (as well
as a server in the LogService). The client API shouldn't be changing to support
this feature – we want a Log to be closed automatically
Instead of the client passing in a specific RaftPeer that we use in
{{REGISTERREQUEST}}, we should be pulling apart the {{RaftGroup}} to get all of
the {{RaftPeers}} from there and add an entry to {{peerLogs}} for each of those
peers. We need to do this because even if the leader of the RaftGroup for a Log
dies, we still want to close that Log.
Does that make sense, Rajeshbabu? I can try to pseudocode if helpful.
> Detect node failures and close the log to prevent additional writes
> -------------------------------------------------------------------
>
> Key: RATIS-556
> URL: https://issues.apache.org/jira/browse/RATIS-556
> Project: Ratis
> Issue Type: Improvement
> Reporter: Rajeshbabu Chintaguntla
> Assignee: Rajeshbabu Chintaguntla
> Priority: Major
> Attachments: RATIS-556-wip.patch, RATIS-556_v1.patch
>
>
> Currently there is no way to detect the node failures at master log servers
> and add new nodes to the group serving the log. We need to analyze how Ozone
> is working in this case.
--
This message was sent by Atlassian Jira
(v8.3.2#803003)