You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@samza.apache.org by Zach Cox <zc...@gmail.com> on 2015/01/26 23:42:51 UTC

Changelog cleanup.policy and segment.bytes in 0.8.0

Hi - in Samza 0.8.0 it seems that the Kafka topic created for a key-value
store changelog does not have compaction enabled, as described in this jira:

https://issues.apache.org/jira/browse/SAMZA-226

If Samza creates this changelog topic, am I correct that we then later need
to run something like this to enable compaction (and smaller segment size)?

bin/kafka-topics.sh --zookeeper whatever:2181/kafka --topic
"the-changelog-topic" --alter --config cleanup.policy=compact --config
segment.bytes=1000000

Thanks,
Zach

Re: Changelog cleanup.policy and segment.bytes in 0.8.0

Posted by Chris Riccomini <cr...@apache.org>.
Hey Zach,

> At LinkedIn, do you ever let Samza or Kafka auto-create topics?

We almost always auto-create. The brokers are configured this way. We're
moving to a model where we'll have a web UI, and developers will be able to
create their topics manually with custom configuration. When we do that,
we'll disable auto-create. But for now, everything is auto-create.

With SAMZA-226, we're now in a position to have Samza create both changelog
topics and checkpoint topics when a job starts. This will work even if a
broker's auto-create feature is disabled, since Samza is actively issuing
an AdminUtils.createTopic call to Kafka.

> Are there any considerations we need to make to the number of partitions
of a changelog topic? Does that need to be the same number as anything else?

