I don't know how Kafka's rollover algorithm is implemented, but this is common behavior for other logging frameworks. You would need a separate watcher/scheduled thread to rollover a log file, even if no events were coming in. Logback (and probably log4j, by the same author) dispenses with the watcher thread. Instead, it checks each message as it comes in and decides whether the message should belong in a new file. If it should, a rollover of the old file is triggered and the message is deposited in the new file. But no rollover will occur until a message that belongs in a new file arrives.
Cheers, Adam On Fri, Apr 26, 2013 at 9:52 AM, Jason Rosenberg <j...@squareup.com> wrote: > By the way, is there a reason why 'log.roll.hours' is not documented on the > apache configuration page: http://kafka.apache.org/configuration.html ? > > It's possible to find this setting (and several other undocumented > settings) by looking at the source code. I'm just not sure why the > complete set of options is not documented on the site (is it meant to be > experimental?). > > Jason > > > On Fri, Apr 26, 2013 at 8:19 AM, Dan Frankowski <dfran...@gmail.com> > wrote: > > > https://issues.apache.org/jira/browse/KAFKA-881 > > > > Thanks. > > > > > > On Fri, Apr 26, 2013 at 7:40 AM, Jun Rao <jun...@gmail.com> wrote: > > > > > Yes, for low volume topic, the time-based rolling can be imprecise. > Could > > > you file a jira and describe your suggestions there? Ideally, we should > > set > > > firstAppendTime to the file creation time. However, it doesn't seem you > > can > > > get the creation time in java. > > > > > > Thanks, > > > > > > Jun > > > > > > > > > On Thu, Apr 25, 2013 at 11:12 PM, Dan Frankowski <dfran...@gmail.com> > > > wrote: > > > > > > > We have high-volume topics and low-volume topics. The problem occurs > > more > > > > often for low-volume topics to be sure. > > > > > > > > But if my hypothesis is correct about why it is happening, here is a > > case > > > > where rolling is longer than an hour, even on a high volume topic: > > > > > > > > - write to a topic for 20 minutes > > > > - restart the broker > > > > - wait for 5 days > > > > - write to a topic for 20 minutes > > > > - restart the broker > > > > - write to a topic for an hour > > > > > > > > The rollover time was now 5 days, 1 hour, 40 minutes. You can make it > > as > > > > long as you want. Does this make sense? > > > > > > > > We would like the rollover time to be no more than an hour, even if > the > > > > broker is restarted, or the topic is low-volume. > > > > > > > > The cleanest way to do that might be to roll over on the hour no > matter > > > > when the file was started. That would be too fast sometimes, but > that's > > > > fine. A second way would be to embed the first append time in the > file > > > > name. A third way (not perfect, but an approximation at least) would > be > > > to > > > > not to write to a segment if firstAppendTime is not defined and the > > > > timestamp on the file is more than an hour old. There are probably > > other > > > > ways. > > > > > > > > What say you? > > > > > > > > > > > > On Thu, Apr 25, 2013 at 9:49 PM, Jun Rao <jun...@gmail.com> wrote: > > > > > > > > > That logic in 0.7.2 seems correct. Basically, firstAppendTime is > set > > on > > > > > first append to a log segment. Then, later on, when a new message > is > > > > > appended and the elapsed time since firstAppendTime is larger than > > the > > > > roll > > > > > time, a new segment is rolled. Is your data constantly being > > produced? > > > > > > > > > > Thanks, > > > > > > > > > > Jun > > > > > > > > > > > > > > > On Thu, Apr 25, 2013 at 12:44 PM, Dan Frankowski < > dfran...@gmail.com > > > > > > > > wrote: > > > > > > > > > > > We are running Kafka 0.7.2. We set log.roll.hours=1. I hoped that > > > meant > > > > > > logs would be rolled every hour, or more. Only, sometimes logs > that > > > are > > > > > > many hours (sometimes days) old have more data added to them. > This > > > > > perturbs > > > > > > our systems for reasons I won't get in to. > > > > > > > > > > > > Have others observed this? Is it a bug? Is there a planned fix? > > > > > > > > > > > > I don't know Scala or Kafka well, but I have proposal for why > this > > > > might > > > > > > happen: upon restart, a broker forgets when its log files have > been > > > > > > appended to ("firstAppendTime"). Then a potentially infinite > amount > > > of > > > > > time > > > > > > later, the restarted broker receives another message for the > > > particular > > > > > > (topic, partition), and starts the clock again. It will then roll > > > over > > > > > that > > > > > > log after an hour. > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > https://svn.apache.org/repos/asf/kafka/branches/0.7/core/src/main/scala/kafka/server/KafkaConfig.scalasays > > > > > > : > > > > > > > > > > > > /* the maximum time before a new log segment is rolled out */ > > > > > > val logRollHours = Utils.getIntInRange(props, "log.roll.hours", > > > 24*7, > > > > > (1, > > > > > > Int.MaxValue)) > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > https://svn.apache.org/repos/asf/kafka/branches/0.7/core/src/main/scala/kafka/log/Log.scalahas > > > > > > maybeRoll, which needs segment.firstAppendTime defined. It also > has > > > > > > updateFirstAppendTime() which says if it's empty, then set it. > > > > > > > > > > > > > > > > > > > > >