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

ASF GitHub Bot updated HDDS-4186:
---------------------------------
    Labels: pull-request-available  (was: )

> Adjust RetryPolicy of SCMConnectionManager for SCM/Recon
> --------------------------------------------------------
>
>                 Key: HDDS-4186
>                 URL: https://issues.apache.org/jira/browse/HDDS-4186
>             Project: Hadoop Distributed Data Store
>          Issue Type: Improvement
>            Reporter: Glen Geng
>            Assignee: Glen Geng
>            Priority: Blocker
>              Labels: pull-request-available
>
> Current RetryPolicy of Datanode for SCM is retryForeverWithFixedSleep:
> {code:java}
> RetryPolicy retryPolicy =
>  RetryPolicies.retryForeverWithFixedSleep(
>  1000, TimeUnit.MILLISECONDS);
> StorageContainerDatanodeProtocolPB rpcProxy = RPC.getProtocolProxy(
>     StorageContainerDatanodeProtocolPB.class, version,
>     address, UserGroupInformation.getCurrentUser(), hadoopConfig,
>     NetUtils.getDefaultSocketFactory(hadoopConfig), getRpcTimeout(),
>     retryPolicy).getProxy();{code}
>  that for Recon is retryUpToMaximumCountWithFixedSleep:
> {code:java}
> RetryPolicy retryPolicy =
>     RetryPolicies.retryUpToMaximumCountWithFixedSleep(10,
>         60000, TimeUnit.MILLISECONDS);
> ReconDatanodeProtocolPB rpcProxy = RPC.getProtocolProxy(
>     ReconDatanodeProtocolPB.class, version,
>     address, UserGroupInformation.getCurrentUser(), hadoopConfig,
>     NetUtils.getDefaultSocketFactory(hadoopConfig), getRpcTimeout(),
>     retryPolicy).getProxy();
> {code}
>  
> The executorService in DatanodeStateMachine is 
> Executors.newFixedThreadPool(...), whose default pool size is 2, one for 
> Recon, another for SCM.
>  
> When encounter rpc failure, call() of RegisterEndpointTask, 
> VersionEndpointTask, HeartbeatEndpointTask will retry while holding the 
> rpcEndpoint.lock(). For example:
> {code:java}
> public EndpointStateMachine.EndPointStates call() throws Exception {
>   rpcEndpoint.lock();
>   try {
>     ....
>     SCMHeartbeatResponseProto reponse = rpcEndpoint.getEndPoint()
>         .sendHeartbeat(request);
>     ....
>   } finally {
>     rpcEndpoint.unlock();
>   }
>   return rpcEndpoint.getState();
> }
> {code}
>  
> *The problem is:*
> If setup one Recon and one SCM, then shutdown the Recon server, all Datanodes 
> will be stale/dead very soon at SCM side.
>  
> *The root cause is:*
> The thread running Recon task will retry due to rpc failure, meanwhile holds 
> the lock of EndpointStateMachine for Recon. When DatanodeStateMachine 
> schedule the next round of SCM/Recon task, the only left thread will be 
> assigned to run Recon task, and blocked at waiting for the lock of 
> EndpointStateMachine for Recon.
> {code:java}
> public EndpointStateMachine.EndPointStates call() throws Exception {
>   rpcEndpoint.lock();
>   ...{code}
>  
> *The solution is:*
> Since DatanodeStateMachine will periodically schedule SCM/Recon tasks, we may 
> adjust RetryPolicy so that won't retry for longer that 1min. 
> The change has no side effect:
> 1) VersionEndpointTask.call() is fine
> 2) RegisterEndpointTask.call() will query containerReport, nodeReport, 
> pipelineReports from OzoneContainer, which is fine.
> 3) HeartbeatEndpointTask.call() will putBackReports(), which is fine.
>  



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

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: ozone-issues-h...@hadoop.apache.org

Reply via email to