You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by Stephen Powis <sp...@salesforce.com> on 2017/01/09 20:45:50 UTC

Taking a long time to roll a new log segment (~1 min)

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/browser)
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

Re: Taking a long time to roll a new log segment (~1 min)

Posted by Stephen Powis <sp...@salesforce.com>.
Yep, that's exactly what the output is showing.

On Fri, Jan 13, 2017 at 11:25 AM, Tauzell, Dave <
Dave.Tauzell@surescripts.com> wrote:

> I haven't used dtrace, but is it possible to have it running and recording
> the ftruncate64 times?  Then when you see one of these long roll times look
> at the dtrace log to see if it was that call?
>
> -Dave
>
> -----Original Message-----
> From: Stephen Powis [mailto:spowis@salesforce.com]
> Sent: Friday, January 13, 2017 9:25 AM
> To: users@kafka.apache.org
> Subject: Re: Taking a long time to roll a new log segment (~1 min)
>
> So the underlying system call is ftruncate64, logged using dtrace.
>
> # BEGIN stack trace for ftruncate, call took 34160541200ns:
> > args==
> >  0x7f5f9a1134d7 : ftruncate64+0x7/0x30 [/lib64/libc-2.12.so]
> >  0x7f5f987b80c3 : 0x7f5f987b80c3
> > [/usr/lib/jvm/jdk1.8.0_51/jre/lib/amd64/libjava.so+0x180c3/0x22c000]
> > # END stack trace for ftruncate
> >
>
> These index files seem to be typically around 10MB for open log segments,
> and get truncated down to ~1MB after being closed.  I'd imagine truncating
> off ~9MB shouldn't be that heavy of an operation.
>
> ZFS options:
>
> > data/kafka       type                  filesystem             -
> > data/kafka       creation              Tue Feb  9 22:45 2016  -
> > data/kafka       used                  196G                   -
> > data/kafka       available             6.65T                  -
> > data/kafka       referenced            196G                   -
> > data/kafka       compressratio         4.59x                  -
> > data/kafka       mounted               yes                    -
> > data/kafka       quota                 none                   default
> > data/kafka       reservation           none                   default
> > data/kafka       recordsize            128K                   default
> > data/kafka       mountpoint            /kafka                 local
> > data/kafka       sharenfs              off                    default
> > data/kafka       checksum              on                     default
> > data/kafka       compression           lz4                    inherited
> > from data
> > data/kafka       atime                 off                    inherited
> > from data
> > data/kafka       devices               on                     default
> > data/kafka       exec                  on                     default
> > data/kafka       setuid                on                     default
> > data/kafka       readonly              off                    default
> > data/kafka       zoned                 off                    default
> > data/kafka       snapdir               hidden                 default
> > data/kafka       aclinherit            restricted             default
> > data/kafka       canmount              on                     default
> > data/kafka       xattr                 on                     default
>
> data/kafka       copies                1                      default
> > data/kafka       version               5                      -
> > data/kafka       utf8only              off                    -
> > data/kafka       normalization         none                   -
> > data/kafka       casesensitivity       sensitive              -
> > data/kafka       vscan                 off                    default
> > data/kafka       nbmand                off                    default
> > data/kafka       sharesmb              off                    default
> > data/kafka       refquota              none                   default
> > data/kafka       refreservation        none                   default
> > data/kafka       primarycache          all                    default
> > data/kafka       secondarycache        all                    default
> > data/kafka       usedbysnapshots       0                      -
> > data/kafka       usedbydataset         196G                   -
> > data/kafka       usedbychildren        0                      -
> > data/kafka       usedbyrefreservation  0                      -
> > data/kafka       logbias               throughput             local
> > data/kafka       dedup                 off                    default
> > data/kafka       mlslabel              none                   default
> > data/kafka       sync                  standard               default
> > data/kafka       refcompressratio      4.59x                  -
> > data/kafka       written               196G                   -
> > data/kafka       logicalused           900G                   -
> > data/kafka       logicalreferenced     900G                   -
> > data/kafka       filesystem_limit      none                   default
> > data/kafka       snapshot_limit        none                   default
> > data/kafka       filesystem_count      none                   default
> > data/kafka       snapshot_count        none                   default
> > data/kafka       snapdev               hidden                 default
> > data/kafka       acltype               off                    default
> > data/kafka       context               none                   default
> > data/kafka       fscontext             none                   default
> > data/kafka       defcontext            none                   default
> > data/kafka       rootcontext           none                   default
> > data/kafka       relatime              off                    default
> > data/kafka       redundant_metadata    all                    default
> > data/kafka       overlay               off                    default
> >
>
> CPU IO Wait is extremely low (below 0.01%) with no recorded spikes.
>
> On Fri, Jan 13, 2017 at 6:57 AM, Ismael Juma <is...@juma.me.uk> wrote:
>
> > Hi Stephen,
> >
> > I think it's worth capturing this in a JIRA so that it doesn't get lost.
> >
> > Ismael
> >
> > On Thu, Jan 12, 2017 at 7:22 PM, Stephen Powis <sp...@salesforce.com>
> > wrote:
> >
> > > 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
> > > <sp...@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
> > > > <spowis@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 <xi...@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
> > > >>> <sp...@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.Tauzell@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:spowis@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 <
> > > >>> > ewen@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 <
> > > >>> spowis@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.
>

RE: Taking a long time to roll a new log segment (~1 min)

Posted by "Tauzell, Dave" <Da...@surescripts.com>.
I haven't used dtrace, but is it possible to have it running and recording the ftruncate64 times?  Then when you see one of these long roll times look at the dtrace log to see if it was that call?

-Dave

-----Original Message-----
From: Stephen Powis [mailto:spowis@salesforce.com]
Sent: Friday, January 13, 2017 9:25 AM
To: users@kafka.apache.org
Subject: Re: Taking a long time to roll a new log segment (~1 min)

So the underlying system call is ftruncate64, logged using dtrace.

# BEGIN stack trace for ftruncate, call took 34160541200ns:
> args==
>  0x7f5f9a1134d7 : ftruncate64+0x7/0x30 [/lib64/libc-2.12.so]
>  0x7f5f987b80c3 : 0x7f5f987b80c3
> [/usr/lib/jvm/jdk1.8.0_51/jre/lib/amd64/libjava.so+0x180c3/0x22c000]
> # END stack trace for ftruncate
>

These index files seem to be typically around 10MB for open log segments, and get truncated down to ~1MB after being closed.  I'd imagine truncating off ~9MB shouldn't be that heavy of an operation.

ZFS options:

> data/kafka       type                  filesystem             -
> data/kafka       creation              Tue Feb  9 22:45 2016  -
> data/kafka       used                  196G                   -
> data/kafka       available             6.65T                  -
> data/kafka       referenced            196G                   -
> data/kafka       compressratio         4.59x                  -
> data/kafka       mounted               yes                    -
> data/kafka       quota                 none                   default
> data/kafka       reservation           none                   default
> data/kafka       recordsize            128K                   default
> data/kafka       mountpoint            /kafka                 local
> data/kafka       sharenfs              off                    default
> data/kafka       checksum              on                     default
> data/kafka       compression           lz4                    inherited
> from data
> data/kafka       atime                 off                    inherited
> from data
> data/kafka       devices               on                     default
> data/kafka       exec                  on                     default
> data/kafka       setuid                on                     default
> data/kafka       readonly              off                    default
> data/kafka       zoned                 off                    default
> data/kafka       snapdir               hidden                 default
> data/kafka       aclinherit            restricted             default
> data/kafka       canmount              on                     default
> data/kafka       xattr                 on                     default

data/kafka       copies                1                      default
> data/kafka       version               5                      -
> data/kafka       utf8only              off                    -
> data/kafka       normalization         none                   -
> data/kafka       casesensitivity       sensitive              -
> data/kafka       vscan                 off                    default
> data/kafka       nbmand                off                    default
> data/kafka       sharesmb              off                    default
> data/kafka       refquota              none                   default
> data/kafka       refreservation        none                   default
> data/kafka       primarycache          all                    default
> data/kafka       secondarycache        all                    default
> data/kafka       usedbysnapshots       0                      -
> data/kafka       usedbydataset         196G                   -
> data/kafka       usedbychildren        0                      -
> data/kafka       usedbyrefreservation  0                      -
> data/kafka       logbias               throughput             local
> data/kafka       dedup                 off                    default
> data/kafka       mlslabel              none                   default
> data/kafka       sync                  standard               default
> data/kafka       refcompressratio      4.59x                  -
> data/kafka       written               196G                   -
> data/kafka       logicalused           900G                   -
> data/kafka       logicalreferenced     900G                   -
> data/kafka       filesystem_limit      none                   default
> data/kafka       snapshot_limit        none                   default
> data/kafka       filesystem_count      none                   default
> data/kafka       snapshot_count        none                   default
> data/kafka       snapdev               hidden                 default
> data/kafka       acltype               off                    default
> data/kafka       context               none                   default
> data/kafka       fscontext             none                   default
> data/kafka       defcontext            none                   default
> data/kafka       rootcontext           none                   default
> data/kafka       relatime              off                    default
> data/kafka       redundant_metadata    all                    default
> data/kafka       overlay               off                    default
>

CPU IO Wait is extremely low (below 0.01%) with no recorded spikes.

On Fri, Jan 13, 2017 at 6:57 AM, Ismael Juma <is...@juma.me.uk> wrote:

> Hi Stephen,
>
> I think it's worth capturing this in a JIRA so that it doesn't get lost.
>
> Ismael
>
> On Thu, Jan 12, 2017 at 7:22 PM, Stephen Powis <sp...@salesforce.com>
> wrote:
>
> > 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
> > <sp...@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
> > > <spowis@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 <xi...@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
> > >>> <sp...@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.Tauzell@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:spowis@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 <
> > >>> > ewen@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 <
> > >>> spowis@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.

Re: Taking a long time to roll a new log segment (~1 min)

Posted by Stephen Powis <sp...@salesforce.com>.
So the underlying system call is ftruncate64, logged using dtrace.

# BEGIN stack trace for ftruncate, call took 34160541200ns:
> args==
>  0x7f5f9a1134d7 : ftruncate64+0x7/0x30 [/lib64/libc-2.12.so]
>  0x7f5f987b80c3 : 0x7f5f987b80c3
> [/usr/lib/jvm/jdk1.8.0_51/jre/lib/amd64/libjava.so+0x180c3/0x22c000]
> # END stack trace for ftruncate
>

These index files seem to be typically around 10MB for open log segments,
and get truncated down to ~1MB after being closed.  I'd imagine truncating
off ~9MB shouldn't be that heavy of an operation.

ZFS options:

> data/kafka       type                  filesystem             -
> data/kafka       creation              Tue Feb  9 22:45 2016  -
> data/kafka       used                  196G                   -
> data/kafka       available             6.65T                  -
> data/kafka       referenced            196G                   -
> data/kafka       compressratio         4.59x                  -
> data/kafka       mounted               yes                    -
> data/kafka       quota                 none                   default
> data/kafka       reservation           none                   default
> data/kafka       recordsize            128K                   default
> data/kafka       mountpoint            /kafka                 local
> data/kafka       sharenfs              off                    default
> data/kafka       checksum              on                     default
> data/kafka       compression           lz4                    inherited
> from data
> data/kafka       atime                 off                    inherited
> from data
> data/kafka       devices               on                     default
> data/kafka       exec                  on                     default
> data/kafka       setuid                on                     default
> data/kafka       readonly              off                    default
> data/kafka       zoned                 off                    default
> data/kafka       snapdir               hidden                 default
> data/kafka       aclinherit            restricted             default
> data/kafka       canmount              on                     default
> data/kafka       xattr                 on                     default

data/kafka       copies                1                      default
> data/kafka       version               5                      -
> data/kafka       utf8only              off                    -
> data/kafka       normalization         none                   -
> data/kafka       casesensitivity       sensitive              -
> data/kafka       vscan                 off                    default
> data/kafka       nbmand                off                    default
> data/kafka       sharesmb              off                    default
> data/kafka       refquota              none                   default
> data/kafka       refreservation        none                   default
> data/kafka       primarycache          all                    default
> data/kafka       secondarycache        all                    default
> data/kafka       usedbysnapshots       0                      -
> data/kafka       usedbydataset         196G                   -
> data/kafka       usedbychildren        0                      -
> data/kafka       usedbyrefreservation  0                      -
> data/kafka       logbias               throughput             local
> data/kafka       dedup                 off                    default
> data/kafka       mlslabel              none                   default
> data/kafka       sync                  standard               default
> data/kafka       refcompressratio      4.59x                  -
> data/kafka       written               196G                   -
> data/kafka       logicalused           900G                   -
> data/kafka       logicalreferenced     900G                   -
> data/kafka       filesystem_limit      none                   default
> data/kafka       snapshot_limit        none                   default
> data/kafka       filesystem_count      none                   default
> data/kafka       snapshot_count        none                   default
> data/kafka       snapdev               hidden                 default
> data/kafka       acltype               off                    default
> data/kafka       context               none                   default
> data/kafka       fscontext             none                   default
> data/kafka       defcontext            none                   default
> data/kafka       rootcontext           none                   default
> data/kafka       relatime              off                    default
> data/kafka       redundant_metadata    all                    default
> data/kafka       overlay               off                    default
>

CPU IO Wait is extremely low (below 0.01%) with no recorded spikes.

On Fri, Jan 13, 2017 at 6:57 AM, Ismael Juma <is...@juma.me.uk> wrote:

> Hi Stephen,
>
> I think it's worth capturing this in a JIRA so that it doesn't get lost.
>
> Ismael
>
> On Thu, Jan 12, 2017 at 7:22 PM, Stephen Powis <sp...@salesforce.com>
> wrote:
>
> > 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 <sp...@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 <spowis@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 <xi...@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 <sp...@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.Tauzell@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:spowis@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 <
> > >>> > ewen@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 <
> > >>> spowis@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.
> > >>> > >
> > >>> >
> > >>>
> > >>
> > >>
> > >
> >
>

Re: Taking a long time to roll a new log segment (~1 min)

Posted by Ismael Juma <is...@juma.me.uk>.
Hi Stephen,

I think it's worth capturing this in a JIRA so that it doesn't get lost.

Ismael

On Thu, Jan 12, 2017 at 7:22 PM, Stephen Powis <sp...@salesforce.com>
wrote:

> 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 <sp...@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 <sp...@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 <xi...@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 <sp...@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.Tauzell@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:spowis@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 <
> >>> > ewen@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 <
> >>> spowis@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.
> >>> > >
> >>> >
> >>>
> >>
> >>
> >
>

Re: Taking a long time to roll a new log segment (~1 min)

Posted by Ewen Cheslack-Postava <ew...@confluent.io>.
That line is truncating a file. If the underlying implementation is
synchronous, it could potentially require multiple disk writes to remove
blocks from the file, update the list of free blocks, etc. That could
potentially cause the latency.

I'm not sure that's actually where the cost is coming from, but definitely
a possible explanation for the latency appearing on that specific line.

-Ewen

On Thu, Jan 12, 2017 at 5:33 PM, Stephane Maarek <
stephane@simplemachines.com.au> wrote:

