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

Reduce spark.memory.fraction default to avoid overrunning old gen in JVM default config

    XMLWordPrintableJSON

Details

    • Improvement
    • Status: Resolved
    • Blocker
    • Resolution: Fixed
    • 1.6.0, 1.6.1
    • 2.0.0
    • None
    • None

    Description

      While debugging performance issues in a Spark program, I've found a simple way to slow down Spark 1.6 significantly by filling the RDD memory cache. This seems to be a regression, because setting "spark.memory.useLegacyMode=true" fixes the problem. Here is a repro that is just a simple program that fills the memory cache of Spark using a MEMORY_ONLY cached RDD (but of course this comes up in more complex situations, too):

      import org.apache.spark.SparkContext
      import org.apache.spark.SparkConf
      import org.apache.spark.storage.StorageLevel
      
      object CacheDemoApp { 
        def main(args: Array[String]) {
          val conf = new SparkConf().setAppName("Cache Demo Application")                                       
          val sc = new SparkContext(conf)
          val startTime = System.currentTimeMillis()
                                                                                                                
          val cacheFiller = sc.parallelize(1 to 500000000, 1000)                                                
            .mapPartitionsWithIndex {
              case (ix, it) =>
                println(s"CREATE DATA PARTITION ${ix}")                                                         
                val r = new scala.util.Random(ix)
                it.map(x => (r.nextLong, r.nextLong))
            }
          cacheFiller.persist(StorageLevel.MEMORY_ONLY)
          cacheFiller.foreach(identity)
      
          val finishTime = System.currentTimeMillis()
          val elapsedTime = (finishTime - startTime) / 1000
          println(s"TIME= $elapsedTime s")
        }
      
      }
      

      If I call it the following way, it completes in around 5 minutes on my Laptop, while often stopping for slow Full GC cycles. I can also see with jvisualvm (Visual GC plugin) that the old generation of JVM is 96.8% filled.

      sbt package
      ~/spark-1.6.0/bin/spark-submit \
        --class "CacheDemoApp" \
        --master "local[2]" \
        --driver-memory 3g \
        --driver-java-options "-XX:+PrintGCDetails" \
        target/scala-2.10/simple-project_2.10-1.0.jar
      

      If I add any one of the below flags, then the run-time drops to around 40-50 seconds and the difference is coming from the drop in GC times:
      --conf "spark.memory.fraction=0.6"
      OR
      --conf "spark.memory.useLegacyMode=true"
      OR
      --driver-java-options "-XX:NewRatio=3"

      All the other cache types except for DISK_ONLY produce similar symptoms. It looks like that the problem is that the amount of data Spark wants to store long-term ends up being larger than the old generation size in the JVM and this triggers Full GC repeatedly.

      I did some research:

      • In Spark 1.6, spark.memory.fraction is the upper limit on cache size. It defaults to 0.75.
      • In Spark 1.5, spark.storage.memoryFraction is the upper limit in cache size. It defaults to 0.6 and...
      • http://spark.apache.org/docs/1.5.2/configuration.html even says that it shouldn't be bigger than the size of the old generation.
      • On the other hand, OpenJDK's default NewRatio is 2, which means an old generation size of 66%. Hence the default value in Spark 1.6 contradicts this advice.

      http://spark.apache.org/docs/1.6.1/tuning.html recommends that if the old generation is running close to full, then setting spark.memory.storageFraction to a lower value should help. I have tried with spark.memory.storageFraction=0.1, but it still doesn't fix the issue. This is not a surprise: http://spark.apache.org/docs/1.6.1/configuration.html explains that storageFraction is not an upper-limit but a lower limit-like thing on the size of Spark's cache. The real upper limit is spark.memory.fraction.

      To sum up my questions/issues:

      • At least http://spark.apache.org/docs/1.6.1/tuning.html should be fixed. Maybe the old generation size should also be mentioned in configuration.html near spark.memory.fraction.
      • Is it a goal for Spark to support heavy caching with default parameters and without GC breakdown? If so, then better default values are needed.

      Attachments

        1. baseline.txt
          925 kB
          Gabor Feher
        2. memfrac06.txt
          913 kB
          Gabor Feher
        3. memfrac063.txt
          914 kB
          Gabor Feher
        4. memfrac066.txt
          930 kB
          Gabor Feher

        Activity

          People

            srowen Sean R. Owen
            gfeher Gabor Feher
            Votes:
            1 Vote for this issue
            Watchers:
            9 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: