cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Norman Maurer (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-13651) Large amount of CPU used by epoll_wait(.., .., .., 0)
Date Wed, 02 Aug 2017 05:33:01 GMT

    [ https://issues.apache.org/jira/browse/CASSANDRA-13651?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16110335#comment-16110335
] 

Norman Maurer commented on CASSANDRA-13651:
-------------------------------------------

Sorry for the late response.. didn't see the mention :(

So yes netty uses `epoll_wait` which only supports milli-seconds resulution so everything
smaller then this will just cause `epoll_wait(...)` be called with a `0` and so a non-blocking
check of ready fds. What we could do in our native transport implementation is that we make
use of `timerfd` [1] to schedule timeouts but again this would only work for the case of using
the native epoll transport and not when you use the nio transport (which works on all OS).
So I think what you really want to do is have timeouts of >= 1ms. 

Comments and ideas welcome :)


> Large amount of CPU used by epoll_wait(.., .., .., 0)
> -----------------------------------------------------
>
>                 Key: CASSANDRA-13651
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-13651
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Corentin Chary
>             Fix For: 4.x
>
>         Attachments: cpu-usage.png
>
>
> I was trying to profile Cassandra under my workload and I kept seeing this backtrace:
> {code}
> epollEventLoopGroup-2-3 State: RUNNABLE CPU usage on sample: 240ms
> io.netty.channel.epoll.Native.epollWait0(int, long, int, int) Native.java (native)
> io.netty.channel.epoll.Native.epollWait(int, EpollEventArray, int) Native.java:111
> io.netty.channel.epoll.EpollEventLoop.epollWait(boolean) EpollEventLoop.java:230
> io.netty.channel.epoll.EpollEventLoop.run() EpollEventLoop.java:254
> io.netty.util.concurrent.SingleThreadEventExecutor$5.run() SingleThreadEventExecutor.java:858
> io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run() DefaultThreadFactory.java:138
> java.lang.Thread.run() Thread.java:745
> {code}
> At fist I though that the profiler might not be able to profile native code properly,
but I wen't further and I realized that most of the CPU was used by {{epoll_wait()}} calls
with a timeout of zero.
> Here is the output of perf on this system, which confirms that most of the overhead was
with timeout == 0.
> {code}
> Samples: 11M of event 'syscalls:sys_enter_epoll_wait', Event count (approx.): 11594448
> Overhead  Trace output                                                              
                                                                                         
                                  ◆
>   90.06%  epfd: 0x00000047, events: 0x7f5588c0c000, maxevents: 0x00002000, timeout: 0x00000000
                                                                                         
                        ▒
>    5.77%  epfd: 0x000000b5, events: 0x7fca419ef000, maxevents: 0x00001000, timeout: 0x00000000
                                                                                         
                        ▒
>    1.98%  epfd: 0x000000b5, events: 0x7fca419ef000, maxevents: 0x00001000, timeout: 0x000003e8
                                                                                         
                        ▒
>    0.04%  epfd: 0x00000003, events: 0x2f6af77b9c00, maxevents: 0x00000020, timeout: 0x00000000
                                                                                         
                        ▒
>    0.04%  epfd: 0x0000002b, events: 0x121ebf63ac00, maxevents: 0x00000040, timeout: 0x00000000
                                                                                         
                        ▒
>    0.03%  epfd: 0x00000026, events: 0x7f51f80019c0, maxevents: 0x00000020, timeout: 0x00000000
                                                                                         
                        ▒
>    0.02%  epfd: 0x00000003, events: 0x7fe4d80019d0, maxevents: 0x00000020, timeout: 0x00000000
> {code}
> Running this time with perf record -ag for call traces:
> {code}
> # Children      Self       sys       usr  Trace output                              
                                         