> Hi Stephen
>
> Out of curiosity, why did you pick ZFS over XFS or ext4 and what options
> are you using when formatting and mounting?
>
> Regards,
> Stephane
>
> On 13 January 2017 at 6:40:18 am, Stephen Powis (spowis@salesforce.com)
> wrote:
>
> 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.Tauzell@surescripts.com
> > wrote:
>
> > You have a local filesystem? Linux?
> >
> > -Dave
> >
> > -----Original Message-----
> > From: Stephen Powis [mailto:spowis@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 <sp...@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
> > > <sp...@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 <xi...@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 <sp...@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.Tauzell@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:spowis@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 <
> > >>> > ewen@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 <
> > >>> spowis@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.
> >
>

Re: Taking a long time to roll a new log segment (~1 min)

Posted by Stephane Maarek <st...@simplemachines.com.au>.
Hi Stephen

Out of curiosity, why did you pick ZFS over XFS or ext4 and what options
are you using when formatting and mounting?

Regards,
Stephane

On 13 January 2017 at 6:40:18 am, Stephen Powis (spowis@salesforce.com)
wrote:

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.Tauzell@surescripts.com
> wrote:

> You have a local filesystem? Linux?
>
> -Dave
>
> -----Original Message-----
> From: Stephen Powis [mailto:spowis@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 <sp...@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
> > <sp...@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 <xi...@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 <sp...@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.Tauzell@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:spowis@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 <
> >>> > ewen@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 <
> >>> spowis@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.
>

Re: Taking a long time to roll a new log segment (~1 min)

Posted by Stephen Powis <sp...@salesforce.com>.
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.Tauzell@surescripts.com
> wrote:

> You have a local filesystem? Linux?
>
> -Dave
>
> -----Original Message-----
> From: Stephen Powis [mailto:spowis@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 <sp...@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
> > <sp...@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 <xi...@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 <sp...@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.Tauzell@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:spowis@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 <
> >>> > ewen@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 <
> >>> spowis@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.
>

RE: Taking a long time to roll a new log segment (~1 min)

Posted by "Tauzell, Dave" <Da...@surescripts.com>.
You have a local filesystem? Linux?

-Dave

-----Original Message-----
From: Stephen Powis [mailto:spowis@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 <sp...@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
> <sp...@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 <xi...@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 <sp...@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.Tauzell@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:spowis@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 <
>>> > ewen@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 <
>>> spowis@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.

Re: Taking a long time to roll a new log segment (~1 min)

Posted by Stephen Powis <sp...@salesforce.com>.
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 <sp...@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 <sp...@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 <xi...@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 <sp...@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.Tauzell@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:spowis@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 <
>>> > ewen@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 <
>>> spowis@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.
>>> > >
>>> >
>>>
>>
>>
>

Re: Taking a long time to roll a new log segment (~1 min)

Posted by Stephen Powis <sp...@salesforce.com>.
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 <sp...@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 <xi...@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 <sp...@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.Tauzell@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:spowis@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 <
>> > ewen@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 <
>> spowis@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.
>> > >
>> >
>>
>
>

Re: Taking a long time to roll a new log segment (~1 min)

Posted by Stephen Powis <sp...@salesforce.com>.
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 <xi...@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 <sp...@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-gS7h4Nuq3TKgHoAVeRHPWnBNs2B0Tz
> 0kCjmdB9c0SDcLQ%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=c2hhcmVkLzIwMTcvMDEvMTIvLS10b3
> RhbEdDLWthZmthMS00LmxvZy5nei0tMTUtMzQtNTk=
> >
> >
> > 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.Tauzell@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:spowis@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 <
> > ewen@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 <
> spowis@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.
> > >
> >
>

Re: Taking a long time to roll a new log segment (~1 min)

Posted by tao xiao <xi...@gmail.com>.
You may run into this bug https://issues.apache.org/jira/browse/KAFKA-4614

On Thu, 12 Jan 2017 at 23:38 Stephen Powis <sp...@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=c2hhcmVkLzIwMTcvMDEvMTIvLS10b3RhbEdDLWthZmthMS00LmxvZy5nei0tMTUtMzQtNTk=
>
>
> 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.Tauzell@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:spowis@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 <
> ewen@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 <sp...@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.
> >
>

Re: Taking a long time to roll a new log segment (~1 min)

Posted by Stephen Powis <sp...@salesforce.com>.
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=c2hhcmVkLzIwMTcvMDEvMTIvLS10b3RhbEdDLWthZmthMS00LmxvZy5nei0tMTUtMzQtNTk=


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.Tauzell@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:spowis@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 <ew...@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 <sp...@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.
>

RE: Taking a long time to roll a new log segment (~1 min)

Posted by "Tauzell, Dave" <Da...@surescripts.com>.
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:spowis@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 <ew...@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 <sp...@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.

Re: Taking a long time to roll a new log segment (~1 min)

Posted by Stephen Powis <sp...@salesforce.com>.
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 <ew...@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 <sp...@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/browser
> )
> > 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
> >
>

Re: Taking a long time to roll a new log segment (~1 min)

Posted by Ewen Cheslack-Postava <ew...@confluent.io>.
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 <sp...@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/browser)
> 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
>