You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by "Ivan Brusentsev (Jira)" <ji...@apache.org> on 2023/12/04 07:44:00 UTC

[jira] [Assigned] (HDDS-9404) SCM executes all pipeline operations under single lock

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

Ivan Brusentsev reassigned HDDS-9404:
-------------------------------------

    Assignee: Ivan Brusentsev

> SCM executes all pipeline operations under single lock
> ------------------------------------------------------
>
>                 Key: HDDS-9404
>                 URL: https://issues.apache.org/jira/browse/HDDS-9404
>             Project: Apache Ozone
>          Issue Type: Task
>            Reporter: Kirill Sizov
>            Assignee: Ivan Brusentsev
>            Priority: Major
>         Attachments: load.png, op_len.png
>
>
> This task came out of the performance testing. 
> *Cluster Setup*
> 12 nodes: 3 master, each running SCM and OM and 9 data nodes, each running DN and S3G.
> 2 CPUs x 24 cores each, having totally 96 vCPUs.
> Each node has 12 SSDs. And 256G RAM.
> The same behaviour was observed on a different setup, 3 master + 5 data nodes.
> *Load generators*
> Experiment 1:
> Running from 3 nodes, each spawned 25 threads. 
> Experiment 2:
> Running from 7 nodes, each spawned 50 threads. 
> Experiment 3:
> Running from 7 nodes, each spawned 100 threads.
>  
> Total load - uploading 100k files, each file is 20MB.
> *Ozone Setup*
> The writes were putting files into a single Erasure Coded bucket RS-6-3-1024k.
> The same was observed on RS-3-2-1024k.
> *Observations*
> We have seen periodic significant drops in the number of uploaded files per second to almost zero.
> No exceptions on the clients, eventually all files were successfully uploaded. 
> The same test was repeated multiple times and the same behaviour has been found in each test run.
> !load.png!
> *Investigation*
> We started the process of searching for the cause from OM. 
> Clearly we see the growing of the upload operation time.
> !op_len.png! 
> Threaddumps showed that OM threads were waiting for the requests to SCM to finish. 
> {noformat}
> "IPC Server handler 99 on default port 9862" Id=245 WAITING on org.apache.hadoop.ipc.Client$Call@6738ef64
> 	at java.lang.Object.wait(Native Method)
> 	-  waiting on org.apache.hadoop.ipc.Client$Call@6738ef64
> 	at java.lang.Object.wait(Object.java:502)
> 	at org.apache.hadoop.util.concurrent.AsyncGet$Util.wait(AsyncGet.java:65)
> 	at org.apache.hadoop.ipc.Client.getRpcResponse(Client.java:1555)
> 	at org.apache.hadoop.ipc.Client.call(Client.java:1513)
> 	at org.apache.hadoop.ipc.Client.call(Client.java:1410)
> 	at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:250)
> 	at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:132)
> 	at com.sun.proxy.$Proxy33.send(Unknown Source)
> 	at sun.reflect.GeneratedMethodAccessor27.invoke(Unknown Source)
> 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> 	at java.lang.reflect.Method.invoke(Method.java:498)
> 	at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:433)
> 	at org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeMethod(RetryInvocationHandler.java:166)
> 	at org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invoke(RetryInvocationHandler.java:158)
> 	at org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeOnce(RetryInvocationHandler.java:96)
> 	at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:362)
> 	at com.sun.proxy.$Proxy33.send(Unknown Source)
> 	at org.apache.hadoop.hdds.scm.protocolPB.ScmBlockLocationProtocolClientSideTranslatorPB.submitRequest(ScmBlockLocationProtocolClientSideTranslatorPB.java:121)
> 	at org.apache.hadoop.hdds.scm.protocolPB.ScmBlockLocationProtocolClientSideTranslatorPB.allocateBlock(ScmBlockLocationProtocolClientSideTranslatorPB.java:195)
> 	at org.apache.hadoop.ozone.om.request.key.OMKeyRequest.allocateBlock(OMKeyRequest.java:151)
> 	at org.apache.hadoop.ozone.om.request.file.OMFileCreateRequest.preExecute(OMFileCreateRequest.java:137)
> 	at org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.processRequest(OzoneManagerProtocolServerSideTranslatorPB.java:193)
> 	at org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB$$Lambda$754/1837556729.apply(Unknown Source)
> 	at org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher.processRequest(OzoneProtocolMessageDispatcher.java:87)
> 	at org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB.submitRequest(OzoneManagerProtocolServerSideTranslatorPB.java:148)
> 	at org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos$OzoneManagerService$2.callBlockingMethod(OzoneManagerProtocolProtos.java)
> 	at org.apache.hadoop.ipc.ProtobufRpcEngine$Server.processCall(ProtobufRpcEngine.java:484)
> 	at org.apache.hadoop.ipc.ProtobufRpcEngine2$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine2.java:595)
> 	at org.apache.hadoop.ipc.ProtobufRpcEngine2$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine2.java:573)
> 	at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1227)
> 	at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:1094)
> 	at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:1017)
> 	at java.security.AccessController.doPrivileged(Native Method)
> 	at javax.security.auth.Subject.doAs(Subject.java:422)
> 	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1899)
> 	at org.apache.hadoop.ipc.Server$Handler.run(Server.java:3048)
>  {noformat}
>  
> So I switched over to the SCM thread dumps for the same timestamp.
> Here is what the threads were doing:
> {noformat}
> "IPC Server handler 75 on default port 9863" Id=254 TIMED_WAITING on java.util.concurrent.CompletableFuture$Signaller@1c9b12a7
> 	at sun.misc.Unsafe.park(Native Method)
> 	-  waiting on java.util.concurrent.CompletableFuture$Signaller@1c9b12a7
> 	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
> 	at java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1709)
> 	at java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
> 	at java.util.concurrent.CompletableFuture.timedGet(CompletableFuture.java:1788)
> 	at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1928)
> 	at org.apache.hadoop.hdds.scm.ha.SCMRatisServerImpl.submitRequest(SCMRatisServerImpl.java:229)
> 	at org.apache.hadoop.hdds.scm.ha.SCMHAInvocationHandler.invokeRatisServer(SCMHAInvocationHandler.java:123)
> 	at org.apache.hadoop.hdds.scm.ha.SCMHAInvocationHandler.invokeRatis(SCMHAInvocationHandler.java:112)
> 	at org.apache.hadoop.hdds.scm.ha.SCMHAInvocationHandler.invoke(SCMHAInvocationHandler.java:74)
> 	at com.sun.proxy.$Proxy18.updatePipelineState(Unknown Source)
> 	at org.apache.hadoop.hdds.scm.pipeline.PipelineManagerImpl.closePipeline(PipelineManagerImpl.java:505)
> 	at org.apache.hadoop.hdds.scm.pipeline.WritableECContainerProvider.getContainer(WritableECContainerProvider.java:141)
> 	at org.apache.hadoop.hdds.scm.pipeline.WritableECContainerProvider.getContainer(WritableECContainerProvider.java:52)
> 	at org.apache.hadoop.hdds.scm.pipeline.WritableContainerFactory.getContainer(WritableContainerFactory.java:76)
> 	at org.apache.hadoop.hdds.scm.block.BlockManagerImpl.allocateBlock(BlockManagerImpl.java:164)
> 	at org.apache.hadoop.hdds.scm.server.SCMBlockProtocolServer.allocateBlock(SCMBlockProtocolServer.java:194)
> 	at org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocolServerSideTranslatorPB.allocateScmBlock(ScmBlockLocationProtocolServerSideTranslatorPB.java:192)
> 	at org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocolServerSideTranslatorPB.processMessage(ScmBlockLocationProtocolServerSideTranslatorPB.java:142)
> 	at org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocolServerSideTranslatorPB$$Lambda$717/605372910.apply(Unknown Source)
> 	at org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher.processRequest(OzoneProtocolMessageDispatcher.java:87)
> 	at org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocolServerSideTranslatorPB.send(ScmBlockLocationProtocolServerSideTranslatorPB.java:113)
> 	at org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos$ScmBlockLocationProtocolService$2.callBlockingMethod(ScmBlockLocationProtocolProtos.java:14238)
> 	at org.apache.hadoop.ipc.ProtobufRpcEngine$Server.processCall(ProtobufRpcEngine.java:484)
> 	at org.apache.hadoop.ipc.ProtobufRpcEngine2$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine2.java:595)
> 	at org.apache.hadoop.ipc.ProtobufRpcEngine2$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine2.java:573)
> 	at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1227)
> 	at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:1094)
> 	at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:1017)
> 	at java.security.AccessController.doPrivileged(Native Method)
> 	at javax.security.auth.Subject.doAs(Subject.java:422)
> 	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1899)
> 	at org.apache.hadoop.ipc.Server$Handler.run(Server.java:3048)
> "IPC Server handler 82 on default port 9863" Id=261 WAITING on java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync@735fd35a owned by "IPC Server handler 75 on default port 9863" Id=254
> 	at sun.misc.Unsafe.park(Native Method)
> 	-  waiting on java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync@735fd35a
> 	at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
> 	at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
> 	at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:870)
> 	at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1199)
> 	at java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.lock(ReentrantReadWriteLock.java:943)
> 	at org.apache.hadoop.hdds.scm.pipeline.PipelineManagerImpl.acquireWriteLock(PipelineManagerImpl.java:924)
> 	at org.apache.hadoop.hdds.scm.pipeline.PipelineManagerImpl.closePipeline(PipelineManagerImpl.java:503)
> 	at org.apache.hadoop.hdds.scm.pipeline.WritableECContainerProvider.getContainer(WritableECContainerProvider.java:141)
> 	at org.apache.hadoop.hdds.scm.pipeline.WritableECContainerProvider.getContainer(WritableECContainerProvider.java:52)
> 	at org.apache.hadoop.hdds.scm.pipeline.WritableContainerFactory.getContainer(WritableContainerFactory.java:76)
> 	at org.apache.hadoop.hdds.scm.block.BlockManagerImpl.allocateBlock(BlockManagerImpl.java:164)
> 	at org.apache.hadoop.hdds.scm.server.SCMBlockProtocolServer.allocateBlock(SCMBlockProtocolServer.java:194)
> 	at org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocolServerSideTranslatorPB.allocateScmBlock(ScmBlockLocationProtocolServerSideTranslatorPB.java:192)
> 	at org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocolServerSideTranslatorPB.processMessage(ScmBlockLocationProtocolServerSideTranslatorPB.java:142)
> 	at org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocolServerSideTranslatorPB$$Lambda$717/605372910.apply(Unknown Source)
> 	at org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher.processRequest(OzoneProtocolMessageDispatcher.java:87)
> 	at org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocolServerSideTranslatorPB.send(ScmBlockLocationProtocolServerSideTranslatorPB.java:113)
> 	at org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos$ScmBlockLocationProtocolService$2.callBlockingMethod(ScmBlockLocationProtocolProtos.java:14238)
> 	at org.apache.hadoop.ipc.ProtobufRpcEngine$Server.processCall(ProtobufRpcEngine.java:484)
> 	at org.apache.hadoop.ipc.ProtobufRpcEngine2$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine2.java:595)
> 	at org.apache.hadoop.ipc.ProtobufRpcEngine2$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine2.java:573)
> 	at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1227)
> 	at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:1094)
> 	at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:1017)
> 	at java.security.AccessController.doPrivileged(Native Method)
> 	at javax.security.auth.Subject.doAs(Subject.java:422)
> 	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1899)
> 	at org.apache.hadoop.ipc.Server$Handler.run(Server.java:3048)
>  {noformat}
>  
> *The cause*
> So what's happening?
> We see that {{allocateBlock}} method looks for the open containers (pipelines) that have enough space to allocate new block of data. If the open container ran out of space, it triggers container close operation.
> The process of closing a container includes
>  # negotiate the close operation with the rest of the SCM raft group
>  # acquire a lock to change the state of PipelineManagerImpl.
> The issue here is that regardless of the number of the clients (7 nodes times 100 threads from the experiment 3 resulted in 700 concurrent request in the worst case), all threads a blocked by a single lock. 
> Moreover, the performance of 7*100 was not significantly different from 7*50. 7*100 just had longer gaps.
> So this is the bottleneck that limits cluster scalability.
> We need to replace the single-lock-per-cluster approach with the concurrent code.



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

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