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

Glen Geng updated HDDS-4186:
----------------------------
    Description: 
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}
 

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 now 
Executors.newFixedThreadPool(...), whose 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().

Here is the problem: if setup one Recon and one SCM, then shutdown the Recon 
server, all Datanodes will be stale/dead very soon. The root cause is that, the 
thread working for Recon will retry while holding the lock of 
EndpointStateMachine for Recon, when DatanodeStateMachine schedule the next 
round of task, the other thread is blocked by waiting for the lock of 
EndpointStateMachine for Recon.

 

Since DatanodeStateMachine will periodically schedule tasks, we may adjust 
RetryPolicy so that the execution of tasks no need to be longer than 1min.

 

  was:
Teragen reported to be slow with low number of mappers compared to HDFS.

In my test (one pipeline, 3 yarn nodes) 10 g teragen with HDFS was ~3 mins but 
with Ozone it was 6 mins. It could be fixed with using more mappers, but when I 
investigated the execution I found a few problems reagrding to the BufferPool 
management.

 1. IncrementalChunkBuffer is slow and it might not be required as BufferPool 
itself is incremental
 2. For each write operation the bufferPool.allocateBufferIfNeeded is called 
which can be a slow operation (positions should be calculated).
 3. There is no explicit support for write(byte) operations

In the flamegraph it's clearly visible that with low number of mappers the 
client is busy with buffer operations. After the patch the rpc call and the 
checksum calculation give the majority of the time. 


> Adjust RetryPolicy of SCMConnectionManager
> ------------------------------------------
>
>                 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
>
> 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}
>  
> 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 now 
> Executors.newFixedThreadPool(...), whose 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().
> Here is the problem: if setup one Recon and one SCM, then shutdown the Recon 
> server, all Datanodes will be stale/dead very soon. The root cause is that, 
> the thread working for Recon will retry while holding the lock of 
> EndpointStateMachine for Recon, when DatanodeStateMachine schedule the next 
> round of task, the other thread is blocked by waiting for the lock of 
> EndpointStateMachine for Recon.
>  
> Since DatanodeStateMachine will periodically schedule tasks, we may adjust 
> RetryPolicy so that the execution of tasks no need to be longer than 1min.
>  



--
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