[ 
https://issues.apache.org/jira/browse/IGNITE-20032?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Denis Chudov updated IGNITE-20032:
----------------------------------
    Description: 
Stopping the striped pool can happen earlier than the last RPC messafe is 
processed.
Just after the pool is stopped, its reference is reset to {{{}null{}}}:
{code:java|title=org.apache.ignite.raft.jraft.core.NodeImpl#join|borderStyle=solid}
if (opts.getStripedExecutor() != null && !opts.isSharedPools()) {
    opts.getStripedExecutor().shutdownGracefully();
    opts.setStripedExecutor(null);
 }
{code}
{noformat}
2023-07-21 18:45:10:571 +0300 
[ERROR][%int_tcpcat_5004%MessagingService-inbound--0][DefaultMessagingService] 
onMessage() failed while processing InvokeRequestImpl [correlationId=30, 
message=AppendEntriesRequestImpl [committedIndex=18558, 
data=org.apache.ignite.raft.jraft.util.ByteString@1, entriesList=null, 
groupId=unittest, peerId=int_tcpcat_5004, prevLogIndex=18559, prevLogTerm=1, 
serverId=int_tcpcat_5007, term=2, timestampLong=110752845696729088]] from 
int_tcpcat_5007
java.lang.NullPointerException
  at 
org.apache.ignite.raft.jraft.rpc.impl.core.AppendEntriesRequestProcessor.getOrCreatePeerRequestContext(AppendEntriesRequestProcessor.java:351)
  at 
org.apache.ignite.raft.jraft.rpc.impl.core.AppendEntriesRequestProcessor$PeerExecutorSelector.select(AppendEntriesRequestProcessor.java:72)
  at 
org.apache.ignite.raft.jraft.rpc.impl.IgniteRpcServer$RpcMessageHandler.onReceived(IgniteRpcServer.java:182)
  at 
org.apache.ignite.network.DefaultMessagingService.onMessage(DefaultMessagingService.java:375)
  at 
org.apache.ignite.network.DefaultMessagingService.lambda$onMessage$4(DefaultMessagingService.java:335)
  at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
  at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
  at java.base/java.lang.Thread.run(Thread.java:834)
{noformat}
*Implementation notes*

Something like busyLock can be introduced into this request processor. We 
should check what error is sent in the request if the raft node is fully 
stopped, and make the same behavior for cases when it is impossible to acquire 
bufy lock (when the node is being stopped).

  was:
Stopping the striped pool can happen earlier than the last RPC messafe is 
processed.
Just after the pool is stopped, its reference is reset to {{null}}:
{code:java|title=org.apache.ignite.raft.jraft.core.NodeImpl#join|borderStyle=solid}
if (opts.getStripedExecutor() != null && !opts.isSharedPools()) {
    opts.getStripedExecutor().shutdownGracefully();
    opts.setStripedExecutor(null);
 }
{code}
{noformat}
2023-07-21 18:45:10:571 +0300 
[ERROR][%int_tcpcat_5004%MessagingService-inbound--0][DefaultMessagingService] 
onMessage() failed while processing InvokeRequestImpl [correlationId=30, 
message=AppendEntriesRequestImpl [committedIndex=18558, 
data=org.apache.ignite.raft.jraft.util.ByteString@1, entriesList=null, 
groupId=unittest, peerId=int_tcpcat_5004, prevLogIndex=18559, prevLogTerm=1, 
serverId=int_tcpcat_5007, term=2, timestampLong=110752845696729088]] from 
int_tcpcat_5007
java.lang.NullPointerException
  at 
org.apache.ignite.raft.jraft.rpc.impl.core.AppendEntriesRequestProcessor.getOrCreatePeerRequestContext(AppendEntriesRequestProcessor.java:351)
  at 
org.apache.ignite.raft.jraft.rpc.impl.core.AppendEntriesRequestProcessor$PeerExecutorSelector.select(AppendEntriesRequestProcessor.java:72)
  at 
org.apache.ignite.raft.jraft.rpc.impl.IgniteRpcServer$RpcMessageHandler.onReceived(IgniteRpcServer.java:182)
  at 
org.apache.ignite.network.DefaultMessagingService.onMessage(DefaultMessagingService.java:375)
  at 
org.apache.ignite.network.DefaultMessagingService.lambda$onMessage$4(DefaultMessagingService.java:335)
  at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
  at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
  at java.base/java.lang.Thread.run(Thread.java:834)
{noformat}
 


> NPE on a try to get a RAFT node's striped executor pool
> -------------------------------------------------------
>
>                 Key: IGNITE-20032
>                 URL: https://issues.apache.org/jira/browse/IGNITE-20032
>             Project: Ignite
>          Issue Type: Bug
>            Reporter: Vladislav Pyatkov
>            Priority: Major
>              Labels: ignite-3
>
> Stopping the striped pool can happen earlier than the last RPC messafe is 
> processed.
> Just after the pool is stopped, its reference is reset to {{{}null{}}}:
> {code:java|title=org.apache.ignite.raft.jraft.core.NodeImpl#join|borderStyle=solid}
> if (opts.getStripedExecutor() != null && !opts.isSharedPools()) {
>     opts.getStripedExecutor().shutdownGracefully();
>     opts.setStripedExecutor(null);
>  }
> {code}
> {noformat}
> 2023-07-21 18:45:10:571 +0300 
> [ERROR][%int_tcpcat_5004%MessagingService-inbound--0][DefaultMessagingService]
>  onMessage() failed while processing InvokeRequestImpl [correlationId=30, 
> message=AppendEntriesRequestImpl [committedIndex=18558, 
> data=org.apache.ignite.raft.jraft.util.ByteString@1, entriesList=null, 
> groupId=unittest, peerId=int_tcpcat_5004, prevLogIndex=18559, prevLogTerm=1, 
> serverId=int_tcpcat_5007, term=2, timestampLong=110752845696729088]] from 
> int_tcpcat_5007
> java.lang.NullPointerException
>   at 
> org.apache.ignite.raft.jraft.rpc.impl.core.AppendEntriesRequestProcessor.getOrCreatePeerRequestContext(AppendEntriesRequestProcessor.java:351)
>   at 
> org.apache.ignite.raft.jraft.rpc.impl.core.AppendEntriesRequestProcessor$PeerExecutorSelector.select(AppendEntriesRequestProcessor.java:72)
>   at 
> org.apache.ignite.raft.jraft.rpc.impl.IgniteRpcServer$RpcMessageHandler.onReceived(IgniteRpcServer.java:182)
>   at 
> org.apache.ignite.network.DefaultMessagingService.onMessage(DefaultMessagingService.java:375)
>   at 
> org.apache.ignite.network.DefaultMessagingService.lambda$onMessage$4(DefaultMessagingService.java:335)
>   at 
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>   at 
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>   at java.base/java.lang.Thread.run(Thread.java:834)
> {noformat}
> *Implementation notes*
> Something like busyLock can be introduced into this request processor. We 
> should check what error is sent in the request if the raft node is fully 
> stopped, and make the same behavior for cases when it is impossible to 
> acquire bufy lock (when the node is being stopped).



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

Reply via email to