Running Centos 6.7 3.10.95-1.el6.elrepo.x86_64. 4 SATA disks in RAID10 with ZFS
On Thu, Jan 12, 2017 at 2:27 PM, Tauzell, Dave <dave.tauz...@surescripts.com > wrote: > You have a local filesystem? Linux? > > -Dave > > -----Original Message----- > From: Stephen Powis [mailto:spo...@salesforce.com] > Sent: Thursday, January 12, 2017 1:22 PM > To: users@kafka.apache.org > Subject: Re: Taking a long time to roll a new log segment (~1 min) > > I've further narrowed it down to this particular line: > https://github.com/apache/kafka/blob/0.10.0/core/src/main/scala/kafka/log/ > OffsetIndex.scala#L294 > > But I'm still at a loss to why this would be slow sometimes and not others. > > On Thu, Jan 12, 2017 at 10:56 AM, Stephen Powis <spo...@salesforce.com> > wrote: > > > Using the little bash script in that JIRA ticket to go thru the GC log > > and sum up the total pause times I come up with the following. I > > don't see anything that would indicate a ~28 second pause. > > > > 2017-01-12T07:00 = 72.2961 > >> 2017-01-12T07:01 = 59.8112 > >> 2017-01-12T07:02 = 69.6915 > >> 2017-01-12T07:03 = 74.4093 > >> 2017-01-12T07:04 = 73.507 > >> 2017-01-12T07:05 = 73.7028 > >> 2017-01-12T07:06 = 72.3392 > >> 2017-01-12T07:07 = 70.502 > >> 2017-01-12T07:08 = 71.7471 > >> 2017-01-12T07:09 = 76.8371 > >> 2017-01-12T07:10 = 72.3836 > >> 2017-01-12T07:11 = 74.7462 > >> 2017-01-12T07:12 = 57.1328 > >> 2017-01-12T07:13 = 80.005 > >> 2017-01-12T07:14 = 76.0012 > >> 2017-01-12T07:15 = 79.2221 > >> 2017-01-12T07:16 = 57.5385 > >> 2017-01-12T07:17 = 58.1185 > >> 2017-01-12T07:18 = 72.0524 > >> 2017-01-12T07:19 = 110.6736 > >> 2017-01-12T07:20 = 64.9808 > >> 2017-01-12T07:21 = 66.8767 > >> 2017-01-12T07:22 = 32.409 > >> 2017-01-12T07:23 = 85.3671 > >> 2017-01-12T07:24 = 123.9212 > >> 2017-01-12T07:25 = 70.3219 > >> > > > > > > > > > > > > On Thu, Jan 12, 2017 at 10:49 AM, Stephen Powis > > <spo...@salesforce.com> > > wrote: > > > >> Just realized that GCEasy doesn't keep reports around for very long > >> anymore, here is a screencap of the report: http://imgur.com/a/MEubD > >> > >> The longest reported GC pause was 30ms, tho they happen somewhat > >> frequently at an average of once per 12 seconds. KAFKA-4616 > >> certainly sounds just like my scenario, but I'm having trouble > >> finding the long GC pauses described in that ticket in my own logs. > >> > >> On Thu, Jan 12, 2017 at 10:42 AM, tao xiao <xiaotao...@gmail.com> > wrote: > >> > >>> You may run into this bug https://issues.apache.org/jira > >>> /browse/KAFKA-4614 > >>> > >>> On Thu, 12 Jan 2017 at 23:38 Stephen Powis <spo...@salesforce.com> > >>> wrote: > >>> > >>> > Per my email to the list in Sept, when I reviewed GC logs then, I > >>> didn't > >>> > see anything out of the ordinary. ( > >>> > > >>> > http://mail-archives.apache.org/mod_mbox/kafka-users/201609. > >>> mbox/%3CCABQB-gS7h4Nuq3TKgHoAVeRHPWnBNs2B0Tz0kCjmdB9c0SDcLQ% > >>> 40mail.gmail.com%3E > >>> > ) > >>> > > >>> > Reviewing the GC logs from this morning around the timestamp in my > >>> previous > >>> > email, I see GC running roughly every 10-12 seconds, with total > >>> > times similar to the following: > >>> > > >>> > 2017-01-12T07:16:46.867-0500: 46891.844: Total time for which > >>> application > >>> > > threads were stopped: 0.0141281 seconds, Stopping threads took: > >>> 0.0002171 > >>> > > seconds > >>> > > > >>> > > > >>> > Here is a link to a GCEasy report: > >>> > > >>> > http://gceasy.io/my-gc-report.jsp?p=c2hhcmVkLzIwMTcvMDEvMTIv > >>> LS10b3RhbEdDLWthZmthMS00LmxvZy5nei0tMTUtMzQtNTk= > >>> > > >>> > > >>> > Currently using G1 gc with the following settings: > >>> > > >>> > -Xmx12G -Xms12G -server -XX:MaxPermSize=48M -verbose:gc > >>> > -Xloggc:/var/log/kafka/gc.log -XX:+PrintGCDateStamps > >>> -XX:+PrintGCDetails > >>> > -XX:+PrintTenuringDistribution -XX:+PrintGCApplicationStoppedTime > >>> > -XX:+PrintTLAB -XX:+DisableExplicitGC -XX:+UseGCLogFileRotation > >>> > -XX:NumberOfGCLogFiles=10 -XX:GCLogFileSize=100M > >>> > -XX:+UseCompressedOops -XX:+AlwaysPreTouch -XX:+UseG1GC > >>> > -XX:MaxGCPauseMillis=20 -XX:+HeapDumpOnOutOfMemoryError > >>> > -XX:HeapDumpPath=/var/log/kafka/heapDump.log > >>> > -Xloggc:/opt/kafka/current/bin/../logs/kafkaServer-gc.log > >>> > -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps > >>> > -XX:+PrintGCTimeStamps > >>> > > >>> > > >>> > > >>> > On Thu, Jan 12, 2017 at 9:55 AM, Tauzell, Dave < > >>> > dave.tauz...@surescripts.com > >>> > > wrote: > >>> > > >>> > > Can you collect garbage collection stats and verify there isn't > >>> > > a > >>> long GC > >>> > > happening at the same time? > >>> > > > >>> > > -Dave > >>> > > > >>> > > -----Original Message----- > >>> > > From: Stephen Powis [mailto:spo...@salesforce.com] > >>> > > Sent: Thursday, January 12, 2017 8:34 AM > >>> > > To: users@kafka.apache.org > >>> > > Subject: Re: Taking a long time to roll a new log segment (~1 > >>> > > min) > >>> > > > >>> > > So per the kafka docs I up'd our FD limit to 100k, and we are no > >>> longer > >>> > > seeing the process die, which is good. > >>> > > > >>> > > Unfortunately we're still seeing very high log segment roll > >>> > > times, > >>> and > >>> > I'm > >>> > > unsure if this is considered 'normal', as it tends to block > >>> > > producers during this period. > >>> > > > >>> > > We are running kafka 0.10.0.1, but I patched in some > >>> > > additionally > >>> timing > >>> > > statements into the kafka.log.log roll() method to narrow down > >>> exactly > >>> > > which part of that method is taking so long. > >>> > > > >>> > > Again, typically the process to roll a new log file takes only > >>> > > 1-2ms > >>> > tops, > >>> > > but several times a day it takes 30-60+ seconds, across all of > >>> > > our brokers. I've narrowed it down to this bit of code causing > >>> > > the > >>> issue: > >>> > > https://github.com/apache/kafka/blob/0.10.0/core/src/ > >>> > > main/scala/kafka/log/Log.scala#L652-L658 > >>> > > > >>> > > Here's an example of output w/ my additional timing log statements: > >>> > > > >>> > > [2017-01-12 07:17:58,199] INFO Rolled new log segment for > >>> 'MyTopic-4' in > >>> > > > 28028 ms. (kafka.log.Log) > >>> > > > >>> > > [2017-01-12 07:17:58,200] INFO LastEntry match time 'EventStream-4' > >>> in > >>> > > > 28027 ms. (kafka.log.Log) > >>> > > > > >>> > > > >>> > > Any thoughts? Help? This considered normal? > >>> > > > >>> > > Thanks > >>> > > Stephen > >>> > > > >>> > > > >>> > > On Tue, Jan 10, 2017 at 2:37 AM, Ewen Cheslack-Postava < > >>> > e...@confluent.io> > >>> > > wrote: > >>> > > > >>> > > > I can't speak to the exact details of why fds would be kept > >>> > > > open longer in that specific case, but are you aware that the > >>> > > > recommendation for production clusters for open fd limits is > >>> > > > much higher? It's been suggested to be 100,000 as a starting > >>> > > > point for > >>> quite > >>> > > awhile: > >>> > > > http://kafka.apache.org/documentation.html#os > >>> > > > > >>> > > > -Ewen > >>> > > > > >>> > > > On Mon, Jan 9, 2017 at 12:45 PM, Stephen Powis < > >>> spo...@salesforce.com> > >>> > > > wrote: > >>> > > > > >>> > > > > Hey! > >>> > > > > > >>> > > > > I've run into something concerning in our production > >>> > > > > cluster....I believe I've posted this question to the > >>> > > > > mailing list previously > >>> ( > >>> > > > > http://mail-archives.apache.org/mod_mbox/kafka-users/201609. > >>> mbox/bro > >>> > > > > wser > >>> > > > ) > >>> > > > > but the problem has become considerably more serious. > >>> > > > > > >>> > > > > We've been fighting issues where Kafka 0.10.0.1 hits its max > >>> > > > > file descriptor limit. Our limit is set to ~16k, and under > >>> > > > > normal operation > >>> > > > it > >>> > > > > holds steady around 4k open files. > >>> > > > > > >>> > > > > But occasionally Kafka will roll a new log segment, which > >>> typically > >>> > > > > takes on the order of magnitude of a few milliseconds. > >>> > > > > However...sometimes it will take a considerable amount of > >>> > > > > time, > >>> any > >>> > > > > where from 40 seconds up to over a minute. When this > >>> > > > > happens, it seems like connections are not released by > >>> > > > > kafka, and we end up > >>> with > >>> > > > > thousands of client connections > >>> > > > stuck > >>> > > > > in CLOSE_WAIT, which pile up and exceed our max file > >>> > > > > descriptor > >>> > limit. > >>> > > > > This happens all in the span of about a minute. > >>> > > > > > >>> > > > > Our logs look like this: > >>> > > > > > >>> > > > > [2017-01-08 01:10:17,117] INFO Rolled new log segment for > >>> > > > > 'MyTopic-8' in > >>> > > > > > 41122 ms. (kafka.log.Log) > >>> > > > > > [2017-01-08 01:10:32,550] INFO Rolled new log segment for > >>> > 'MyTopic-4' > >>> > > > in > >>> > > > > 1 > >>> > > > > > ms. (kafka.log.Log) > >>> > > > > > [2017-01-08 01:11:10,039] INFO [Group Metadata Manager on > >>> Broker > >>> > 4]: > >>> > > > > > Removed 0 expired offsets in 0 milliseconds. > >>> > > > > > (kafka.coordinator.GroupMetadataManager) > >>> > > > > > [2017-01-08 01:19:02,877] ERROR Error while accepting > >>> connection > >>> > > > > > (kafka.network.Acceptor) > >>> > > > > > java.io.IOException: Too many open files at > >>> > > > > > sun.nio.ch.ServerSocketChannelImpl.accept0(Native Method) > >>> > > > > > > >>> > > > > at > >>> > > > > > sun.nio.ch.ServerSocketChannelImpl.accept( > >>> > > > ServerSocketChannelImpl.java: > >>> > > > > 422) > >>> > > > > > at > >>> > > > > > sun.nio.ch.ServerSocketChannelImpl.accept( > >>> > > > ServerSocketChannelImpl.java: > >>> > > > > 250) > >>> > > > > > at kafka.network.Acceptor.accept( > >>> SocketServer.scala:323) > >>> > > > > > at kafka.network.Acceptor.run( > SocketServer.scala:268) > >>> > > > > > at java.lang.Thread.run(Thread.java:745) > >>> > > > > > [2017-01-08 01:19:02,877] ERROR Error while accepting > >>> connection > >>> > > > > > (kafka.network.Acceptor) > >>> > > > > > java.io.IOException: Too many open files > >>> > > > > > at > >>> > > > > > sun.nio.ch.ServerSocketChannelImpl.accept0(Native > >>> > Method) > >>> > > > > > at > >>> > > > > > sun.nio.ch.ServerSocketChannelImpl.accept( > >>> > > > ServerSocketChannelImpl.java: > >>> > > > > 422) > >>> > > > > > at > >>> > > > > > sun.nio.ch.ServerSocketChannelImpl.accept( > >>> > > > ServerSocketChannelImpl.java: > >>> > > > > 250) > >>> > > > > > at kafka.network.Acceptor.accept( > >>> SocketServer.scala:323) > >>> > > > > > at kafka.network.Acceptor.run( > SocketServer.scala:268) > >>> > > > > > at java.lang.Thread.run(Thread.java:745) > >>> > > > > > ..... > >>> > > > > > > >>> > > > > > >>> > > > > > >>> > > > > And then kafka crashes. > >>> > > > > > >>> > > > > Has anyone seen this behavior of slow log segmented being > rolled? > >>> > > > > Any ideas of how to track down what could be causing this? > >>> > > > > > >>> > > > > Thanks! > >>> > > > > Stephen > >>> > > > > > >>> > > > > >>> > > This e-mail and any files transmitted with it are confidential, > >>> > > may contain sensitive information, and are intended solely for > >>> > > the use > >>> of the > >>> > > individual or entity to whom they are addressed. If you have > >>> > > received > >>> > this > >>> > > e-mail in error, please notify the sender by reply e-mail > >>> immediately and > >>> > > destroy all copies of the e-mail and any attachments. > >>> > > > >>> > > >>> > >> > >> > > > This e-mail and any files transmitted with it are confidential, may > contain sensitive information, and are intended solely for the use of the > individual or entity to whom they are addressed. If you have received this > e-mail in error, please notify the sender by reply e-mail immediately and > destroy all copies of the e-mail and any attachments. >