Yes, the partition count of the changelog topic must be equal to the number
of StreamTasks that you have. The StreamTask count is determined by your
input partitions. If you are consuming a topic with 8 partitions, you'll
have 8 StreamTasks, and thus need 8 partitions in your changelog. If you're
consuming from multiple streams, the partition count will be the max
partition count of all input streams (e.g. is1 has 8 partitions and is2 has
16, you'd need 16 partitions in your changelog).

I'm making the assumption that you're using the default SSPGrouper, which I
think is quite likely.

I'll let Jon comment on the segment.ms stuff.

Cheers,
Chris

On Tue, Jan 27, 2015 at 11:10 AM, Zach Cox <zc...@gmail.com> wrote:

> Thanks a bunch for that info, Jon! It is pure gold and helps a lot.
>
> At LinkedIn, do you ever let Samza or Kafka auto-create topics? Or do you
> always create them by-hand before deploying code that uses them?
>
> I understand why setting the topic config segment.bytes to smaller than 1GB
> is beneficial in a lot of cases, to allow log compaction to run more often
> on smaller data sizes. Is setting segment.ms to less than 7 days used for
> the same reason? i.e. to roll off the head segment file so it can be
> compacted sooner?
>
> Are there any considerations we need to make to the number of partitions of
> a changelog topic? Does that need to be the same number as anything else?
>
> Thanks,
> Zach
>
>
> On Mon, Jan 26, 2015 at 6:02 PM, Jon Bringhurst <
> jbringhurst@linkedin.com.invalid> wrote:
>
> > Hey Zach,
> >
> > That's correct. You probably want to look into the following topic-level
> > configs:
> >
> > cleanup.policy
> > min.cleanable.dirty.ratio
> > segment.ms
> > segment.bytes (we usually use the default value)
> >
> > Also, here's some broker configs of interest that you might want to tweak
> > (along with the settings we usually use... YMMV):
> >
> > log.cleaner.enable
> > log.cleaner.io.buffer.load.factor (0.9)
> > log.cleaner.io.buffer.size (524288)
> > log.cleaner.backoff.ms (30000)
> > log.cleaner.dedupe.buffer.size (524288000)
> > log.cleaner.io.max.bytes.per.second (1000000000000.0)
> > log.cleaner.delete.retention.ms (86400000)
> > log.cleaner.min.cleanable.ratio (0.5)
> > log.cleaner.threads (1)
> >
> > On a side note, sometimes it's nice to set min.cleanable.dirty.ratio to
> > 0.01, then view the files on disk to make sure things are working.
> >
> > -Jon
> >
> > On Jan 26, 2015, at 2:42 PM, Zach Cox <zc...@gmail.com> wrote:
> >
> > > Hi - in Samza 0.8.0 it seems that the Kafka topic created for a
> key-value
> > > store changelog does not have compaction enabled, as described in this
> > jira:
> > >
> > > https://issues.apache.org/jira/browse/SAMZA-226
> > >
> > > If Samza creates this changelog topic, am I correct that we then later
> > need
> > > to run something like this to enable compaction (and smaller segment
> > size)?
> > >
> > > bin/kafka-topics.sh --zookeeper whatever:2181/kafka --topic
> > > "the-changelog-topic" --alter --config cleanup.policy=compact --config
> > > segment.bytes=1000000
> > >
> > > Thanks,
> > > Zach
> >
> >
>

Re: Changelog cleanup.policy and segment.bytes in 0.8.0

Posted by Zach Cox <zc...@gmail.com>.
Thanks a bunch for that info, Jon! It is pure gold and helps a lot.

At LinkedIn, do you ever let Samza or Kafka auto-create topics? Or do you
always create them by-hand before deploying code that uses them?

I understand why setting the topic config segment.bytes to smaller than 1GB
is beneficial in a lot of cases, to allow log compaction to run more often
on smaller data sizes. Is setting segment.ms to less than 7 days used for
the same reason? i.e. to roll off the head segment file so it can be
compacted sooner?

Are there any considerations we need to make to the number of partitions of
a changelog topic? Does that need to be the same number as anything else?

Thanks,
Zach


On Mon, Jan 26, 2015 at 6:02 PM, Jon Bringhurst <
jbringhurst@linkedin.com.invalid> wrote:

> Hey Zach,
>
> That's correct. You probably want to look into the following topic-level
> configs:
>
> cleanup.policy
> min.cleanable.dirty.ratio
> segment.ms
> segment.bytes (we usually use the default value)
>
> Also, here's some broker configs of interest that you might want to tweak
> (along with the settings we usually use... YMMV):
>
> log.cleaner.enable
> log.cleaner.io.buffer.load.factor (0.9)
> log.cleaner.io.buffer.size (524288)
> log.cleaner.backoff.ms (30000)
> log.cleaner.dedupe.buffer.size (524288000)
> log.cleaner.io.max.bytes.per.second (1000000000000.0)
> log.cleaner.delete.retention.ms (86400000)
> log.cleaner.min.cleanable.ratio (0.5)
> log.cleaner.threads (1)
>
> On a side note, sometimes it's nice to set min.cleanable.dirty.ratio to
> 0.01, then view the files on disk to make sure things are working.
>
> -Jon
>
> On Jan 26, 2015, at 2:42 PM, Zach Cox <zc...@gmail.com> wrote:
>
> > Hi - in Samza 0.8.0 it seems that the Kafka topic created for a key-value
> > store changelog does not have compaction enabled, as described in this
> jira:
> >
> > https://issues.apache.org/jira/browse/SAMZA-226
> >
> > If Samza creates this changelog topic, am I correct that we then later
> need
> > to run something like this to enable compaction (and smaller segment
> size)?
> >
> > bin/kafka-topics.sh --zookeeper whatever:2181/kafka --topic
> > "the-changelog-topic" --alter --config cleanup.policy=compact --config
> > segment.bytes=1000000
> >
> > Thanks,
> > Zach
>
>

Re: Changelog cleanup.policy and segment.bytes in 0.8.0

Posted by Jon Bringhurst <jb...@linkedin.com.INVALID>.
Hey Zach,

That's correct. You probably want to look into the following topic-level configs:

cleanup.policy
min.cleanable.dirty.ratio
segment.ms
segment.bytes (we usually use the default value)

Also, here's some broker configs of interest that you might want to tweak (along with the settings we usually use... YMMV):

log.cleaner.enable
log.cleaner.io.buffer.load.factor (0.9)
log.cleaner.io.buffer.size (524288)
log.cleaner.backoff.ms (30000)
log.cleaner.dedupe.buffer.size (524288000)
log.cleaner.io.max.bytes.per.second (1000000000000.0)
log.cleaner.delete.retention.ms (86400000)
log.cleaner.min.cleanable.ratio (0.5)
log.cleaner.threads (1)

On a side note, sometimes it's nice to set min.cleanable.dirty.ratio to 0.01, then view the files on disk to make sure things are working.

-Jon

On Jan 26, 2015, at 2:42 PM, Zach Cox <zc...@gmail.com> wrote:

> Hi - in Samza 0.8.0 it seems that the Kafka topic created for a key-value
> store changelog does not have compaction enabled, as described in this jira:
> 
> https://issues.apache.org/jira/browse/SAMZA-226
> 
> If Samza creates this changelog topic, am I correct that we then later need
> to run something like this to enable compaction (and smaller segment size)?
> 
> bin/kafka-topics.sh --zookeeper whatever:2181/kafka --topic
> "the-changelog-topic" --alter --config cleanup.policy=compact --config
> segment.bytes=1000000
> 
> Thanks,
> Zach