> # ........  ........  ........  ........  ....................................................................................
> #
>      8.61%     8.61%     0.00%     8.61%  epfd: 0x000000a7, events: 0x7fca452d6000, maxevents:
0x00001000, timeout: 0x00000000
>             |
>             ---0x1000200af313
>                |          
>                 --8.61%--0x7fca6117bdac
>                           0x7fca60459804
>                           epoll_wait
>      2.98%     2.98%     0.00%     2.98%  epfd: 0x000000a7, events: 0x7fca452d6000, maxevents:
0x00001000, timeout: 0x000003e8
>             |
>             ---0x1000200af313
>                0x7fca6117b830
>                0x7fca60459804
>                epoll_wait
> {code}
> That looks like a lot of CPU used to wait for nothing. I'm not sure if pref reports a
per-CPU percentage or a per-system percentage, but that would be still be 10% of the total
CPU usage of Cassandra at the minimum.
> I went further and found the code of all that: We schedule a lot of {{Message::Flusher}}
with a deadline of 10 usec (5 per messages I think) but netty+epoll only support timeouts
above the milliseconds and will convert everything bellow to 0.
> I added some traces to netty (4.1):
> {code}
> diff --git a/transport-native-epoll/src/main/java/io/netty/channel/epoll/EpollEventLoop.java
b/transport-native-epoll/src/main/java/io/netty/channel/epoll/EpollEventLoop.java
> index 909088fde..8734bbfd4 100644
> --- a/transport-native-epoll/src/main/java/io/netty/channel/epoll/EpollEventLoop.java
> +++ b/transport-native-epoll/src/main/java/io/netty/channel/epoll/EpollEventLoop.java
> @@ -208,10 +208,15 @@ final class EpollEventLoop extends SingleThreadEventLoop {
>          long currentTimeNanos = System.nanoTime();
>          long selectDeadLineNanos = currentTimeNanos + delayNanos(currentTimeNanos);
>          for (;;) {
> -            long timeoutMillis = (selectDeadLineNanos - currentTimeNanos + 500000L)
/ 1000000L;
> +            long timeoutNanos = selectDeadLineNanos - currentTimeNanos + 500000L;
> +            long timeoutMillis =  timeoutNanos / 1000000L;
> +            System.out.printf("timeoutNanos: %d, timeoutMillis: %d | deadline: %d -
now: %d | hastask: %d\n",
> +                    timeoutNanos, timeoutMillis,
> +                    selectDeadLineNanos, currentTimeNanos, hasTasks() ? 1 : 0);
>              if (timeoutMillis <= 0) {
>                  if (selectCnt == 0) {
>                      int ready = Native.epollWait(epollFd.intValue(), events, 0);
> +                    System.out.printf("ready: %d\n", ready);
>                      if (ready > 0) {
>                          return ready;
>                      }
> {code}
> And this gives :
> {code}
> timeoutNanos: 1000500000, timeoutMillis: 1000 | deadline: 2001782341816510 - now: 2001781341816510
| hastask: 0
> timeoutNanos: 1000500000, timeoutMillis: 1000 | deadline: 2001782342087239 - now: 2001781342087239
| hastask: 0
> timeoutNanos: 1000500000, timeoutMillis: 1000 | deadline: 2001782342166947 - now: 2001781342166947
| hastask: 0
> timeoutNanos: 508459, timeoutMillis: 0 | deadline: 2001781342297987 - now: 2001781342289528
| hastask: 0
> ready: 0
> timeoutNanos: 508475, timeoutMillis: 0 | deadline: 2001781342357719 - now: 2001781342349244
| hastask: 0
> ready: 0
> timeoutNanos: 509327, timeoutMillis: 0 | deadline: 2001781342394822 - now: 2001781342385495
| hastask: 0
> ready: 0
> timeoutNanos: 509339, timeoutMillis: 0 | deadline: 2001781342430192 - now: 2001781342420853
| hastask: 0
> ready: 0
> timeoutNanos: 509510, timeoutMillis: 0 | deadline: 2001781342461588 - now: 2001781342452078
| hastask: 0
> ready: 0
> timeoutNanos: 509493, timeoutMillis: 0 | deadline: 2001781342495044 - now: 2001781342485551
| hastask: 0
> ready: 0
> {code}
> The nanosecond timeout all come from {{eventLoop.schedule(this, 10000, TimeUnit.NANOSECONDS);}}
in {{Message::Flusher}}.
> Knowing that, I'm not sure what would be best to do, and I have a hard time understanding
Message::Flusher, but to me it looks like trying to schedule less tasks would probably help
and I didn't think anything obvious that could be done with netty.
> Changing {{if (++runsWithNoWork > 5)}} to 2 seems to help a little bit, but that isn't
really significant.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


Mime
View raw message