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

lot of strings get accumulated in the heap dump of spark thrift server

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • 3.2.0
    • None
    • Spark Core, SQL
    • None
    • Open JDK (8 build 1.8.0_312-b07) and scala 2.12

      OS: Red Hat Enterprise Linux 8.4 (Ootpa), platform:el8

    Description

      I am starting spark thrift server using the following options

      ```

      /data/spark/sbin/start-thriftserver.sh --master spark://*****:7077 --conf "spark.cores.max=320" --conf "spark.executor.cores=3" --conf "spark.driver.cores=15" --executor-memory=10G --driver-memory=50G --conf spark.sql.adaptive.coalescePartitions.enabled=true --conf spark.sql.adaptive.skewJoin.enabled=true --conf spark.sql.cbo.enabled=true --conf spark.sql.adaptive.enabled=true --conf spark.rpc.io.serverThreads=64 --conf "spark.driver.maxResultSize=4G" --conf "spark.max.fetch.failures.per.stage=10" --conf "spark.sql.thriftServer.incrementalCollect=false" --conf "spark.ui.reverseProxy=true" --conf "spark.ui.reverseProxyUrl=/spark_ui" --conf "spark.sql.autoBroadcastJoinThreshold=1073741824" --conf spark.sql.thriftServer.interruptOnCancel=true --conf spark.sql.thriftServer.queryTimeout=0 --hiveconf hive.server2.transport.mode=http --hiveconf hive.server2.thrift.http.path=spark_sql --hiveconf hive.server2.thrift.min.worker.threads=500 --hiveconf hive.server2.thrift.max.worker.threads=2147483647 --hiveconf hive.server2.thrift.http.cookie.is.secure=false --hiveconf hive.server2.thrift.http.cookie.auth.enabled=false --hiveconf hive.server2.authentication=NONE --hiveconf hive.server2.enable.doAs=false --hiveconf spark.sql.hive.thriftServer.singleSession=true --hiveconf hive.server2.thrift.bind.host=0.0.0.0 --conf "spark.sql.extensions=io.delta.sql.DeltaSparkSessionExtension" --conf "spark.sql.catalog.spark_catalog=org.apache.spark.sql.delta.catalog.DeltaCatalog" --conf "spark.sql.cbo.joinReorder.enabled=true" --conf "spark.sql.optimizer.dynamicPartitionPruning.enabled=true" --conf "spark.worker.cleanup.enabled=true" --conf "spark.worker.cleanup.appDataTtl=3600" --hiveconf hive.exec.scratchdir=/data/spark_scratch/hive --hiveconf hive.exec.local.scratchdir=/data/spark_scratch/local_scratch_dir --hiveconf hive.download.resources.dir=/data/spark_scratch/hive.downloaded.resources.dir --hiveconf hive.querylog.location=/data/spark_scratch/hive.querylog.location --conf spark.executor.extraJavaOptions="-XX:+PrintGCDetails -XX:+PrintGCTimeStamps" --conf spark.driver.extraJavaOptions="-verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps -XX:+PrintGCDateStamps -Xloggc:/data/thrift_driver_gc.log -XX:+ExplicitGCInvokesConcurrent -XX:MinHeapFreeRatio=20 -XX:MaxHeapFreeRatio=40 -XX:GCTimeRatio=4 -XX:AdaptiveSizePolicyWeight=90 -XX:MaxRAM=55g" --hiveconf "hive.server2.session.check.interval=60000" --hiveconf "hive.server2.idle.session.timeout=900000" --hiveconf "hive.server2.idle.session.check.operation=true" --conf "spark.eventLog.enabled=false" --conf "spark.cleaner.periodicGC.interval=5min" --conf "spark.appStateStore.asyncTracking.enable=false" --conf "spark.ui.retainedJobs=30" --conf "spark.ui.retainedStages=100" --conf "spark.ui.retainedTasks=500" --conf "spark.sql.ui.retainedExecutions=10" --conf "spark.ui.retainedDeadExecutors=10" --conf "spark.worker.ui.retainedExecutors=10" --conf "spark.worker.ui.retainedDrivers=10" --conf spark.ui.enabled=false --conf spark.stage.maxConsecutiveAttempts=10 --conf spark.executor.memoryOverhead=1G --conf "spark.io.compression.codec=snappy" --conf "spark.default.parallelism=640" --conf spark.memory.offHeap.enabled=true --conf "spark.memory.offHeap.size=3g" --conf "spark.memory.fraction=0.75" --conf "spark.memory.storageFraction=0.75"

      ```

      the java heap dump after heavy usage is as follows
      ```
      1: 50465861 9745837152 [C
      2: 23337896 1924089944 [Ljava.lang.Object;
      3: 72524905 1740597720 java.lang.Long
      4: 50463694 1614838208 java.lang.String
      5: 22718029 726976928 org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema
      6: 2259416 343483328 [Lscala.collection.mutable.HashEntry;
      7: 16 141744616 [Lorg.apache.spark.sql.Row;
      8: 532529 123546728 org.apache.spark.sql.catalyst.expressions.Cast
      9: 535418 72816848 org.apache.spark.sql.catalyst.expressions.Literal
      10: 1105284 70738176 scala.collection.mutable.LinkedHashSet
      11: 1725833 70655016 [J
      12: 1154128 55398144 scala.collection.mutable.HashMap
      13: 1720740 55063680 org.apache.spark.util.collection.BitSet
      14: 572222 50355536 scala.collection.immutable.Vector
      15: 1602297 38455128 scala.Some
      16: 1154303 36937696 scala.collection.immutable.$colon$colon
      17: 1105284 26526816 org.apache.spark.sql.catalyst.expressions.AttributeSet
      18: 1066442 25594608 java.lang.Integer
      19: 735502 23536064 scala.collection.immutable.HashSet$HashSet1
      20: 10300 19511408 [B
      21: 543994 17407808 scala.Tuple2
      22: 530244 16967808 org.apache.spark.sql.catalyst.trees.Origin
      23: 274445 13173360 java.util.Hashtable$Entry
      24: 225826 13089920 [Lscala.collection.immutable.HashSet;
      25: 529922 12718128 org.apache.spark.sql.catalyst.expressions.CastBase$$Lambda$3882/1746188635
      26: 221866 10649568 java.util.concurrent.ConcurrentHashMap$Node
      27: 384729 9233496 java.lang.Double
      28: 225826 7226432 scala.collection.immutable.HashSet$HashTrieSet
      29: 1071 6659680 [Ljava.util.concurrent.ConcurrentHashMap$Node;
      30: 25853 4348744 java.lang.Class
      31: 5760 3916800 io.netty.util.internal.shaded.org.jctools.queues.MpscArrayQueue
      32: 1232 3710888 [Ljava.util.Hashtable$Entry;
      33: 17303 3460600 org.apache.spark.sql.catalyst.expressions.objects.Invoke
      34: 20409 3265440 java.lang.reflect.Method
      35: 47506 2280288 java.util.HashMap$Node
      36: 7489 1568600 [Ljava.util.HashMap$Node;
      37: 285 1539888 [Ljava.nio.ByteBuffer;
      38: 237 1452984 [[B
      39: 4138 1291056 org.apache.spark.status.TaskDataWrapper
      ```
       there is 9.7 gb of heap memory that is occupied by char array, not sure, why is it ? Can someone help me understand remove this 9.7 gb of memory.

      attaching the after and before image, as you can see memory is continuously increasing.

      Attachments

        1. screenshot-1.png
          402 kB
          ramakrishna chilaka

        Activity

          People

            Unassigned Unassigned
            rkchilaka ramakrishna chilaka
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated: