[ https://issues.apache.org/jira/browse/HADOOP-17749?focusedWorklogId=618489&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-618489 ]
ASF GitHub Bot logged work on HADOOP-17749: ------------------------------------------- Author: ASF GitHub Bot Created on: 05/Jul/21 02:55 Start Date: 05/Jul/21 02:55 Worklog Time Spent: 10m Work Description: liangxs commented on a change in pull request #3080: URL: https://github.com/apache/hadoop/pull/3080#discussion_r663608644 ########## File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/SocketIOWithTimeout.java ########## @@ -426,34 +408,41 @@ private synchronized SelectorInfo get(SelectableChannel channel) * * @param info */ - private synchronized void release(SelectorInfo info) { + private static void release(SelectorInfo info) { long now = Time.now(); trimIdleSelectors(now); info.lastActivityTime = now; - info.queue.addLast(info); + // SelectorInfos in queue are sorted by lastActivityTime + providerMap.get(info.provider).addLast(info); } + private static AtomicBoolean trimming = new AtomicBoolean(false); + private static volatile long lastTrimTime = Time.now(); Review comment: Thanks for your suggestion, I added a new commit. ########## File path: hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/SocketIOWithTimeout.java ########## @@ -426,34 +408,41 @@ private synchronized SelectorInfo get(SelectableChannel channel) * * @param info */ - private synchronized void release(SelectorInfo info) { + private static void release(SelectorInfo info) { long now = Time.now(); trimIdleSelectors(now); info.lastActivityTime = now; - info.queue.addLast(info); + // SelectorInfos in queue are sorted by lastActivityTime + providerMap.get(info.provider).addLast(info); } + private static AtomicBoolean trimming = new AtomicBoolean(false); + private static volatile long lastTrimTime = Time.now(); + /** * Closes selectors that are idle for IDLE_TIMEOUT (10 sec). It does not * traverse the whole list, just over the one that have crossed * the timeout. */ - private void trimIdleSelectors(long now) { + private static void trimIdleSelectors(long now) { + if (!trimming.compareAndSet(false, true)) { + return; + } + long cutoff = now - IDLE_TIMEOUT; - - for(ProviderInfo pList=providerList; pList != null; pList=pList.next) { - if (pList.queue.isEmpty()) { - continue; - } - for(Iterator<SelectorInfo> it = pList.queue.iterator(); it.hasNext();) { - SelectorInfo info = it.next(); - if (info.lastActivityTime > cutoff) { + for (ConcurrentLinkedDeque<SelectorInfo> infoQ : providerMap.values()) { + SelectorInfo oldest; + while ((oldest = infoQ.peekFirst()) != null) { + if (oldest.lastActivityTime <= cutoff && infoQ.remove(oldest)) { Review comment: @aajisaka, Thanks for your review, but I think the first element of `infoQ` may be changed in multi-thread environment. For example: 1. Precondition: `infoQ.size() == 1` 1. Thread t1: Line 435: `oldest = infoQ.peekFirst()` 1. Thread t2: Line 394: `infoQ.pollLast()` take oldest selector away and use the selector 1. Thread t3: Line 416: `providerMap.get(info.provider).addLast(info)` put a newest selector 1. Thread t1: Line 438: `infoQ.removeFirst()` remove the newest selector 1. Thread t1: `oldest.close()` close the oldest selector, which is using by Thread t2. 1. GC Thread: the newest selector is collect due to no reference to it, but the selector is not closed. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 618489) Time Spent: 3h 20m (was: 3h 10m) > Remove lock contention in SelectorPool of SocketIOWithTimeout > ------------------------------------------------------------- > > Key: HADOOP-17749 > URL: https://issues.apache.org/jira/browse/HADOOP-17749 > Project: Hadoop Common > Issue Type: Improvement > Components: common > Reporter: Xuesen Liang > Priority: Major > Labels: pull-request-available > Time Spent: 3h 20m > Remaining Estimate: 0h > > *SelectorPool* in > hadoop-common/src/main/java/org/apache/hadoop/net/*SocketIOWithTimeout.java* > is a point of lock contention. > For example: > {code:java} > $ grep 'waiting to lock <0x00007f7d94006d90>' 63692.jstack | uniq -c > 1005 - waiting to lock <0x00007f7d94006d90> (a > org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool) > {code} > and the thread stack is as follows: > {code:java} > "IPC Client (324579982) connection to /100.10.6.10:60020 from user_00" #14139 > daemon prio=5 os_prio=0 tid=0x00007f7374039000 nid=0x85cc waiting for monitor > entry [0x00007f6f45939000] > java.lang.Thread.State: BLOCKED (on object monitor) > at > org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.get(SocketIOWithTimeout.java:390) > - waiting to lock <0x00007f7d94006d90> (a > org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool) > at > org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:325) > at > org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:157) > at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:161) > at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:131) > at java.io.FilterInputStream.read(FilterInputStream.java:133) > at java.io.BufferedInputStream.fill(BufferedInputStream.java:246) > at java.io.BufferedInputStream.read(BufferedInputStream.java:265) > - locked <0x00007fa818caf258> (a java.io.BufferedInputStream) > at java.io.DataInputStream.readInt(DataInputStream.java:387) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection.readResponse(RpcClientImpl.java:967) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection.run(RpcClientImpl.java:568) > {code} > We should remove the lock contention. -- This message was sent by Atlassian Jira (v8.3.4#803005) --------------------------------------------------------------------- To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org