[
https://issues.apache.org/jira/browse/HDDS-5703?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Mark Gui updated HDDS-5703:
---------------------------
Description:
When we use the SCM benchmark tool
(https://issues.apache.org/jira/browse/HDDS-5702) to test SCM throughput for
AllocateContainer, we found a dramatic degradation in the throughput when one
scm peer(follower) is down.
Here is some statistics.
cmd:
{code:java}
./bin/ozone freon stb \
--benchmark AllocateContainers \
--scmHost 9.44.15.145 \
--num-containers 500000 \
--num-datanodes 50 \
--num-threads 50{code}
Normal Case:
{code:java}
***************************************
Total allocated containers: 500000
Total failed containers: 0
Execution Time: 02:36:50,151
Throughput: 53.000000 (ops)
***************************************
{code}
One scm follower down:
{code:java}
***************************************
Total allocated containers: 50000
Total failed containers: 0
Execution Time: 02:22:00,245
Throughput: 5.000000 (ops)
***************************************
{code}
The overall throughput drops to 1/10 of the original.
We have a dig into this problem.
There are 2 flame graphs captured by an open source tool (arthas) before and
after the scm follower down in the attachments
[^20210830-183345.svg]
[^20210830-184833.svg].
We could see that the GrpcLogAppender related to the down scm follower is
consuming the most cpu busy reading segment files from the disk.
At the same time we could see that the metrics
`ratis_log_worker_cacheMissCount` and `ratis_grpc_log_appender_num_retries` are
increasing.
So a down scm follower leads to segmented raft log cache misses, and
GrpcLogAppender got to load cache from the disk.
But actually we are reading disk files with an intrinsic lock held:
{code:java}
// GrpcLogAppender.java
private void appendLog(boolean excludeLogEntries) throws IOException {
final AppendEntriesRequestProto pending;
final AppendEntriesRequest request;
final StreamObserver<AppendEntriesRequestProto> s;
synchronized (this) {
<-- will block others
// prepare and enqueue the append request. note changes on follower's
// nextIndex and ops on pendingRequests should always be associated
// together and protected by the lock
pending = newAppendEntriesRequest(callId++, excludeLogEntries);
<-- segment file read inside
if (pending == null) {
return;
}
request = new AppendEntriesRequest(pending, getFollowerId(),
grpcServerMetrics);
pendingRequests.put(request);
increaseNextIndex(pending);
if (appendLogRequestObserver == null) {
appendLogRequestObserver = getClient().appendEntries(new
AppendLogResponseHandler());
}
s = appendLogRequestObserver;
}
if (isRunning()) {
sendRequest(request, pending, s);
}
}
{code}
Unluckily, when RaftServer submit a client request, it will notify the
GrpcLogAppender under the same lock:
{code:java}
// LogAppender.java
default void notifyLogAppender() {
synchronized (this) { <-- blocked by others
notify();
}
}{code}
So finally the slow disk file reading operation blocks the client request
submission.
This could be detected with arthas as follows:
{code:java}
[arthas@85790]$ thread -b
"IPC Server handler 112 on default port 9860" Id=168 BLOCKED on
org.apache.ratis.grpc.server.GrpcLogAppender@68e40b62 owned by
"fc19acc0-d5cb-4e64-a1b3-a107fe38d6a7@group-F43758AD903B->d1fa1468-5d66-4c17-be19-493b372ee241-GrpcLogAppender-LogAppenderDaemon"
Id=691
at
org.apache.ratis.server.leader.LogAppender.notifyLogAppender(LogAppender.java:133)
- blocked on org.apache.ratis.grpc.server.GrpcLogAppender@68e40b62
at
org.apache.ratis.server.impl.LeaderStateImpl$$Lambda$421/1164237000.accept(Unknown
Source)
at
java.util.concurrent.CopyOnWriteArrayList.forEach(CopyOnWriteArrayList.java:891)
at
org.apache.ratis.server.impl.LeaderStateImpl$SenderList.forEach(LeaderStateImpl.java:203)
at
org.apache.ratis.server.impl.LeaderStateImpl.notifySenders(LeaderStateImpl.java:321)
at
org.apache.ratis.server.impl.RaftServerImpl.appendTransaction(RaftServerImpl.java:723)
- locked org.apache.ratis.server.impl.RaftServerImpl@4a0e7199
at
org.apache.ratis.server.impl.RaftServerImpl.submitClientRequestAsync(RaftServerImpl.java:800)
at
org.apache.ratis.server.impl.RaftServerProxy.lambda$submitClientRequestAsync$9(RaftServerProxy.java:417)
at
org.apache.ratis.server.impl.RaftServerProxy$$Lambda$525/1313234420.apply(Unknown
Source)
at
org.apache.ratis.server.impl.RaftServerProxy.lambda$null$7(RaftServerProxy.java:412)
at
org.apache.ratis.server.impl.RaftServerProxy$$Lambda$527/34265965.get(Unknown
Source)
at org.apache.ratis.util.JavaUtils.callAsUnchecked(JavaUtils.java:115)
at
org.apache.ratis.server.impl.RaftServerProxy.lambda$submitRequest$8(RaftServerProxy.java:412)
at
org.apache.ratis.server.impl.RaftServerProxy$$Lambda$526/1858964023.apply(Unknown
Source)
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:411)
at
org.apache.ratis.server.impl.RaftServerProxy.submitClientRequestAsync(RaftServerProxy.java:417)
at
org.apache.hadoop.hdds.scm.ha.SCMRatisServerImpl.submitRequest(SCMRatisServerImpl.java:212)
at
org.apache.hadoop.hdds.scm.ha.SCMHAInvocationHandler.invokeRatis(SCMHAInvocationHandler.java:110)
at
org.apache.hadoop.hdds.scm.ha.SCMHAInvocationHandler.invoke(SCMHAInvocationHandler.java:67)
at com.sun.proxy.$Proxy15.addContainer(Unknown Source)
at
org.apache.hadoop.hdds.scm.container.ContainerManagerImpl.allocateContainer(ContainerManagerImpl.java:229)
at
org.apache.hadoop.hdds.scm.container.ContainerManagerImpl.allocateContainer(ContainerManagerImpl.java:198)
at
org.apache.hadoop.hdds.scm.server.SCMClientProtocolServer.allocateContainer(SCMClientProtocolServer.java:201)
at
org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocolServerSideTranslatorPB.allocateContainer(StorageContainerLocationProtocolServerSideTranslatorPB.java:414)
at
org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocolServerSideTranslatorPB.processRequest(StorageContainerLocationProtocolServerSideTranslatorPB.java:182)
at
org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocolServerSideTranslatorPB$$Lambda$582/601306741.apply(Unknown
Source)
at
org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher.processRequest(OzoneProtocolMessageDispatcher.java:87)
at
org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocolServerSideTranslatorPB.submitRequest(StorageContainerLocationProtocolServerSideTranslatorPB.java:169)
at
org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos$StorageContainerLocationProtocolService$2.callBlockingMethod(StorageContainerLocationProtocolProtos.java:55800)
at
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:529)
at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1073)
at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:1024)
at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:948)
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:2002)
at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2993) Number of
locked synchronizers = 1
- java.util.concurrent.locks.ReentrantLock$NonfairSync@18fbf528 <---- but
blocks 49 other threads!
{code}
Actually there are several problems here in general:
1.The LogAppender should not do heavy works(disk file reading) with a lock held
for somewhat meaningless job, we could known that the follower has dead and
prevent it from happening.
2.The client handler thread should not be blocked by the LogAppender, there
should be more light-weight way to do a notify of new jobs, such as a classic
producer-consumer pattern with queues.
This should be a ratis side problem according to the trace.
was:
When we use the SCM benchmark tool
(https://issues.apache.org/jira/browse/HDDS-5702) to test SCM throughput for
AllocateContainer, we found a dramatic degradation in the throughput when one
scm peer(follower) is down.
Here is some statistics.
cmd:
{code:java}
./bin/ozone freon stb \
--benchmark AllocateContainers \
--scmHost 9.44.15.145 \
--num-containers 500000 \
--num-datanodes 50 \
--num-threads 50{code}
Normal Case:
{code:java}
***************************************
Total allocated containers: 500000
Total failed containers: 0
Execution Time: 02:36:50,151
Throughput: 53.000000 (ops)
***************************************
{code}
One scm follower down:
{code:java}
***************************************
Total allocated containers: 50000
Total failed containers: 0
Execution Time: 02:22:00,245
Throughput: 5.000000 (ops)
***************************************
{code}
The overall throughput drops to 1/10 of the original.
We have a dig into this problem.
There are 2 flame graphs captured by an open source tool (arthas) before and
after the scm follower down in the attachments
[^20210830-183345.svg]
[^20210830-184833.svg].
We could see that the GrpcLogAppender related to the down scm follower is
consuming the most cpu busy reading segment files from the disk.
At the same time we could see that the metrics
`ratis_log_worker_cacheMissCount` and `ratis_grpc_log_appender_num_retries` are
increasing.
So a down scm follower leads to segmented raft log cache misses, and
GrpcLogAppender got to load cache from the disk.
But actually we are reading disk files with an intrinsic lock held:
{code:java}
// GrpcLogAppender.java
private void appendLog(boolean excludeLogEntries) throws IOException {
final AppendEntriesRequestProto pending;
final AppendEntriesRequest request;
final StreamObserver<AppendEntriesRequestProto> s;
synchronized (this) {
<-- will block others
// prepare and enqueue the append request. note changes on follower's
// nextIndex and ops on pendingRequests should always be associated
// together and protected by the lock
pending = newAppendEntriesRequest(callId++, excludeLogEntries);
<-- segment file read inside
if (pending == null) {
return;
}
request = new AppendEntriesRequest(pending, getFollowerId(),
grpcServerMetrics);
pendingRequests.put(request);
increaseNextIndex(pending);
if (appendLogRequestObserver == null) {
appendLogRequestObserver = getClient().appendEntries(new
AppendLogResponseHandler());
}
s = appendLogRequestObserver;
}
if (isRunning()) {
sendRequest(request, pending, s);
}
}
{code}
Unluckily, when RaftServer submit a client request, it will notify the
GrpcLogAppender under the same lock:
{code:java}
// LogAppender.java
default void notifyLogAppender() {
synchronized (this) { <-- blocked by others
notify();
}
}{code}
So finally the slow disk file reading operation blocks the client request
submission.
This could be detected with arthas as follows:
{code:java}
[arthas@85790]$ thread -b
"IPC Server handler 112 on default port 9860" Id=168 BLOCKED on
org.apache.ratis.grpc.server.GrpcLogAppender@68e40b62 owned by
"fc19acc0-d5cb-4e64-a1b3-a107fe38d6a7@group-F43758AD903B->d1fa1468-5d66-4c17-be19-493b372ee241-GrpcLogAppender-LogAppenderDaemon"
Id=691
at
org.apache.ratis.server.leader.LogAppender.notifyLogAppender(LogAppender.java:133)
- blocked on org.apache.ratis.grpc.server.GrpcLogAppender@68e40b62
at
org.apache.ratis.server.impl.LeaderStateImpl$$Lambda$421/1164237000.accept(Unknown
Source)
at
java.util.concurrent.CopyOnWriteArrayList.forEach(CopyOnWriteArrayList.java:891)
at
org.apache.ratis.server.impl.LeaderStateImpl$SenderList.forEach(LeaderStateImpl.java:203)
at
org.apache.ratis.server.impl.LeaderStateImpl.notifySenders(LeaderStateImpl.java:321)
at
org.apache.ratis.server.impl.RaftServerImpl.appendTransaction(RaftServerImpl.java:723)
- locked org.apache.ratis.server.impl.RaftServerImpl@4a0e7199
at
org.apache.ratis.server.impl.RaftServerImpl.submitClientRequestAsync(RaftServerImpl.java:800)
at
org.apache.ratis.server.impl.RaftServerProxy.lambda$submitClientRequestAsync$9(RaftServerProxy.java:417)
at
org.apache.ratis.server.impl.RaftServerProxy$$Lambda$525/1313234420.apply(Unknown
Source)
at
org.apache.ratis.server.impl.RaftServerProxy.lambda$null$7(RaftServerProxy.java:412)
at
org.apache.ratis.server.impl.RaftServerProxy$$Lambda$527/34265965.get(Unknown
Source)
at org.apache.ratis.util.JavaUtils.callAsUnchecked(JavaUtils.java:115)
at
org.apache.ratis.server.impl.RaftServerProxy.lambda$submitRequest$8(RaftServerProxy.java:412)
at
org.apache.ratis.server.impl.RaftServerProxy$$Lambda$526/1858964023.apply(Unknown
Source)
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:411)
at
org.apache.ratis.server.impl.RaftServerProxy.submitClientRequestAsync(RaftServerProxy.java:417)
at
org.apache.hadoop.hdds.scm.ha.SCMRatisServerImpl.submitRequest(SCMRatisServerImpl.java:212)
at
org.apache.hadoop.hdds.scm.ha.SCMHAInvocationHandler.invokeRatis(SCMHAInvocationHandler.java:110)
at
org.apache.hadoop.hdds.scm.ha.SCMHAInvocationHandler.invoke(SCMHAInvocationHandler.java:67)
at com.sun.proxy.$Proxy15.addContainer(Unknown Source)
at
org.apache.hadoop.hdds.scm.container.ContainerManagerImpl.allocateContainer(ContainerManagerImpl.java:229)
at
org.apache.hadoop.hdds.scm.container.ContainerManagerImpl.allocateContainer(ContainerManagerImpl.java:198)
at
org.apache.hadoop.hdds.scm.server.SCMClientProtocolServer.allocateContainer(SCMClientProtocolServer.java:201)
at
org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocolServerSideTranslatorPB.allocateContainer(StorageContainerLocationProtocolServerSideTranslatorPB.java:414)
at
org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocolServerSideTranslatorPB.processRequest(StorageContainerLocationProtocolServerSideTranslatorPB.java:182)
at
org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocolServerSideTranslatorPB$$Lambda$582/601306741.apply(Unknown
Source)
at
org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher.processRequest(OzoneProtocolMessageDispatcher.java:87)
at
org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocolServerSideTranslatorPB.submitRequest(StorageContainerLocationProtocolServerSideTranslatorPB.java:169)
at
org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos$StorageContainerLocationProtocolService$2.callBlockingMethod(StorageContainerLocationProtocolProtos.java:55800)
at
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:529)
at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1073)
at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:1024)
at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:948)
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:2002)
at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2993) Number of
locked synchronizers = 1
- java.util.concurrent.locks.ReentrantLock$NonfairSync@18fbf528 <---- but
blocks 49 other threads!
{code}
Actually there are several problems here in general:
1.The LogAppender should not do heavy works(disk file reading) with a lock held
for somewhat meaningless job, we could known that the follower has dead and
prevent it from happening.
2.The client handler thread should not be blocked by the LogAppender, there
should be more light-weight way to do a notify of new jobs, such as a classic
producer-consumer pattern with queues.
> SCM HA performance degradation upon one peer down.
> --------------------------------------------------
>
> Key: HDDS-5703
> URL: https://issues.apache.org/jira/browse/HDDS-5703
> Project: Apache Ozone
> Issue Type: Bug
> Reporter: Mark Gui
> Priority: Major
> Attachments: 20210830-183345.svg, 20210830-184833.svg
>
>
> When we use the SCM benchmark tool
> (https://issues.apache.org/jira/browse/HDDS-5702) to test SCM throughput for
> AllocateContainer, we found a dramatic degradation in the throughput when one
> scm peer(follower) is down.
> Here is some statistics.
> cmd:
> {code:java}
> ./bin/ozone freon stb \
> --benchmark AllocateContainers \
> --scmHost 9.44.15.145 \
> --num-containers 500000 \
> --num-datanodes 50 \
> --num-threads 50{code}
> Normal Case:
> {code:java}
> ***************************************
> Total allocated containers: 500000
> Total failed containers: 0
> Execution Time: 02:36:50,151
> Throughput: 53.000000 (ops)
> ***************************************
> {code}
> One scm follower down:
> {code:java}
> ***************************************
> Total allocated containers: 50000
> Total failed containers: 0
> Execution Time: 02:22:00,245
> Throughput: 5.000000 (ops)
> ***************************************
> {code}
> The overall throughput drops to 1/10 of the original.
>
> We have a dig into this problem.
> There are 2 flame graphs captured by an open source tool (arthas) before and
> after the scm follower down in the attachments
> [^20210830-183345.svg]
> [^20210830-184833.svg].
> We could see that the GrpcLogAppender related to the down scm follower is
> consuming the most cpu busy reading segment files from the disk.
> At the same time we could see that the metrics
> `ratis_log_worker_cacheMissCount` and `ratis_grpc_log_appender_num_retries`
> are increasing.
> So a down scm follower leads to segmented raft log cache misses, and
> GrpcLogAppender got to load cache from the disk.
> But actually we are reading disk files with an intrinsic lock held:
> {code:java}
> // GrpcLogAppender.java
> private void appendLog(boolean excludeLogEntries) throws IOException {
> final AppendEntriesRequestProto pending;
> final AppendEntriesRequest request;
> final StreamObserver<AppendEntriesRequestProto> s;
> synchronized (this) {
> <-- will block others
> // prepare and enqueue the append request. note changes on follower's
> // nextIndex and ops on pendingRequests should always be associated
> // together and protected by the lock
> pending = newAppendEntriesRequest(callId++, excludeLogEntries);
> <-- segment file read inside
> if (pending == null) {
> return;
> }
> request = new AppendEntriesRequest(pending, getFollowerId(),
> grpcServerMetrics);
> pendingRequests.put(request);
> increaseNextIndex(pending);
> if (appendLogRequestObserver == null) {
> appendLogRequestObserver = getClient().appendEntries(new
> AppendLogResponseHandler());
> }
> s = appendLogRequestObserver;
> }
> if (isRunning()) {
> sendRequest(request, pending, s);
> }
> }
> {code}
> Unluckily, when RaftServer submit a client request, it will notify the
> GrpcLogAppender under the same lock:
> {code:java}
> // LogAppender.java
> default void notifyLogAppender() {
> synchronized (this) { <-- blocked by others
> notify();
> }
> }{code}
> So finally the slow disk file reading operation blocks the client request
> submission.
> This could be detected with arthas as follows:
> {code:java}
> [arthas@85790]$ thread -b
> "IPC Server handler 112 on default port 9860" Id=168 BLOCKED on
> org.apache.ratis.grpc.server.GrpcLogAppender@68e40b62 owned by
> "fc19acc0-d5cb-4e64-a1b3-a107fe38d6a7@group-F43758AD903B->d1fa1468-5d66-4c17-be19-493b372ee241-GrpcLogAppender-LogAppenderDaemon"
> Id=691
> at
> org.apache.ratis.server.leader.LogAppender.notifyLogAppender(LogAppender.java:133)
> - blocked on org.apache.ratis.grpc.server.GrpcLogAppender@68e40b62
> at
> org.apache.ratis.server.impl.LeaderStateImpl$$Lambda$421/1164237000.accept(Unknown
> Source)
> at
> java.util.concurrent.CopyOnWriteArrayList.forEach(CopyOnWriteArrayList.java:891)
> at
> org.apache.ratis.server.impl.LeaderStateImpl$SenderList.forEach(LeaderStateImpl.java:203)
> at
> org.apache.ratis.server.impl.LeaderStateImpl.notifySenders(LeaderStateImpl.java:321)
> at
> org.apache.ratis.server.impl.RaftServerImpl.appendTransaction(RaftServerImpl.java:723)
> - locked org.apache.ratis.server.impl.RaftServerImpl@4a0e7199
> at
> org.apache.ratis.server.impl.RaftServerImpl.submitClientRequestAsync(RaftServerImpl.java:800)
> at
> org.apache.ratis.server.impl.RaftServerProxy.lambda$submitClientRequestAsync$9(RaftServerProxy.java:417)
> at
> org.apache.ratis.server.impl.RaftServerProxy$$Lambda$525/1313234420.apply(Unknown
> Source)
> at
> org.apache.ratis.server.impl.RaftServerProxy.lambda$null$7(RaftServerProxy.java:412)
> at
> org.apache.ratis.server.impl.RaftServerProxy$$Lambda$527/34265965.get(Unknown
> Source)
> at org.apache.ratis.util.JavaUtils.callAsUnchecked(JavaUtils.java:115)
> at
> org.apache.ratis.server.impl.RaftServerProxy.lambda$submitRequest$8(RaftServerProxy.java:412)
> at
> org.apache.ratis.server.impl.RaftServerProxy$$Lambda$526/1858964023.apply(Unknown
> Source)
> 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:411)
> at
> org.apache.ratis.server.impl.RaftServerProxy.submitClientRequestAsync(RaftServerProxy.java:417)
> at
> org.apache.hadoop.hdds.scm.ha.SCMRatisServerImpl.submitRequest(SCMRatisServerImpl.java:212)
> at
> org.apache.hadoop.hdds.scm.ha.SCMHAInvocationHandler.invokeRatis(SCMHAInvocationHandler.java:110)
> at
> org.apache.hadoop.hdds.scm.ha.SCMHAInvocationHandler.invoke(SCMHAInvocationHandler.java:67)
> at com.sun.proxy.$Proxy15.addContainer(Unknown Source)
> at
> org.apache.hadoop.hdds.scm.container.ContainerManagerImpl.allocateContainer(ContainerManagerImpl.java:229)
> at
> org.apache.hadoop.hdds.scm.container.ContainerManagerImpl.allocateContainer(ContainerManagerImpl.java:198)
> at
> org.apache.hadoop.hdds.scm.server.SCMClientProtocolServer.allocateContainer(SCMClientProtocolServer.java:201)
> at
> org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocolServerSideTranslatorPB.allocateContainer(StorageContainerLocationProtocolServerSideTranslatorPB.java:414)
> at
> org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocolServerSideTranslatorPB.processRequest(StorageContainerLocationProtocolServerSideTranslatorPB.java:182)
> at
> org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocolServerSideTranslatorPB$$Lambda$582/601306741.apply(Unknown
> Source)
> at
> org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher.processRequest(OzoneProtocolMessageDispatcher.java:87)
> at
> org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocolServerSideTranslatorPB.submitRequest(StorageContainerLocationProtocolServerSideTranslatorPB.java:169)
> at
> org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos$StorageContainerLocationProtocolService$2.callBlockingMethod(StorageContainerLocationProtocolProtos.java:55800)
> at
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:529)
> at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1073)
> at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:1024)
> at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:948)
> 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:2002)
> at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2993) Number
> of locked synchronizers = 1
> - java.util.concurrent.locks.ReentrantLock$NonfairSync@18fbf528 <---- but
> blocks 49 other threads!
> {code}
>
> Actually there are several problems here in general:
> 1.The LogAppender should not do heavy works(disk file reading) with a lock
> held for somewhat meaningless job, we could known that the follower has dead
> and prevent it from happening.
> 2.The client handler thread should not be blocked by the LogAppender, there
> should be more light-weight way to do a notify of new jobs, such as a classic
> producer-consumer pattern with queues.
> This should be a ratis side problem according to the trace.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]