You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Yuto Kawamura (JIRA)" <ji...@apache.org> on 2017/01/11 08:19:58 UTC

[jira] [Commented] (KAFKA-4614) Long GC pause harming broker performance which is caused by mmap objects created for OffsetIndex

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

Yuto Kawamura commented on KAFKA-4614:
--------------------------------------

I've tried to fix this and now I'm testing patched broker in our environment.
Will submit a PR once it confirmed to fix this problem.

> Long GC pause harming broker performance which is caused by mmap objects created for OffsetIndex
> ------------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-4614
>                 URL: https://issues.apache.org/jira/browse/KAFKA-4614
>             Project: Kafka
>          Issue Type: Improvement
>          Components: core
>    Affects Versions: 0.10.0.1
>            Reporter: Yuto Kawamura
>            Assignee: Yuto Kawamura
>
> First let me clarify our system environment information as I think it's important to understand this issue:
> OS: CentOS6
> Kernel version: 2.6.32-XX
> Filesystem used for data volume: XFS
> Java version: 1.8.0_66
> GC option: Kafka default(G1GC)
> Kafka version: 0.10.0.1
> h2. Phenomenon
> In our Kafka cluster, an usual response time for Produce request is about 1ms for 50th percentile to 10ms for 99th percentile. All topics are configured to have 3 replicas and all producers are configured {{acks=all}} so this time includes replication latency.
> Sometimes we observe 99th percentile latency get increased to 100ms ~ 500ms but for the most cases the time consuming part is "Remote" which means it is caused by slow replication which is known to happen by various reasons(which is also an issue that we're trying to improve, but out of interest within this issue).
> However, we found that there are some different patterns which happens rarely but stationary 3 ~ 5 times a day for each servers. The interesting part is that "RequestQueue" also got increased as well as "Total" and "Remote".
> At the same time, we observed that the disk read metrics(in terms of both read bytes and read time) spikes exactly for the same moment. Currently we have only caught up consumers so this metric sticks to zero while all requests are served by page cache.
> In order to investigate what Kafka is "read"ing, I employed SystemTap and wrote the following script. It traces all disk reads(only actual read by physical device) made for the data volume by broker process.
> {code}
> global target_pid = KAFKA_PID
> global target_dev = DATA_VOLUME
> probe ioblock.request {
>   if (rw == BIO_READ && pid() == target_pid && devname == target_dev) {
>      t_ms = gettimeofday_ms() + 9 * 3600 * 1000 // timezone adjustment
>      printf("%s,%03d:  tid = %d, device = %s, inode = %d, size = %d\n", ctime(t_ms / 1000), t_ms % 1000, tid(), devname, ino, size)
>      print_backtrace()
>      print_ubacktrace()
>   }
> }
> {code}
> As the result, we could observe many logs like below:
> {code}
> Thu Dec 22 17:21:39 2016,209:  tid = 126123, device = sdb1, inode = -1, size = 4096
>  0xffffffff81275050 : generic_make_request+0x0/0x5a0 [kernel]
>  0xffffffff81275660 : submit_bio+0x70/0x120 [kernel]
>  0xffffffffa036bcaa : _xfs_buf_ioapply+0x16a/0x200 [xfs]
>  0xffffffffa036d95f : xfs_buf_iorequest+0x4f/0xe0 [xfs]
>  0xffffffffa036db46 : _xfs_buf_read+0x36/0x60 [xfs]
>  0xffffffffa036dc1b : xfs_buf_read+0xab/0x100 [xfs]
>  0xffffffffa0363477 : xfs_trans_read_buf+0x1f7/0x410 [xfs]
>  0xffffffffa033014e : xfs_btree_read_buf_block+0x5e/0xd0 [xfs]
>  0xffffffffa0330854 : xfs_btree_lookup_get_block+0x84/0xf0 [xfs]
>  0xffffffffa0330edf : xfs_btree_lookup+0xbf/0x470 [xfs]
>  0xffffffffa032456f : xfs_bmbt_lookup_eq+0x1f/0x30 [xfs]
>  0xffffffffa032628b : xfs_bmap_del_extent+0x12b/0xac0 [xfs]
>  0xffffffffa0326f34 : xfs_bunmapi+0x314/0x850 [xfs]
>  0xffffffffa034ad79 : xfs_itruncate_extents+0xe9/0x280 [xfs]
>  0xffffffffa0366de5 : xfs_inactive+0x2f5/0x450 [xfs]
>  0xffffffffa0374620 : xfs_fs_clear_inode+0xa0/0xd0 [xfs]
>  0xffffffff811affbc : clear_inode+0xac/0x140 [kernel]
>  0xffffffff811b0776 : generic_delete_inode+0x196/0x1d0 [kernel]
>  0xffffffff811b0815 : generic_drop_inode+0x65/0x80 [kernel]
>  0xffffffff811af662 : iput+0x62/0x70 [kernel]
>  0x37ff2e5347 : munmap+0x7/0x30 [/lib64/libc-2.12.so]
>  0x7ff169ba5d47 : Java_sun_nio_ch_FileChannelImpl_unmap0+0x17/0x50 [/usr/jdk1.8.0_66/jre/lib/amd64/libnio.so]
>  0x7ff269a1307e
> {code}
> We took a jstack dump of the broker process and found that tid = 126123 corresponds to the thread which is for GC(hex(tid) == nid(Native Thread ID)):
> {code}
> $ grep 0x1ecab /tmp/jstack.dump
> "Reference Handler" #2 daemon prio=10 os_prio=0 tid=0x00007ff278d0c800 nid=0x1ecab in Object.wait() [0x00007ff17da11000]
> {code}
> In order to confirm, we enabled {{PrintGCApplicationStoppedTime}} switch and confirmed that in total the time the broker paused is longer than usual, when a broker experiencing the issue.
> {code}
> $ grep --text 'Total time for which application threads were stopped' kafkaServer-gc.log  | grep --text '11T01:4' | perl -ne '/^(.*T\d{2}:\d{2}).*stopped: ([0-9\.]+)/; $h{$1} += $2 * 1000; END { print "$_ = $h{$_}\n" for sort keys %h }'
> 2017-01-11T01:40 = 332.0601
> 2017-01-11T01:41 = 315.1805
> 2017-01-11T01:42 = 318.4317
> 2017-01-11T01:43 = 317.8821
> 2017-01-11T01:44 = 302.1132
> 2017-01-11T01:45 = 950.5807 # << here
> 2017-01-11T01:46 = 344.9449
> 2017-01-11T01:47 = 328.936
> 2017-01-11T01:48 = 296.3204
> 2017-01-11T01:49 = 299.2834
> $ grep --text 'Total time for which application threads were stopped' kafkaServer-gc.log  | grep --text '11T01:45' | perl -ne '/^(.*T\d{2}:\d{2}:\d{2}).*stopped: ([0-9\.]+)/; $h{$1} += $2 * 1000; END { print "$_ = $h{$_}\n" for sort keys %h }' | head -20
> 2017-01-11T01:45:00 = 31.1591
> 2017-01-11T01:45:01 = 28.7814
> 2017-01-11T01:45:02 = 30.0265
> 2017-01-11T01:45:03 = 66.7318
> 2017-01-11T01:45:04 = 20.6993
> 2017-01-11T01:45:05 = 52.8268
> 2017-01-11T01:45:06 = 29.6571
> 2017-01-11T01:45:07 = 25.6357
> 2017-01-11T01:45:08 = 58.6863
> 2017-01-11T01:45:09 = 33.022
> 2017-01-11T01:45:10 = 3.2497
> 2017-01-11T01:45:11 = 9.916
> 2017-01-11T01:45:12 = 19.7215
> 2017-01-11T01:45:13 = 61.652
> 2017-01-11T01:45:15 = 8.9058
> 2017-01-11T01:45:16 = 16.7779
> 2017-01-11T01:45:17 = 11.3215
> 2017-01-11T01:45:18 = 82.2986
> 2017-01-11T01:45:19 = 32.2718
> 2017-01-11T01:45:20 = 1.1403
> {code}
> h2. Hypothesis
> In case of Kafka broker running on Linux, an mmap object of offset index are cleaned up like below:
> - OffsetIndex.delete() called and the corresponding file marked as deleted on VFS level(still not physically deleted at this moment, because there's an mmap object which holds opened reference to the same file)
> - OffsetIndex object also cut off from any reference and becomes a collectable garbage
> - While after, GC collects mmap object(MappedByteBuffer) and run the cleanup routine so it munmaps the mapping
> Hence, munmap being called from GC thread itself sounds normal.
> So the question should be why releasing mmap inside the GC thread likely causes such a long STW moment.
> After a tough investigation, we found that the following scenario can explain it:
> 1. Since OffsetIndex object kept living for a long moment(depending retention.ms?) it should have already promoted to old gen.
> 2. OffsetIndex.delete() called so the file marked as deleted. At the same time, the operation should warmed up inode cache.
> 3. However, since G1GC collects a region which contains a lot of garbages preferentially, if a region that an mmap object contained still has many living objects, it might be left and not being collected for a long moment.
> 4. During this moment, inode cache for the index file evicted from XFS layer cache.
> 5. Eventually the mmap object collected and munmap called, which triggers kernel to perform physical deletion of the inode associated to the file, but at this moment, metadata cache for the index file already evicted from cache so XFS driver have to read it from disk. Since data volume disks of Kafka logs are very busy for write workload, reading disk is very time-consuming operation. During the moment, application threads are blocked and after resumed, all pending requests marks slow response time.
> h2. Experiment
> In order to confirm this scenario, I tried to reproduce STW caused by GC freeing mmap which corresponds to a file that it's metadata out of the cache.
> I wrote below Java code and ran several times with observations.
> {code}
> import java.io.File;
> import java.io.FileInputStream;
> import java.io.FileOutputStream;
> import java.io.IOException;
> import java.io.RandomAccessFile;
> import java.nio.MappedByteBuffer;
> import java.nio.channels.FileChannel.MapMode;
> import java.text.SimpleDateFormat;
> import java.util.Date;
> import java.util.concurrent.TimeUnit;
> import sun.nio.ch.DirectBuffer;
> /**
>  * Test scenario:
>  *
>  * - Create N(given as an argument) number of files under the directory and corresponding N mmaps.
>  * - Remove all files created.
>  * - Sleep for a while, with expect to periodic XFS cache cleaner routine executed.
>  * - Drop VFS dentry cache
>  * - Cut references to mmaps
>  * - Keep creating objects to pressure heap and lead GC to collect mmap garbages.
>  */
> public final class MmapGcTest {
>     private static String filename(int num) {
>         return "/path/to/data_volume/mmaps/mmap-" + num;
>     }
>     public static void main(String[] args) throws IOException, InterruptedException {
>         int numFiles = Integer.parseInt(args[0]);
>         boolean cleanMmapsExplicitly = Boolean.parseBoolean(args[1]);
>         Thread.sleep(20000);
>         MappedByteBuffer[] mmaps = new MappedByteBuffer[numFiles];
>         for (int i = 0; i < numFiles; i++) {
>             RandomAccessFile file = new RandomAccessFile(filename(i), "rw");
>             file.setLength(128);
>             MappedByteBuffer mmap = file.getChannel().map(MapMode.READ_WRITE, 0, file.length());
>             file.close();
>             mmaps[i] = mmap;
>         }
>         System.err.println("removing files");
>         for (int i = 0; i < numFiles; i++) {
>             boolean deleted = new File(filename(i)).delete();
>             if (!deleted) {
>                 throw new RuntimeException("failed to delete file");
>             }
>         }
>         // Wait until XFS expires inode buffer cache
>         final long timeToSleep;
>         try (FileInputStream fin = new FileInputStream("/proc/sys/fs/xfs/xfssyncd_centisecs")) {
>             byte[] buf = new byte[256];
>             fin.read(buf);
>             timeToSleep = Long.parseLong(new String(buf).trim()) * 10 /* centisecs to millisecs */ + 1000 /* and little bit more */;
>         }
>         System.err.println("Sleeping " + timeToSleep + " ms for waiting XFS inode buffer expiration");
>         Thread.sleep(timeToSleep);
>         System.err.println("drop caches");
>         try (FileOutputStream out = new FileOutputStream("/proc/sys/vm/drop_caches")) {
>             out.write("2\n".getBytes());
>         }
>         // Cut references to mmaps and let GC to collect them
>         if (cleanMmapsExplicitly) {
>             for (MappedByteBuffer mmap : mmaps) {
>                 ((DirectBuffer) mmap).cleaner().clean();
>             }
>             System.err.println("mmaps cleaned up");
>         }
>         mmaps = null;
>         System.err.println("cut reference to mmaps");
>         SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS");
>         // Keep creating relatively large objects and see if there's a significantly large interval which
>         // seemingly caused by GC pause.
>         byte[][] chunks = new byte[32768][];
>         for (int i = 0; i < Integer.MAX_VALUE; i++) {
>             long t0 = System.nanoTime();
>             byte[] chunk = new byte[4096];
>             long elapsedMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - t0);
>             if (elapsedMillis > 100) {
>                 System.err.printf("%s likely long GC pause: %d ms\n", sdf.format(new Date()), elapsedMillis);
>             }
>             chunks[i % chunks.length] = chunk;
>         }
>     }
> }
> {code}
> At the same time the above program executed, I ran below two commands in different session:
> - The SystemTap script that I pasted above: to confirm same disk access pattern(code path)
> - {{dd if=/dev/zero of=$DATA_VOLUME/huge-zero.out bs=1024 count=$((100 * 1024 * 1024))}}: to simulate disk device pressure of broker servers.
> Here's the logs of execution:
> {code}
> $ sudo java -Xmx1024m -server -XX:+UseG1GC -XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35 -XX:+DisableExplicitGC -Xloggc:gc.log -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps -XX:+PrintGCTimeStamps -XX:+PrintReferenceGC -XX:+PrintGCApplicationStoppedTime MmapGcTest 10000 false
> removing files
> Sleeping 31000 ms for waiting XFS inode buffer expiration
> drop caches
> cut reference to mmaps
> 2017-01-06 20:52:58.195 likely long GC pause: 3372 ms
> 2017-01-06 20:53:02.802 likely long GC pause: 2311 ms
> 2017-01-06 20:53:04.275 likely long GC pause: 1467 ms
> 2017-01-06 20:53:06.572 likely long GC pause: 2268 ms
> ...
> $ grep -A 30 20:53:02 stap.log | head -30
> Fri Jan  6 20:53:02 2017,024:  tid = 95825, device = XXX, inode = -1, size = 8192
>  0xffffffff812746d0 : generic_make_request+0x0/0x5a0 [kernel]
>  0xffffffff81274ce0 : submit_bio+0x70/0x120 [kernel]
>  0xffffffffa042acba : _xfs_buf_ioapply+0x16a/0x200 [xfs]
>  0xffffffffa042c96f : xfs_buf_iorequest+0x4f/0xe0 [xfs]
>  0xffffffffa042cb56 : _xfs_buf_read+0x36/0x60 [xfs]
>  0xffffffffa042cc2b : xfs_buf_read+0xab/0x100 [xfs]
>  0xffffffffa0422487 : xfs_trans_read_buf+0x1f7/0x410 [xfs]
>  0xffffffffa040a654 : xfs_imap_to_bp+0x54/0x130 [xfs]
>  0xffffffffa040abdc : xfs_inotobp+0x8c/0xc0 [xfs]
>  0xffffffffa040acf1 : xfs_iunlink_remove+0xe1/0x320 [xfs]
>  0xffffffffa040af57 : xfs_ifree+0x27/0x110 [xfs]
>  0xffffffffa0425c98 : xfs_inactive+0x198/0x450 [xfs]
>  0xffffffffa0433630 : xfs_fs_clear_inode+0xa0/0xd0 [xfs]
>  0xffffffff811af6fc : clear_inode+0xac/0x140 [kernel]
>  0xffffffff811afeb6 : generic_delete_inode+0x196/0x1d0 [kernel]
>  0xffffffff811aff55 : generic_drop_inode+0x65/0x80 [kernel]
>  0xffffffff811aeda2 : iput+0x62/0x70 [kernel]
>  0xffffffff811ab7a0 : dentry_iput+0x90/0x100 [kernel]
>  0xffffffff811ab901 : d_kill+0x31/0x60 [kernel]
>  0xffffffff811ad53d : dput+0xcd/0x160 [kernel]
>  0x391d6e5347 : munmap+0x7/0x30 [/lib64/libc-2.12.so]
>  0x7ff8d873dd47 : Java_sun_nio_ch_FileChannelImpl_unmap0+0x17/0x50 [/usr/jdk1.8.0_92/jre/lib/amd64/libnio.so]
>  0x7ff9893376fe
> ...
> $ grep 20:53:02 gc.log | grep -E 'Total time for which application threads were stopped: [^0]\.'
> 2017-01-06T20:53:02.802+0900: 60.898: Total time for which application threads were stopped: 2.3118335 seconds, Stopping threads took: 2.3054883 seconds
> $ grep 0x17651 jstack-case1
> "Reference Handler" #2 daemon prio=10 os_prio=0 tid=0x00007ff9982b6000 nid=0x17651 in Object.wait() [0x00007ff8da12f000]
> {code}
> As you can see, GC is obviously blocking the application unusually long, and at the same time there is a same code path that the XFS is reading metadata from physical disk.
> So next I switched the second argument of the program to {{true}} and enforeced mmap cleanup within the main thread:
> {code}
> $ sudo java -Xmx1024m -server -XX:+UseG1GC -XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35 -XX:+DisableExplicitGC -Xloggc:gc.log -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps -XX:+PrintGCTimeStamps -XX:+PrintReferenceGC -XX:+PrintGCApplicationStoppedTime MmapGcTest 10000 true
> removing files
> Sleeping 31000 ms for waiting XFS inode buffer expiration
> drop caches
> mmaps cleaned up
> cut reference to mmaps
> $ tail -24 stap.log
> Fri Jan  6 21:06:53 2017,166:  tid = 118226, device = XXX, inode = -1, size = 4096
>  0xffffffff812746d0 : generic_make_request+0x0/0x5a0 [kernel]
>  0xffffffff81274ce0 : submit_bio+0x70/0x120 [kernel]
>  0xffffffffa042acba : _xfs_buf_ioapply+0x16a/0x200 [xfs]
>  0xffffffffa042c96f : xfs_buf_iorequest+0x4f/0xe0 [xfs]
>  0xffffffffa042cb56 : _xfs_buf_read+0x36/0x60 [xfs]
>  0xffffffffa042cc2b : xfs_buf_read+0xab/0x100 [xfs]
>  0xffffffffa0422487 : xfs_trans_read_buf+0x1f7/0x410 [xfs]
>  0xffffffffa03ef15e : xfs_btree_read_buf_block+0x5e/0xd0 [xfs]
>  0xffffffffa03ef864 : xfs_btree_lookup_get_block+0x84/0xf0 [xfs]
>  0xffffffffa03efeef : xfs_btree_lookup+0xbf/0x470 [xfs]
>  0xffffffffa03d6fd9 : xfs_alloc_lookup_eq+0x19/0x20 [xfs]
>  0xffffffffa03d7dd4 : xfs_free_ag_extent+0x1c4/0x750 [xfs]
>  0xffffffffa03da111 : xfs_free_extent+0x101/0x130 [xfs]
>  0xffffffffa03e3fcd : xfs_bmap_finish+0x15d/0x1a0 [xfs]
>  0xffffffffa0425e97 : xfs_inactive+0x397/0x450 [xfs]
>  0xffffffffa0433630 : xfs_fs_clear_inode+0xa0/0xd0 [xfs]
>  0xffffffff811af6fc : clear_inode+0xac/0x140 [kernel]
>  0xffffffff811afeb6 : generic_delete_inode+0x196/0x1d0 [kernel]
>  0xffffffff811aff55 : generic_drop_inode+0x65/0x80 [kernel]
>  0xffffffff811aeda2 : iput+0x62/0x70 [kernel]
>  0x391d6e5347 : munmap+0x7/0x30 [/lib64/libc-2.12.so]
>  0x7fc11ca7ad47 : Java_sun_nio_ch_FileChannelImpl_unmap0+0x17/0x50 [/usr/jdk1.8.0_92/jre/lib/amd64/libnio.so]
>  0x7fc1e90174f4
> $ grep 0x1cdd2 jstack-case2
> "main" #1 prio=5 os_prio=0 tid=0x00007fc1f8008800 nid=0x1cdd2 runnable [0x00007fc1fe8dd000]
> $ grep -E 'Total time for which application threads were stopped: [^0]\.' gc.log | wc -l
> 0
> {code}
> Of course there were same code path that XFS is reading from physical disk, but this time it's comming from main thread.
> This time, none of GC occurrence blocked application thread more than 100ms, which can be confirmed by both of program's log and gc log.
> h2. Suggestion
> In order to solve this problem, we can call {{DirectBuffer.cleaner().clean()}} explicitly within log cleaner thread so no more GC time affected by transparent disk reads for metadata.
> The only downside that I can imagine is that it uses hidden API which is not recommended to use directly, but we're already importing it in order to support Windows so it's out of consideration.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)