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 B10A1200BD2 for ; Sat, 19 Nov 2016 04:03:26 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id AF7D4160B16; Sat, 19 Nov 2016 03:03:26 +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 D282C160B04 for ; Sat, 19 Nov 2016 04:03:25 +0100 (CET) Received: (qmail 7140 invoked by uid 500); 19 Nov 2016 03:03:24 -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 7128 invoked by uid 99); 19 Nov 2016 03:03:24 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd2-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 19 Nov 2016 03:03:24 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd2-us-west.apache.org (ASF Mail Server at spamd2-us-west.apache.org) with ESMTP id 305C41A0371 for ; Sat, 19 Nov 2016 03:03:24 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 3.381 X-Spam-Level: *** X-Spam-Status: No, score=3.381 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, KAM_INFOUSMEBIZ=0.75, 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, URIBL_BLOCKED=0.001, WEIRD_PORT=0.001] autolearn=disabled Authentication-Results: spamd2-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 (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id kHrDQLlG963p for ; Sat, 19 Nov 2016 03:03:20 +0000 (UTC) Received: from mail-ua0-f178.google.com (mail-ua0-f178.google.com [209.85.217.178]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTPS id 5BC9A5F295 for ; Sat, 19 Nov 2016 03:03:20 +0000 (UTC) Received: by mail-ua0-f178.google.com with SMTP id 51so186104475uai.1 for ; Fri, 18 Nov 2016 19:03:20 -0800 (PST) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=20120113; h=mime-version:in-reply-to:references:from:date:message-id:subject:to; bh=Xcg6u4X5BemD7kx42uiZVxemrS4VyXlR6c2T5JZcEzM=; b=op5Ir8RIzkdEeZfC8XwMPuW9Z+2rRW0d2bjCOUctyf2DXiSmtUZlnES6KtZUBnH/Mc mB9hEUHfx66i7mlz6F9Pu645zP1gRgi2+/nOK810vUzi+F2dLu1wKKdC9RvtiT8ro75R rVeOT4KkgaMeCT7zmtwactS13ZsP0vLr6/upz+wuS1KJsM2rYakTp7j5uqPkC9KmhM3B Rn0lCaATCbdQdfc8EuVWSCXbyMT8aAUy4Z++IJGJVqexjKDFEA3rXbXI6Nn3uMm3L89e U9yWf4mzTl/84g3b1dhyFJfkiOr834Etu884XGp+1fQGZfLbDdL64gYwO5Z/5uBNkn5j rA3Q== X-Google-DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=1e100.net; s=20130820; h=x-gm-message-state:mime-version:in-reply-to:references:from:date :message-id:subject:to; bh=Xcg6u4X5BemD7kx42uiZVxemrS4VyXlR6c2T5JZcEzM=; b=EdxJWukGGTe6T6N8u8gbLCVVmSE6HO76N60r9CPeA5VmBR7P3tokYQyLXLT4yP44gM 4k9V5XIWsCCxqR+eucUl55LxvNcg4pIp8oyvuyo56ulDNwjAQGCEC6sn+49gN7hCjDnQ AoEeuCvgFzvvFn5LaCCTE5jfIwwTcB9cVva3C5+r3sZ938u4X4UUU9Qc4VTNH7XIGgXr NYMs9CsGswGP3V4jV/nlsyABkOcjKksRtciZkruWBacZr5j5wZi3D/TRaMO++cqP7gFa +XQbzS7ah9D1CqPDxePHMiIwZLbtH5WzLtCtBi5F77Eli9K4iHTv7HTybgoUfSOdjCxP SyPQ== X-Gm-Message-State: AKaTC03tBDGhS8LrPlf537ptfoqbBmjkBVR070hjgAvwF0jHykIvojsX+WOaAGDexc678OIn+6IWK1ADXlr7Lg== X-Received: by 10.159.36.244 with SMTP id 107mr1487042uar.51.1479524599803; Fri, 18 Nov 2016 19:03:19 -0800 (PST) MIME-Version: 1.0 Received: by 10.176.0.74 with HTTP; Fri, 18 Nov 2016 19:03:19 -0800 (PST) In-Reply-To: References: From: Jagadish Venkatraman Date: Fri, 18 Nov 2016 19:03:19 -0800 Message-ID: Subject: Re: Samza job having failed containers in YARN To: dev@samza.apache.org Content-Type: multipart/alternative; boundary=001a113f0fa43534c605419eaa46 archived-at: Sat, 19 Nov 2016 03:03:26 -0000 --001a113f0fa43534c605419eaa46 Content-Type: text/plain; charset=UTF-8 Hi Suraj, A couple of questions: 1. Do you write your output to kafka at all? (For example, is it possible for a message you write to an output topic to be more than 1M?) 2. Is it possible that an entry written to the changelog is more than 1M? (Possibly, because you had a > 1M write to the local rocksdb, that ended up writing the message to the changelog). You can re-produce this easily (if this is the cause). 3. Just to confirm, you can explicitly disable metrics reporters by the `metrics.reporters` property. Also, remove all configs that refer to org.apache.samza.metrics.reporter.MetricsSnapshotReporterFactory. On Fri, Nov 18, 2016 at 2:53 PM, Suraj choudhary wrote: > Hey there, > > We are having a Samza job running in YARN which reads from Kafka and writes > to Cassandra. We are also enabling the job to write the metrics to a Kafka > topic, the changelogs for the RocksDB stores and checkpointing of offsets > are also written to Kafka. > > We are having failed containers in YARN for this job. The job is having a > total of 4 containers but one container fails and the newly allocated > containers by YARN keep failing. > > Here is the stack trace from one of the failed containers: > > 2016-11-18 22:01:10,930 62058 [main] ERROR > o.a.s.s.kafka.KafkaSystemProducer - Unable to send message from > TaskName-Partition 6 to system kafka > 2016-11-18 22:01:10,943 62071 [main] ERROR > o.a.samza.container.SamzaContainer - Caught exception in process loop. > org.apache.samza.SamzaException: Unable to send message from > TaskName-Partition 6 to system kafka. > at > org.apache.samza.system.kafka.KafkaSystemProducer$$anonfun$ > flush$1.apply$mcV$sp(KafkaSystemProducer.scala:152) > at > org.apache.samza.system.kafka.KafkaSystemProducer$$anonfun$flush$1.apply( > KafkaSystemProducer.scala:136) > at > org.apache.samza.system.kafka.KafkaSystemProducer$$anonfun$flush$1.apply( > KafkaSystemProducer.scala:136) > at > org.apache.samza.util.TimerUtils$class.updateTimer(TimerUtils.scala:37) > at > org.apache.samza.system.kafka.KafkaSystemProducer.updateTimer( > KafkaSystemProducer.scala:39) > at > org.apache.samza.system.kafka.KafkaSystemProducer.flush( > KafkaSystemProducer.scala:136) > at > org.apache.samza.system.SystemProducers$$anonfun$ > flush$2.apply(SystemProducers.scala:64) > at > org.apache.samza.system.SystemProducers$$anonfun$ > flush$2.apply(SystemProducers.scala:64) > at scala.collection.Iterator$class.foreach(Iterator.scala:727) > at scala.collection.AbstractIterator.foreach(Iterator.scala:1157) > at > scala.collection.MapLike$DefaultValuesIterable.foreach(MapLike.scala:206) > at > org.apache.samza.system.SystemProducers.flush(SystemProducers.scala:64) > at > org.apache.samza.task.TaskInstanceCollector.flush( > TaskInstanceCollector.scala:70) > at > org.apache.samza.container.TaskInstance.commit(TaskInstance.scala:182) > at > org.apache.samza.container.RunLoop$$anonfun$commit$1$$ > anonfun$apply$mcVJ$sp$7.apply(RunLoop.scala:162) > at > org.apache.samza.container.RunLoop$$anonfun$commit$1$$ > anonfun$apply$mcVJ$sp$7.apply(RunLoop.scala:162) > at scala.collection.Iterator$class.foreach(Iterator.scala:727) > at scala.collection.AbstractIterator.foreach(Iterator.scala:1157) > at > scala.collection.MapLike$DefaultValuesIterable.foreach(MapLike.scala:206) > at > org.apache.samza.container.RunLoop$$anonfun$commit$1. > apply$mcVJ$sp(RunLoop.scala:162) > at > org.apache.samza.util.TimerUtils$class.updateTimerAndGetDuration( > TimerUtils.scala:51) > at > org.apache.samza.container.RunLoop.updateTimerAndGetDuration( > RunLoop.scala:35) > at org.apache.samza.container.RunLoop.commit(RunLoop.scala:157) > at org.apache.samza.container.RunLoop.run(RunLoop.scala:76) > at > org.apache.samza.container.SamzaContainer.run(SamzaContainer.scala:553) > at > org.apache.samza.container.SamzaContainer$.safeMain( > SamzaContainer.scala:92) > at > org.apache.samza.container.SamzaContainer$.main(SamzaContainer.scala:66) > at > org.apache.samza.container.SamzaContainer.main(SamzaContainer.scala) > Caused by: org.apache.kafka.common.errors.RecordTooLargeException: The > request included a message larger than the max message size the server will > accept. > > We tried turning off the metrics to be written to Kafka but still having > the problem. We enabled the 'trace' level debugging for the > 'org.apache.samza' package and found following additional info: > > 2016-11-18 22:01:10,323 61451 [main] INFO o.a.k.c.producer.ProducerConfig > - ProducerConfig values: > compression.type = none > metric.reporters = [] > metadata.max.age.ms = 300000 > metadata.fetch.timeout.ms = 60000 > acks = all > batch.size = 16384 > reconnect.backoff.ms = 10 > bootstrap.servers = [usw2a-daalt-an-kaf-int5.prsn.us:9092, > usw2b-daalt-an-kaf-int5.prsn.us:9092, usw2c-daalt-an-kaf-int5.prsn.us:9092 > ] > receive.buffer.bytes = 32768 > retry.backoff.ms = 100 > buffer.memory = 33554432 > timeout.ms = 30000 > key.serializer = class > org.apache.kafka.common.serialization.ByteArraySerializer > retries = 2147483647 > max.request.size = 1048576 > block.on.buffer.full = true > value.serializer = class > org.apache.kafka.common.serialization.ByteArraySerializer > metrics.sample.window.ms = 30000 > send.buffer.bytes = 131072 > max.in.flight.requests.per.connection = 1 > metrics.num.samples = 2 > linger.ms = 0 > client.id = > samza_checkpoint_manager-course_section_analytics-1-1479506410048-6 > > > Which is compelling us to believe that it is still producing the metrics in > spite of us disabling the metrics writing to Kafka. > > Any help from you guys would be much appreciated. Have a nice weekend! > > > Thanks, > Suraj Choudhary > -- Jagadish V, Graduate Student, Department of Computer Science, Stanford University --001a113f0fa43534c605419eaa46--