Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-34689

Spark Thrift Server: Memory leak for SparkSession objects

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Duplicate
    • 3.0.1, 3.1.1
    • None
    • Spark Core, SQL
    • None

    Description

      When running the Spark Thrift Server (3.0.1, standalone cluster), we have noticed that each new JDBC connection creates a new SparkSession object. This object (and anything being referenced by it), however, remains in memory indefinitely even though the JDBC connection is closed, and full GCs do not remove it. After about 18 hours of heavy use, we get more than 46.000 such objects (heap_sparksession.png).

      In a small local installation test, I replicated the behavior by simply opening a JDBC connection, executing SHOW SCHEMAS and closing the connection (heapdump_local_attempt.png). For each connection, a new SparkSession object is created and never removed. I have noticed the same behavior in Spark 3.1.1 as well.

      Our settings are as follows. Please note that this was occuring even before we added the ExplicitGCInvokesConcurrent option (i.e. it happened even when a full GC was performed every 20 minutes).

      spark-defaults.conf:

      spark.master                    spark://...:7077,...:7077
      spark.master.rest.enabled       true
      spark.eventLog.enabled          false
      spark.eventLog.dir              file:///...
      
      spark.driver.cores             1
      spark.driver.maxResultSize     4g
      spark.driver.memory            5g
      spark.executor.memory          1g
      
      spark.executor.logs.rolling.maxRetainedFiles   2
      spark.executor.logs.rolling.strategy           size
      spark.executor.logs.rolling.maxSize            1G
      
      spark.local.dir ...
      
      spark.sql.ui.retainedExecutions=10
      spark.ui.retainedDeadExecutors=10
      spark.worker.ui.retainedExecutors=10
      spark.worker.ui.retainedDrivers=10
      spark.ui.retainedJobs=30
      spark.ui.retainedStages=100
      spark.ui.retainedTasks=500
      spark.appStateStore.asyncTracking.enable=false
      
      spark.sql.shuffle.partitions=200
      spark.default.parallelism=200
      spark.task.reaper.enabled=true
      spark.task.reaper.threadDump=false
      
      spark.memory.offHeap.enabled=true
      spark.memory.offHeap.size=4g
      

      spark-env.sh:

      HADOOP_CONF_DIR="/.../hadoop/etc/hadoop"
      
      SPARK_WORKER_CORES=28
      SPARK_WORKER_MEMORY=54g
      
      SPARK_WORKER_OPTS="-Dspark.worker.cleanup.enabled=true -Dspark.worker.cleanup.appDataTtl=172800 -XX:+UseG1GC -XX:MaxGCPauseMillis=200 -XX:InitiatingHeapOccupancyPercent=40 "
      
      SPARK_DAEMON_JAVA_OPTS="-Dlog4j.configuration=file:///.../log4j.properties -Dspark.deploy.recoveryMode=ZOOKEEPER -Dspark.deploy.zookeeper.dir="..." -Dspark.deploy.zookeeper.url=...:2181,...:2181,...:2181 -XX:+UseG1GC -XX:MaxGCPauseMillis=200 -XX:InitiatingHeapOccupancyPercent=40"
      

      start-thriftserver.sh:

      export SPARK_DAEMON_MEMORY=16g
      
      exec "${SPARK_HOME}"/sbin/spark-daemon.sh submit $CLASS 1 \
        --conf "spark.sql.extensions=io.delta.sql.DeltaSparkSessionExtension" \
        --conf "spark.ui.retainedJobs=30" \
        --conf "spark.ui.retainedStages=100" \
        --conf "spark.ui.retainedTasks=500" \
        --conf "spark.sql.ui.retainedExecutions=10" \
        --conf "spark.appStateStore.asyncTracking.enable=false" \
        --conf "spark.cleaner.periodicGC.interval=20min" \
        --conf "spark.sql.autoBroadcastJoinThreshold=-1" \
        --conf "spark.executor.extraJavaOptions=-verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps -XX:+PrintGCDateStamps -XX:+UseG1GC -XX:MaxGCPauseMillis=200" \
        --conf "spark.driver.extraJavaOptions=-verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps -XX:+PrintGCDateStamps -Xloggc:/.../thrift_driver_gc.log -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=7 -XX:GCLogFileSize=35M -XX:+UseG1GC -XX:MaxGCPauseMillis=200 -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false -Dcom.sun.management.jmxremote.port=11990 -XX:+ExplicitGCInvokesConcurrent" \
        --conf "spark.metrics.namespace=..." --name "..." --packages io.delta:delta-core_2.12:0.7.0 --hiveconf spark.ui.port=4038 --hiveconf spark.cores.max=22 --hiveconf spark.executor.cores=3 --hiveconf spark.executor.memory=6144M --hiveconf spark.scheduler.mode=FAIR --hiveconf spark.scheduler.allocation.file=.../conf/thrift-scheduler.xml \
        --conf spark.sql.thriftServer.incrementalCollect=true "$@"
      

      Attachments

        1. heap_sparksession.png
          154 kB
          Dimitris Batis
        2. heapdump_local_attempt_250_closed_connections.png
          525 kB
          Dimitris Batis
        3. test_patch.diff
          2 kB
          Dimitris Batis

        Issue Links

          Activity

            People

              Unassigned Unassigned
              dbatis Dimitris Batis
              Votes:
              0 Vote for this issue
              Watchers:
              1 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: