[
https://issues.apache.org/jira/browse/RATIS-556?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16885631#comment-16885631
]
Ankit Singhal commented on RATIS-556:
-------------------------------------
{code}
Failed to read from log
org.apache.ratis.logservice.common.LogNotFoundException: 'testlog1'
at
org.apache.ratis.logservice.server.MetaStateMachine.processGetLogRequest(MetaStateMachine.java:382)
at
org.apache.ratis.logservice.server.MetaStateMachine.query(MetaStateMachine.java:213)
at
org.apache.ratis.server.impl.RaftServerImpl.submitClientRequestAsync(RaftServerImpl.java:547)
at
org.apache.ratis.server.impl.RaftServerProxy.lambda$submitClientRequestAsync$7(RaftServerProxy.java:333)
at
org.apache.ratis.server.impl.RaftServerProxy.lambda$null$5(RaftServerProxy.java:328)
at org.apache.ratis.util.JavaUtils.callAsUnchecked(JavaUtils.java:109)
at
org.apache.ratis.server.impl.RaftServerProxy.lambda$submitRequest$6(RaftServerProxy.java:328)
at
java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:981)
at
java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2124)
at
org.apache.ratis.server.impl.RaftServerProxy.submitRequest(RaftServerProxy.java:327)
at
org.apache.ratis.server.impl.RaftServerProxy.submitClientRequestAsync(RaftServerProxy.java:333)
at
org.apache.ratis.grpc.client.GrpcClientProtocolService$RequestStreamObserver.processClientRequest(GrpcClientProtocolService.java:220)
at
org.apache.ratis.grpc.client.GrpcClientProtocolService$UnorderedRequestStreamObserver.processClientRequest(GrpcClientProtocolService.java:276)
at
org.apache.ratis.grpc.client.GrpcClientProtocolService$RequestStreamObserver.onNext(GrpcClientProtocolService.java:240)
at
org.apache.ratis.grpc.client.GrpcClientProtocolService$RequestStreamObserver.onNext(GrpcClientProtocolService.java:168)
at
org.apache.ratis.thirdparty.io.grpc.stub.ServerCalls$StreamingServerCallHandler$StreamingServerCallListener.onMessage(ServerCalls.java:248)
at
org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailable(ServerCallImpl.java:263)
at
org.apache.ratis.thirdparty.io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1MessagesAvailable.runInContext(ServerImpl.java:686)
at
org.apache.ratis.thirdparty.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
at
org.apache.ratis.thirdparty.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:123)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
{code}
why LSClient is not also able to get logInfo from meta servers? As all meta
servers are readable , so if the log was created(and not deleted later) , it
should be present in the map maintained at metatStateMachine and altleast we
should get the peers and groupId(though subsequent call to access the log can
fail)
{quote} I think the idea was that we could get some listener callback in the
metadata quorum and force that log into a "CLOSED" state, so that no more
updates can be accepted by it.
{quote}
At meta quorum, we can utilize PINGREQUEST & timeout to detect server
failure(though it's not yet implemented at workers to report liveliness) and
try changing the state of the log to CLOSE and start archiving.
{quote}
However, if we force a CLOSED state via sending it a transaction which can't be
applied, maybe we have a bigger problem
{quote}
With sufficient no. of nodes (>= 2), I think we should be able to do it in a
transactional way.
> Detect node failures and add other workers to group serving the log and
> replicate the data of the log
> -----------------------------------------------------------------------------------------------------
>
> 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
>
> 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
(v7.6.14#76016)