One caveat. If you are relying on log.segment.ms to roll the current log
segment, it will not roll until the both time elapses and something new
arrives for the log.

In other words, if your topic/log segment are idle, no rolling will happen.
The theoretically ineligible log will still be the current open log segment.

On Mon, Sep 21, 2015 at 10:33 PM, Todd Palino <tpal...@gmail.com> wrote:

> Retention is going to be based on a combination of both the retention and
> segment size settings (as a side note, it's recommended to use
> log.retention.ms and log.segment.ms, not the hours config. That's there
> for
> legacy reasons, but the ms configs are more consistent). As messages are
> received by Kafka, they are written to the current open log segment for
> each partition. That segment is rotated when either the log.segment.bytes
> or the log.segment.ms limit is reached. Once that happens, the log segment
> is closed and a new one is opened. Only after a log segment is closed can
> it be deleted via the retention settings. Once the log segment is closed
> AND either all the messages in the segment are older than log.retention.ms
> OR the total partition size is greater than log.retention.bytes, then the
> log segment is purged.
>
> As a note, the default segment limit is 1 gibibyte. So if you've only
> written in 1k of messages, you have a long way to go before that segment
> gets rotated. This is why the retention is referred to as a minimum time.
> You can easily retain much more than you're expecting for slow topics.
>
> -Todd
>
>
> On Mon, Sep 21, 2015 at 7:28 PM, allen chan <allen.michael.c...@gmail.com>
> wrote:
>
> > I guess that kind of makes sense.
> > The following section in the config is what confused me:
> > *"# The following configurations control the disposal of log segments.
> The
> > policy can*
> > *# be set to delete segments after a period of time, or after a given
> size
> > has accumulated.*
> > *# A segment will be deleted whenever *either* of these criteria are met.
> > Deletion always happens*
> > *# from the end of the log."*
> >
> > That makes it sound like deletion will happen if either of the criteria
> is
> > met.
> > I thought the whole idea of those two settings (time and bytes) is
> telling
> > the application when it will need to delete.
> >
> >
> >
> > On Mon, Sep 21, 2015 at 7:10 PM, noah <iamn...@gmail.com> wrote:
> >
> > > "minimum age of a log file to be eligible for deletion" Key word is
> > > minimum. If you only have 1k logs, Kafka doesn't need to delete
> anything.
> > > Try to push more data through and when it needs to, it will start
> > deleting
> > > old logs.
> > >
> > > On Mon, Sep 21, 2015 at 8:58 PM allen chan <
> allen.michael.c...@gmail.com
> > >
> > > wrote:
> > >
> > > > Hi,
> > > >
> > > > Just brought up new kafka cluster for testing.
> > > > Was able to use the console producers to send 1k of logs and received
> > it
> > > on
> > > > the console consumer side.
> > > >
> > > > The one issue that i have right now is that the retention period does
> > not
> > > > seem to be working.
> > > >
> > > > *# The minimum age of a log file to be eligible for deletion*
> > > > *log.retention.hours=1*
> > > >
> > > > I have waited for almost 2 hours and the 1k of logs are still in
> kafka.
> > > >
> > > > I did see these messages pop up on the console
> > > > *[2015-09-21 17:12:01,236] INFO Scheduling log segment 0 for log
> test-1
> > > for
> > > > deletion. (kafka.log.Log)*
> > > > *[2015-09-21 17:13:01,238] INFO Deleting segment 0 from log test-1.
> > > > (kafka.log.Log)*
> > > > *[2015-09-21 17:13:01,239] INFO Deleting index
> > > > /var/log/kafka/test-1/00000000000000000000.index.deleted
> > > > (kafka.log.OffsetIndex)*
> > > >
> > > > I know the logs are still in there because i am using
> > > > the kafka-consumer-offset-checker.sh and it says how many messages
> the
> > > > logSize is.
> > > >
> > > > What am i missing in my configuration?
> > > >
> > > >
> > > >
> > > > Thanks!
> > > >
> > > > --
> > > > Allen Michael Chan
> > > >
> > >
> >
> >
> >
> > --
> > Allen Michael Chan
> >
>

Reply via email to