Skip to content

Commit

Permalink
fix thread safe
Browse files Browse the repository at this point in the history
  • Loading branch information
lianhuiwang committed Apr 21, 2016
1 parent 743ef16 commit 97fd174
Show file tree
Hide file tree
Showing 3 changed files with 7 additions and 7 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ class ExternalAppendOnlyMap[K, V, C](
this(createCombiner, mergeValue, mergeCombiners, serializer, blockManager, TaskContext.get())
}

private var currentMap = new SizeTrackingAppendOnlyMap[K, C]
@volatile private var currentMap = new SizeTrackingAppendOnlyMap[K, C]
private val spilledMaps = new ArrayBuffer[DiskMapIterator]
private val sparkConf = SparkEnv.get.conf
private val diskBlockManager = blockManager.diskBlockManager
Expand Down Expand Up @@ -115,7 +115,7 @@ class ExternalAppendOnlyMap[K, V, C](
private val keyComparator = new HashComparator[K]
private val ser = serializer.newInstance()

private var readingIterator: SpillableIterator = null
@volatile private var readingIterator: SpillableIterator = null

/**
* Number of files this map has spilled so far.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -124,8 +124,8 @@ private[spark] class ExternalSorter[K, V, C](
// Data structures to store in-memory objects before we spill. Depending on whether we have an
// Aggregator set, we either put objects into an AppendOnlyMap where we combine them, or we
// store them in an array buffer.
private var map = new PartitionedAppendOnlyMap[K, C]
private var buffer = new PartitionedPairBuffer[K, C]
@volatile private var map = new PartitionedAppendOnlyMap[K, C]
@volatile private var buffer = new PartitionedPairBuffer[K, C]

// Total spilling statistics
private var _diskBytesSpilled = 0L
Expand All @@ -135,9 +135,9 @@ private[spark] class ExternalSorter[K, V, C](
private var _peakMemoryUsedBytes: Long = 0L
def peakMemoryUsedBytes: Long = _peakMemoryUsedBytes

private var isShuffleSort: Boolean = true
@volatile private var isShuffleSort: Boolean = true
private val forceSpillFiles = new ArrayBuffer[SpilledFile]
private var readingIterator: SpillableIterator = null
@volatile private var readingIterator: SpillableIterator = null

// A comparator for keys K that orders them within a partition to allow aggregation or sorting.
// Can be a partial ordering by hash code if a total ordering is not provided through by the
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ private[spark] abstract class Spillable[C](taskMemoryManager: TaskMemoryManager)
SparkEnv.get.conf.getLong("spark.shuffle.spill.numElementsForceSpillThreshold", Long.MaxValue)

// Threshold for this collection's size in bytes before we start tracking its memory usage
// To avoid memory leak for rdd.first(), initialize this to a value orders of magnitude > 0
// To avoid a large number of small spills, initialize this to a value orders of magnitude > 0
private[this] var myMemoryThreshold = initialMemoryThreshold

// Number of elements read from input since last spill
Expand Down

0 comments on commit 97fd174

Please sign in to comment.