Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id AE5D0200D0C for ; Wed, 20 Sep 2017 21:21:46 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id ACA5F1609E2; Wed, 20 Sep 2017 19:21:46 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id EC0011609D8 for ; Wed, 20 Sep 2017 21:21:45 +0200 (CEST) Received: (qmail 68721 invoked by uid 500); 20 Sep 2017 19:21:45 -0000 Mailing-List: contact dev-help@samza.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@samza.apache.org Delivered-To: mailing list dev@samza.apache.org Received: (qmail 68697 invoked by uid 99); 20 Sep 2017 19:21:44 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 20 Sep 2017 19:21:44 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id 3DB20C552F for ; Wed, 20 Sep 2017 19:21:44 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 2.629 X-Spam-Level: ** X-Spam-Status: No, score=2.629 tagged_above=-999 required=6.31 tests=[DKIM_SIGNED=0.1, DKIM_VALID=-0.1, DKIM_VALID_AU=-0.1, FREEMAIL_ENVFROM_END_DIGIT=0.25, HTML_MESSAGE=2, RCVD_IN_DNSWL_NONE=-0.0001, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, RCVD_IN_SORBS_SPAM=0.5, SPF_PASS=-0.001] autolearn=disabled Authentication-Results: spamd1-us-west.apache.org (amavisd-new); dkim=pass (2048-bit key) header.d=gmail.com Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id N_IjDTdM_d19 for ; Wed, 20 Sep 2017 19:21:42 +0000 (UTC) Received: from mail-qk0-f174.google.com (mail-qk0-f174.google.com [209.85.220.174]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTPS id 215B55FBDF for ; Wed, 20 Sep 2017 19:21:42 +0000 (UTC) Received: by mail-qk0-f174.google.com with SMTP id j5so3765373qkd.0 for ; Wed, 20 Sep 2017 12:21:42 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=20161025; h=mime-version:in-reply-to:references:from:date:message-id:subject:to; bh=jr75AP5zZkMJRYzUaug0qrt5l16qmOuV18EE+zXYjS0=; b=l4ZEtrRMRV/jf3Az11lwzWQ7NhZvMMPVtFEWMTadPTarxls85aQXBmiAmEBoJwQAQq wLC3BvMa1DBPRN5ZGv8sb9fa3fAS6nmhADfAa6FCeSSczR4ns9ovCUHvdqukoisd/CQk D/+V7bNw5vWgXZe5opOH955BD2dF9DcXjSD/i5L1o1HN2RpvRnVwL28TQblH4/dneEQV 96rEuzdQo0lwxRGSMJSW7cyA7RI/GC9yrVs1nJlDy+h6Nf009NIv4dpdTQ694ZOlHw+a 1nlruGt0bK5yyY0XBaUn9+mZD1/GQjfYzV6thFoeqVB5AXk4YlFUSSWbugqttsjL0QAg 2t5w== X-Google-DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=1e100.net; s=20161025; h=x-gm-message-state:mime-version:in-reply-to:references:from:date :message-id:subject:to; bh=jr75AP5zZkMJRYzUaug0qrt5l16qmOuV18EE+zXYjS0=; b=hoaVCZlNBALKhInAzVOTFA0J7xicwnjKFTCMXI3NEwELQuU5zkOlow6VtiUWSnlFv4 remyN3CsowKW48qL5cHZgsGMPXSPaMMjvIpM0fZ1KAnu9eBDQvukrPNjKcOWSPW0h3HK XrU6Q+n4iOqKn+KlpJ35WF9jGZd4NsIkqlY6YUBINNeQaQGJpC+UDA+Q4MbIq11+X9wS GPsgjYS4JSh1PDDIMAIYvWQzA+tE80PX6A9mhGqgY5kZLr0CJTyo2+YnFZk8c5pLP0kL Nv0W6kJFKVGmB9obHOeQ0yz1FAow36CJP+w4hOhWolyB8qSh4g57+UnaFnbRmThMUS87 6Uxg== X-Gm-Message-State: AHPjjUgjhzNOknl40e1p8s6TSJUPx+J++2lbsUORLotn0pIWKxJZEBJU WtuW0TgIuLeparwY/Sldq6Wb3RPn7FLzta+vORClQQ== X-Google-Smtp-Source: AOwi7QD6TPDSBVC6SvdmczEQgZBopwqu26ZU1EPWujFteMFLfz39fN0fiPStO5kkkastvDT8xRRa/UeEbOZksynlsjY= X-Received: by 10.55.71.6 with SMTP id u6mr8158659qka.166.1505935301530; Wed, 20 Sep 2017 12:21:41 -0700 (PDT) MIME-Version: 1.0 Received: by 10.237.42.195 with HTTP; Wed, 20 Sep 2017 12:21:41 -0700 (PDT) In-Reply-To: References: From: Jagadish Venkatraman Date: Wed, 20 Sep 2017 12:21:41 -0700 Message-ID: Subject: Re: Samza Job Slow to Restart To: dev@samza.apache.org Content-Type: multipart/alternative; boundary="001a114a72aeb3dcb20559a3e28c" archived-at: Wed, 20 Sep 2017 19:21:46 -0000 --001a114a72aeb3dcb20559a3e28c Content-Type: text/plain; charset="UTF-8" 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 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 > 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 --001a114a72aeb3dcb20559a3e28c--