From issues-return-197693-apmail-spark-issues-archive=spark.apache.org@spark.apache.org Wed Aug 1 11:01:05 2018 Return-Path: X-Original-To: apmail-spark-issues-archive@minotaur.apache.org Delivered-To: apmail-spark-issues-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 15A1E1879D for ; Wed, 1 Aug 2018 11:01:05 +0000 (UTC) Received: (qmail 23943 invoked by uid 500); 1 Aug 2018 11:01:05 -0000 Delivered-To: apmail-spark-issues-archive@spark.apache.org Received: (qmail 23910 invoked by uid 500); 1 Aug 2018 11:01:05 -0000 Mailing-List: contact issues-help@spark.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list issues@spark.apache.org Received: (qmail 23901 invoked by uid 99); 1 Aug 2018 11:01:04 -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, 01 Aug 2018 11:01:04 +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 4360DCDADF for ; Wed, 1 Aug 2018 11:01:04 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -110.301 X-Spam-Level: X-Spam-Status: No, score=-110.301 tagged_above=-999 required=6.31 tests=[ENV_AND_HDR_SPF_MATCH=-0.5, RCVD_IN_DNSWL_MED=-2.3, SPF_PASS=-0.001, USER_IN_DEF_SPF_WL=-7.5, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id OO4AfeJio7pQ for ; Wed, 1 Aug 2018 11:01:02 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTP id 5E9355F3ED for ; Wed, 1 Aug 2018 11:01:01 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 989EEE25AA for ; Wed, 1 Aug 2018 11:01:00 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 16BCF27752 for ; Wed, 1 Aug 2018 11:01:00 +0000 (UTC) Date: Wed, 1 Aug 2018 11:01:00 +0000 (UTC) From: "Yuval Itzchakov (JIRA)" To: issues@spark.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Updated] (SPARK-24987) Kafka Cached Consumer Leaking File Descriptors MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: quoted-printable X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/SPARK-24987?page=3Dcom.atlassi= an.jira.plugin.system.issuetabpanels:all-tabpanel ] Yuval Itzchakov updated SPARK-24987: ------------------------------------ Summary: Kafka Cached Consumer Leaking File Descriptors (was: Kafka Ca= ched Consumer Leaking Consumers) > Kafka Cached Consumer Leaking File Descriptors > ---------------------------------------------- > > Key: SPARK-24987 > URL: https://issues.apache.org/jira/browse/SPARK-24987 > Project: Spark > Issue Type: Bug > Components: Structured Streaming > Affects Versions: 2.3.0, 2.3.1 > Environment: Spark 2.3.1 > Java(TM) SE Runtime Environment (build 1.8.0_112-b15) > Java HotSpot(TM) 64-Bit Server VM (build 25.112-b15, mixed mode) > =C2=A0 > Reporter: Yuval Itzchakov > Priority: Critical > > Setup: > * Spark 2.3.1 > * Java 1.8.0 (112) > * Standalone Cluster Manager > * 3 Nodes, 1 Executor per=C2=A0node. > Spark 2.3.0 introduced a new mechanism for caching Kafka consumers (https= ://issues.apache.org/jira/browse/SPARK-23623?page=3Dcom.atlassian.jira.plug= in.system.issuetabpanels%3Aall-tabpanel) via KafkaDataConsumer.acquire. > It seems that there are situations (I've been trying to debug it, haven't= been able to find the root cause as of yet) where cached consumers remain = "in use" throughout the life time of the task and are never released. This = can be identified by the following=C2=A0line of the stack trace: > at org.apache.spark.sql.kafka010.KafkaDataConsumer$.acquire(KafkaDataCons= umer.scala:460) > Which points to: > {code:java} > } else if (existingInternalConsumer.inUse) { > // If consumer is already cached but is currently in use, then return a= new consumer > NonCachedKafkaDataConsumer(newInternalConsumer) > {code} > =C2=A0Meaning the existing consumer created for that `TopicPartition` is = still in use for some reason. The weird thing is that you can see this for = very old tasks which have already finished successfully. > I've traced down this leak using file leak detector, attaching it to the = running Executor JVM process. I've emitted the list of open file descriptor= s which [you can find here|https://gist.github.com/YuvalItzchakov/cdbdd7f67= 604557fccfbcce673c49e5d], and you can see that the majority of them are epo= ll FD used by Kafka Consumers, indicating that they aren't closing. > =C2=A0Spark graph: > {code:java} > kafkaStream > =C2=A0 .load() > =C2=A0 .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") > =C2=A0 .as[(String, String)] > =C2=A0 .flatMap {...} > =C2=A0 .groupByKey(...) > =C2=A0 .mapGroupsWithState(GroupStateTimeout.ProcessingTimeTimeout())(...= ) > =C2=A0 .foreach(...) > =C2=A0 .outputMode(OutputMode.Update) > =C2=A0 .option("checkpointLocation", > sparkConfiguration.properties.checkpointDirectory) > =C2=A0 .start() > =C2=A0 .awaitTermination(){code} -- This message was sent by Atlassian JIRA (v7.6.3#76005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org For additional commands, e-mail: issues-help@spark.apache.org