From issues-return-237639-apmail-spark-issues-archive=spark.apache.org@spark.apache.org Tue Oct 1 10:57:04 2019 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 [207.244.88.153]) by minotaur.apache.org (Postfix) with SMTP id F374219663 for ; Tue, 1 Oct 2019 10:57:03 +0000 (UTC) Received: (qmail 96320 invoked by uid 500); 1 Oct 2019 10:57:03 -0000 Delivered-To: apmail-spark-issues-archive@spark.apache.org Received: (qmail 96303 invoked by uid 500); 1 Oct 2019 10:57:03 -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 96293 invoked by uid 99); 1 Oct 2019 10:57:03 -0000 Received: from mailrelay1-us-west.apache.org (HELO mailrelay1-us-west.apache.org) (209.188.14.139) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 01 Oct 2019 10:57:03 +0000 Received: from jira-he-de.apache.org (static.172.67.40.188.clients.your-server.de [188.40.67.172]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 62E7FE2F7F for ; Tue, 1 Oct 2019 10:57:02 +0000 (UTC) Received: from jira-he-de.apache.org (localhost.localdomain [127.0.0.1]) by jira-he-de.apache.org (ASF Mail Server at jira-he-de.apache.org) with ESMTP id 5AC88780899 for ; Tue, 1 Oct 2019 10:57:00 +0000 (UTC) Date: Tue, 1 Oct 2019 10:57:00 +0000 (UTC) From: "George Papa (Jira)" To: issues@spark.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (SPARK-29055) Memory leak in Spark 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-29055?page=3Dcom.atlassia= n.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=3D169= 41736#comment-16941736 ]=20 George Papa commented on SPARK-29055: ------------------------------------- =C2=A0 [~kabhwan] First of all thank for your time about this issue!! As I described in the initial description, after a long time run, I start h= aving spill to disk. So, maybe the bug in the UI memory usage report which = is look to be corrected with ([GitHub Pull Request #25973|https://github.co= m/apache/spark/pull/25973]), there is another hidden bug. =C2=A0 I run the code in the snippet (I test it without any sleeping time, in orde= r to see the results faster) and I have recorded the JVM memory usage for a= pproximately 1 hour between Spark 2.4.4 and your branch with your patch. Spark JVM memory with Spark 2.4.4: ||Time||RES||SHR||MEM%|| |1min|{color:#de350b}1349{color}|32724|1.5| |3min|{color:#de350b}1936{color}|32724|2.2| |5min|{color:#de350b}2506{color}|32724|2.6| |7min|{color:#de350b}2564{color}|32724|2.7| |9min|{color:#de350b}2584{color}|32724|2.7| |11min|{color:#de350b}2585{color}|32724|2.7| |13min|{color:#de350b}2592{color}|32724|2.7| |15min|{color:#de350b}2591{color}|32724|2.7| |17min|{color:#de350b}2591{color}|32724|2.7| |30min|{color:#de350b}2600{color}|32724|2.7| |1h|{color:#de350b}2618{color}|32724|2.7| =C2=A0 Spark JVM memory with Spark patch([GitHub Pull Request #25973|https://githu= b.com/apache/spark/pull/25973]) =C2=A0 ||Time||RES||SHR||MEM%|| |1min|{color:#de350b}1134{color}|25380|1.4| |3min|{color:#de350b}1520{color}|25380|1.6| |5min|{color:#de350b}1570{color}|25380|1.6| |7min|{color:#de350b}1598{color}|25380|1.7| |9min|{color:#de350b}1613{color}|25380|1.7| |11min|{color:#de350b}1616{color}|25380|1.7| |15min|{color:#de350b}1620{color}|25380|1.7| |17min|{color:#de350b}1625{color}|25380|1.7| |30min|{color:#de350b}1629{color}|25380|1.7| |1h|{color:#de350b}1660{color}|25380|1.7| =C2=A0 As you can see the RES memory is slightly increasing in both cases overtime= . Also, when I tested with a real streaming application in a testing env af= ter hours, the persisted dataframes overflows the memory and spill to disk. * You can easily reproduce the above behavior, by running the snippet code = (I prefer to run without any sleeping delay) and track the JVM memory with = top or htop command. =C2=A0 =C2=A0 =C2=A0 > Memory leak in Spark > -------------------- > > Key: SPARK-29055 > URL: https://issues.apache.org/jira/browse/SPARK-29055 > Project: Spark > Issue Type: Bug > Components: Block Manager, Spark Core > Affects Versions: 2.3.3 > Reporter: George Papa > Priority: Major > Attachments: test_csvs.zip > > > I used Spark 2.1.1 and I upgraded into new versions. After Spark version = 2.3.3,=C2=A0 I observed from Spark UI that the driver memory is{color:#ff00= 00} increasing continuously.{color} > In more detail, the driver memory and executors memory have the same used= memory storage and after each iteration the storage memory is increasing. = You can reproduce this behavior by running the following snippet code. The = following example, is very simple, without any dataframe persistence, but t= he memory consumption is not stable as it was in former Spark versions (Spe= cifically until Spark 2.3.2). > Also, I tested with Spark streaming and structured streaming API and I ha= d the same behavior. I tested with an existing application which reads from= Kafka source and do some aggregations, persist dataframes and then unpersi= st them. The persist and unpersist it works correct, I see the dataframes i= n the storage tab in Spark UI and after the unpersist, all dataframe have r= emoved. But, after the unpersist the executors memory is not zero, BUT has = the same value with the driver memory. This behavior also affects the appli= cation performance because the memory of the executors is increasing as the= driver increasing and after a while the persisted dataframes are not fit i= n the executors memory and=C2=A0 I have spill to disk. > Another error which I had after a long running, was {color:#ff0000}java.l= ang.OutOfMemoryError: GC overhead limit exceeded, but I don't know if its r= elevant with the above behavior or not.{color} > =C2=A0 > *HOW TO REPRODUCE THIS BEHAVIOR:* > Create a very simple application(streaming count_file.py) in order to rep= roduce this behavior. This application reads CSV files from a directory, co= unt the rows and then remove the processed files. > {code:java} > import time > import os > from pyspark.sql import SparkSession > from pyspark.sql import functions as F > from pyspark.sql import types as T > target_dir =3D "..." > spark=3DSparkSession.builder.appName("DataframeCount").getOrCreate() > while True: > for f in os.listdir(target_dir): > df =3D spark.read.load(target_dir + f, format=3D"csv") > print("Number of records: {0}".format(df.count())) > time.sleep(15){code} > Submit code: > {code:java} > spark-submit=20 > --master spark://xxx.xxx.xx.xxx > --deploy-mode client > --executor-memory 4g > --executor-cores 3 > streaming count_file.py > {code} > =C2=A0 > *TESTED CASES WITH THE SAME BEHAVIOUR:* > * I tested with default settings (spark-defaults.conf) > * Add spark.cleaner.periodicGC.interval 1min (or less) > * {{Turn spark.cleaner.referenceTracking.blocking}}=3Dfalse > * Run the application in cluster mode > * Increase/decrease the resources of the executors and driver > * I tested with extraJavaOptions in driver and executor -XX:+UseG1GC -XX= :InitiatingHeapOccupancyPercent=3D35 -XX:ConcGCThreads=3D12 > =C2=A0 > *DEPENDENCIES* > * Operation system: Ubuntu 16.04.3 LTS > * Java: jdk1.8.0_131 (tested also with jdk1.8.0_221) > * Python: Python 2.7.12 > =C2=A0 > *NOTE:* In Spark 2.1.1 the driver memory consumption (Storage Memory tab)= was extremely low and after the run of ContextCleaner and BlockManager the= memory was decreasing. -- This message was sent by Atlassian Jira (v8.3.4#803005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org For additional commands, e-mail: issues-help@spark.apache.org