[ https://issues.apache.org/jira/browse/HADOOP-17749?focusedWorklogId=611297&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-611297 ]
ASF GitHub Bot logged work on HADOOP-17749: ------------------------------------------- Author: ASF GitHub Bot Created on: 15/Jun/21 12:59 Start Date: 15/Jun/21 12:59 Worklog Time Spent: 10m Work Description: liangxs commented on pull request #3080: URL: https://github.com/apache/hadoop/pull/3080#issuecomment-861476259 I tested the performance of the trunk version and the optimized version. ### Test Case The test steps are as follow: 1. Start a netty-echo-server with 100 service ports. ``` private static void startServer() throws Exception { ChannelHandler serverHandler = new EchoHandler(); for (int i = 0; i < 100; ++i) { ServerBootstrap b = new ServerBootstrap(); b.group(new NioEventLoopGroup(1), new NioEventLoopGroup(2)) .channel(NioServerSocketChannel.class) .option(ChannelOption.SO_BACKLOG, 512) .childOption(ChannelOption.SO_TIMEOUT, timeout) .childOption(ChannelOption.TCP_NODELAY, true) .childHandler(serverHandler); ChannelFuture f = b.bind(host, port + i).sync(); } Thread.sleep(Integer.MAX_VALUE); } ``` 2. Start a hadoop socket client with multi-threads. These threads connect to the netty-echo-server's ports by a round robin manner. ``` private static void startClient(int threadCnt) throws Exception { SocketFactory factory = new StandardSocketFactory(); Thread[] tArray = new Thread[threadCnt]; CountDownLatch latch = new CountDownLatch(threadCnt); for (int i = 0; i < threadCnt; ++i) { final int curPort = port + (i % 100); // round robin Thread t = new Thread(() -> { try { Socket socket = factory.createSocket(); socket.setTcpNoDelay(true); socket.setKeepAlive(false); NetUtils.connect(socket, new java.net.InetSocketAddress(host, curPort), timeout); socket.setSoTimeout(timeout); ... ... ``` 3. Each client thread send-to/recv-from its corresponding netty-server port 1024 times, with 256-byte data each time. ``` InputStream inStream = NetUtils.getInputStream(socket); OutputStream outStream = NetUtils.getOutputStream(socket, timeout); DataInputStream in = new DataInputStream(new BufferedInputStream(inStream)); DataOutputStream out = new DataOutputStream(new BufferedOutputStream(outStream)); byte[] buf = new byte[256]; for (int j = 0; j < 1024; ++j) { out.write(buf); out.flush(); in.readFully(buf); } ``` 4. print the total cost. Code project: [https://github.com/liangxs/test-HADOOP-17749](https://github.com/liangxs/test-HADOOP-17749) ### Test Result The test result is as follow: ``` | client thread count | 100 | 200 | 400 | 800 | 1200 | 1600 | 2000 | 2400 | 2800 | |---------------------|-----|-----|------|------|------|------|------|------|------| | trunk (millis) | 351 | 609 | 1058 | 2024 | 2907 | 3882 | 5062 | 5675 | 7117 | | optimized (millis) | 253 | 438 | 799 | 1167 | 1561 | 2422 | 2784 | 2813 | 3371 | | improved | 38% | 39% | 32% | 70% | 86% | 60% | 82% | 102% | 111% | ``` ps, I used two test machines with same hardware and software configuration and on a same rack: ``` $ lscpu Architecture: x86_64 CPU(s): 56 On-line CPU(s) list: 0-55 Thread(s) per core: 2 Core(s) per socket: 14 Socket(s): 2 NUMA node(s): 2 Model name: Intel(R) Xeon(R) CPU E5-2680 v4 @ 2.40GHz CPU MHz: 2401.000 $ free -g total used free shared buff/cache available Mem: 62 7 37 0 17 54 $ lspci | grep Ethernet 06:00.0 Ethernet controller: Intel Corporation Ethernet Controller 10-Gigabit X540-AT2 (rev 01) $ uname -r 3.10.107-1 ``` -- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 611297) Time Spent: 1h (was: 50m) > 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: 1h > 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