Awesome that you have figured it out! Just a general notice: any logcompact
topic used in Samza may see this slow-down if the Kafka log cleaner thread
dies, which include checkpoint, coordinator stream, and changelog topics.

Best!

-Yi

On Thu, Oct 19, 2017 at 12:14 PM, XiaoChuan Yu <xiaochuan...@kik.com> wrote:

> Hi,
>
> We were finally able to find out why the job takes so long to start.
> There was higher than normal network IO during job startup and so we
> checked size of the checkpoint topic on disk and it was ~21GB.
> We then restarted the Kafka node who was the leader for the checkpoint
> topic, the topic disk size went down to ~1.8GB and the job started up
> fairly quickly.
> Its probably due to a bug in Kafka where log cleaner died and we never
> noticed: https://issues.apache.org/jira/browse/KAFKA-3894.
> We have since been working on upgrading Kafka to avoid this bug.
> Hope this helps if anyone else ever runs into it.
>
> Thanks,
> Xiaochuan Yu
>
> On Sat, Sep 23, 2017 at 6:17 PM XiaoChuan Yu <xiaochuan...@kik.com> wrote:
>
> > >> How long does it take?
> > It took around 10 minute from "Got offset 0 for topic <checkpoint topic>
> > ..." to init() being called on the Task.
> >
> > >> Have you measured which parts of the start up sequence take the most
> > time?
> > >> - is it checkpoint restoration, or restore of local state?
> > Should be checkpoint restoration. There is no local state for this job.
> >
> > >> If reading from the checkpoint topic takes the most time, then I'd
> > >> recommend reading from the beginning from that topic, and benchmarking
> > how
> > >> long it takes? It'll also help to verify if the checkpoint topic is
> > >> actually log-compacted.
> > I'm not sure how to verify how much the topic is compacted by Kafka.
> > The cleanup policy is to compact though.
> >
> > >> Do containers eventually start? Or does the start-up hang?
> > >> If so, a thread dump will be useful.
> > It does eventually start up.
> >
> > >> Can you please link and attach the entire log file for us to take a
> > look?
> > Unfortunately there is too much stuff for me to redact from the log right
> > now.
> > However, I can tell you that the job has two input topics both with the
> > following settings:
> > systems.kafka.streams.my-special-topic.samza.reset.offset=true
> > systems.kafka.streams.my-special-topic.samza.offset.default=upcoming
> > It was thought that this would speedup startup of the job to no avail.
> >
> > On Wed, Sep 20, 2017 at 3:21 PM Jagadish Venkatraman <
> > jagadish1...@gmail.com> wrote:
> >
> >> Hi Xiaochuan,
> >>
> >> >> What does that loop do exactly?
> >>
> >> Most of what the run-loop does is documented in
> >> https://samza.apache.org/learn/documentation/0.9/
> container/event-loop.html
> >>
> >> >> We are running into a problem where it seems to take a very long time
> >> to
> >> restart a Samza job.
> >>
> >> Some follow-up questions,
> >>
> >> How long does it take?
> >> Have you measured which parts of the start up sequence take the most
> time?
> >> - is it checkpoint restoration, or restore of local state?
> >> If reading from the checkpoint topic takes the most time, then I'd
> >> recommend reading from the beginning from that topic, and benchmarking
> how
> >> long it takes? It'll also help to verify if the checkpoint topic is
> >> actually log-compacted.
> >> Do containers eventually start? Or does the start-up hang? If so, a
> thread
> >> dump will be useful.
> >> Can you please link and attach the entire log file for us to take a
> look?
> >>
> >> >> 3. Any ideas on how to fix this?
> >>
> >> We can perhaps, try to narrow down where the time is spent in startup
> from
> >> the logs? Depending on that, I can suggest a fix :-)
> >>
> >> Thanks,
> >> Jagadish
> >>
> >> On Wed, Sep 20, 2017 at 11:21 AM, XiaoChuan Yu <xiaochuan...@kik.com>
> >> wrote:
> >>
> >> > Hi,
> >> >
> >> > We are running into a problem where it seems to take a very long time
> to
> >> > restart a Samza job.
> >> > We are using Samza 0.9.1 at the moment.
> >> >
> >> > From the logs for a particular container it looks like it has
> something
> >> to
> >> > do with reading checkpoints from Kafka:
> >> >
> >> > 2017-09-20 03:21:02.060 INFO  o.a.s.c.kafka.KafkaCheckpointManager
> >> [main]
> >> > -
> >> > Got offset 0 for topic __samza_checkpoint_ver_1_for_test-job_1 and
> >> > partition 0. Attempting to fetch messages for checkpoint log.
> >> > 2017-09-20 03:21:02.072 INFO  o.a.s.c.kafka.KafkaCheckpointManager
> >> [main]
> >> > -
> >> > Get latest offset 42890599 for topic
> >> > __samza_checkpoint_ver_1_for_test-job_1 and partition 0.
> >> >
> >> > Looking at this line in KafkaCheckpointManager
> >> > <https://github.com/apache/samza/blob/0.9.1/samza-kafka/
> >> > src/main/scala/org/apache/samza/checkpoint/kafka/
> >> > KafkaCheckpointManager.scala#L275>,
> >> > it seems to indicate that the loop iterates from 0 to 42890599 and
> make
> >> > requests for each.
> >> >
> >> > Questions:
> >> > 1. What does that loop do exactly?
> >> > 2. Is this an expected behaviour? Is "Got offset 0 for topic ..."
> >> normal?
> >> > 3. Any ideas on how to fix this?
> >> >
> >> > Thanks,
> >> > Xiaochuan Yu
> >> >
> >>
> >>
> >>
> >> --
> >> Jagadish V,
> >> Graduate Student,
> >> Department of Computer Science,
> >> Stanford University
> >>
> >
>

Reply via email to