Skip to content

Commit

Permalink
Fix tests
Browse files Browse the repository at this point in the history
  • Loading branch information
Andrew Or committed Nov 19, 2014
1 parent 7012595 commit f2e552c
Show file tree
Hide file tree
Showing 4 changed files with 21 additions and 11 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package org.apache.spark.shuffle

import scala.collection.mutable

import org.apache.spark.{Logging, SparkException, SparkConf}
import org.apache.spark.{Logging, SparkConf, SparkEnv, SparkException}

/**
* Allocates a pool of memory to task threads for use in shuffle operations. Each disk-spilling
Expand Down Expand Up @@ -124,5 +124,5 @@ private[spark] object ShuffleMemoryManager {
}

// Initial threshold for the size of a collection before we start tracking its memory usage
val INITIAL_MEMORY_TRACKING_THRESHOLD: Long = 5 * 1024 * 1024
val DEFAULT_INITIAL_MEMORY_THRESHOLD: Long = 5 * 1024 * 1024
}
Original file line number Diff line number Diff line change
Expand Up @@ -82,9 +82,14 @@ class ExternalAppendOnlyMap[K, V, C](
// Number of in-memory pairs inserted before tracking the map's shuffle memory usage
private val trackMemoryThreshold = 1000

// Initial threshold for the size of a collection before we start tracking its memory usage
private val initialMemoryThreshold =
SparkEnv.get.conf.getLong("spark.shuffle.spill.initialMemoryThreshold",
ShuffleMemoryManager.DEFAULT_INITIAL_MEMORY_THRESHOLD)

// Threshold for the collection's size in bytes before we start tracking its memory usage
// To avoid a large number of small spills, initialize this to a value orders of magnitude > 0
private var myMemoryThreshold = ShuffleMemoryManager.INITIAL_MEMORY_TRACKING_THRESHOLD
private var myMemoryThreshold = initialMemoryThreshold

/**
* Size of object batches when reading/writing from serializers.
Expand Down Expand Up @@ -238,11 +243,10 @@ class ExternalAppendOnlyMap[K, V, C](

// Release our memory back to the shuffle pool so that other threads can grab it
// The amount we requested does not include the initial memory tracking threshold
shuffleMemoryManager.release(
myMemoryThreshold - ShuffleMemoryManager.INITIAL_MEMORY_TRACKING_THRESHOLD)
shuffleMemoryManager.release(myMemoryThreshold - initialMemoryThreshold)

// Reset this to the initial threshold to avoid spilling many small files
myMemoryThreshold = ShuffleMemoryManager.INITIAL_MEMORY_TRACKING_THRESHOLD
myMemoryThreshold = initialMemoryThreshold

elementsRead = 0
_memoryBytesSpilled += mapSize
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -135,9 +135,14 @@ private[spark] class ExternalSorter[K, V, C](
// Write metrics for current spill
private var curWriteMetrics: ShuffleWriteMetrics = _

// Initial threshold for the size of a collection before we start tracking its memory usage
private val initialMemoryThreshold =
SparkEnv.get.conf.getLong("spark.shuffle.spill.initialMemoryThreshold",
ShuffleMemoryManager.DEFAULT_INITIAL_MEMORY_THRESHOLD)

// Threshold for the collection's size in bytes before we start tracking its memory usage
// To avoid a large number of small spills, initialize this to a value orders of magnitude > 0
private var myMemoryThreshold = ShuffleMemoryManager.INITIAL_MEMORY_TRACKING_THRESHOLD
private var myMemoryThreshold = initialMemoryThreshold

// If there are fewer than spark.shuffle.sort.bypassMergeThreshold partitions and we don't need
// local aggregation and sorting, write numPartitions files directly and just concatenate them
Expand Down Expand Up @@ -287,11 +292,10 @@ private[spark] class ExternalSorter[K, V, C](

// Release our memory back to the shuffle pool so that other threads can grab it
// The amount we requested does not include the initial memory tracking threshold
shuffleMemoryManager.release(
myMemoryThreshold - ShuffleMemoryManager.INITIAL_MEMORY_TRACKING_THRESHOLD)
shuffleMemoryManager.release(myMemoryThreshold - initialMemoryThreshold)

// Reset this to the initial threshold to avoid spilling many small files
myMemoryThreshold = ShuffleMemoryManager.INITIAL_MEMORY_TRACKING_THRESHOLD
myMemoryThreshold = initialMemoryThreshold

_memoryBytesSpilled += memorySize
elementsRead = 0
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -127,6 +127,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe
test("empty partitions with spilling") {
val conf = createSparkConf(false)
conf.set("spark.shuffle.memoryFraction", "0.001")
conf.set("spark.shuffle.spill.initialMemoryThreshold", "512")
conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager")
sc = new SparkContext("local", "test", conf)

Expand All @@ -152,6 +153,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe
test("empty partitions with spilling, bypass merge-sort") {
val conf = createSparkConf(false)
conf.set("spark.shuffle.memoryFraction", "0.001")
conf.set("spark.shuffle.spill.initialMemoryThreshold", "512")
conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager")
sc = new SparkContext("local", "test", conf)

Expand Down Expand Up @@ -761,5 +763,5 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe
}

sorter2.stop()
}
}
}

0 comments on commit f2e552c

Please sign in to comment.