From c12f093151c2c6cacf2c2cfade4dd1e9512d9edd Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 19 Jun 2014 17:55:59 -0700 Subject: [PATCH 01/55] Add SizeTrackingAppendOnlyBuffer and tests This buffer is supported by an underlying PrimitiveVector. It is to be used for unrolling partitions, so we can efficiently check the size of the in-memory buffer periodically. Note that the underlying buffer cannot be an existing implementation of a mutable Scala or Java collection. This is because we need to be exposed to when the underlying array is resized. Otherwise, size estimation may not be accurate. --- .../org/apache/spark/util/SizeEstimator.scala | 2 +- .../util/collection/PrimitiveVector.scala | 2 +- .../spark/util/collection/SizeTracker.scala | 96 +++++++++ .../SizeTrackingAppendOnlyBuffer.scala | 39 ++++ .../SizeTrackingAppendOnlyMap.scala | 71 +----- .../util/SizeTrackingAppendOnlyMapSuite.scala | 120 ----------- .../util/collection/SizeTrackerSuite.scala | 204 ++++++++++++++++++ 7 files changed, 346 insertions(+), 188 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/collection/SizeTracker.scala create mode 100644 core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyBuffer.scala delete mode 100644 core/src/test/scala/org/apache/spark/util/SizeTrackingAppendOnlyMapSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/util/collection/SizeTrackerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala index 08465575309c6..bce3b3afe9aba 100644 --- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -180,7 +180,7 @@ private[spark] object SizeEstimator extends Logging { } } - // Estimat the size of arrays larger than ARRAY_SIZE_FOR_SAMPLING by sampling. + // Estimate the size of arrays larger than ARRAY_SIZE_FOR_SAMPLING by sampling. private val ARRAY_SIZE_FOR_SAMPLING = 200 private val ARRAY_SAMPLE_SIZE = 100 // should be lower than ARRAY_SIZE_FOR_SAMPLING diff --git a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala index b84eb65c62bc7..3a22d998d1612 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala @@ -36,7 +36,7 @@ class PrimitiveVector[@specialized(Long, Int, Double) V: ClassTag](initialSize: _array(index) } - def +=(value: V) { + def +=(value: V): Unit = { if (_numElements == _array.length) { resize(_array.length * 2) } diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTracker.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTracker.scala new file mode 100644 index 0000000000000..a40359038545c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTracker.scala @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.util.SizeEstimator + +/** + * A general interface for collections that keeps track of its estimated size in bytes. + * We sample with a slow exponential back-off using the SizeEstimator to amortize the time, + * as each call to SizeEstimator can take a sizable amount of time (order of a few milliseconds). + */ +private[spark] trait SizeTracker { + + import SizeTracker._ + + /** + * Controls the base of the exponential which governs the rate of sampling. + * E.g., a value of 2 would mean we sample at 1, 2, 4, 8, ... elements. + */ + private val SAMPLE_GROWTH_RATE = 1.1 + + /** All samples taken since last resetSamples(). Only the last two are used for extrapolation. */ + private val samples = new ArrayBuffer[Sample] + + /** The average number of bytes per update between our last two samples. */ + private var bytesPerUpdate: Double = _ + + /** Total number of insertions and updates into the map since the last resetSamples(). */ + private var numUpdates: Long = _ + + /** The value of 'numUpdates' at which we will take our next sample. */ + private var nextSampleNum: Long = _ + + resetSamples() + + /** Called after the collection undergoes a dramatic change in size. */ + protected def resetSamples(): Unit = { + numUpdates = 1 + nextSampleNum = 1 + samples.clear() + takeSample() + } + + /** Callback to be invoked after an update. */ + protected def afterUpdate(): Unit = { + numUpdates += 1 + if (nextSampleNum == numUpdates) { + takeSample() + } + } + + /** Takes a new sample of the current collection's size. */ + private def takeSample(): Unit = { + samples += Sample(SizeEstimator.estimate(this), numUpdates) + // Only use the last two samples to extrapolate + if (samples.size > 2) { + samples.remove(0) + } + val bytesDelta = samples.toSeq.reverse match { + case latest :: previous :: tail => + (latest.size - previous.size).toDouble / (latest.numUpdates - previous.numUpdates) + // If fewer than 2 samples, assume no change + case _ => 0 + } + bytesPerUpdate = math.max(0, bytesDelta) + nextSampleNum = math.ceil(numUpdates * SAMPLE_GROWTH_RATE).toLong + } + + /** Estimates the current size of the collection in bytes. O(1) time. */ + def estimateSize(): Long = { + assert(samples.nonEmpty) + val extrapolatedDelta = bytesPerUpdate * (numUpdates - samples.last.numUpdates) + (samples.last.size + extrapolatedDelta).toLong + } +} + +private object SizeTracker { + case class Sample(size: Long, numUpdates: Long) +} diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyBuffer.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyBuffer.scala new file mode 100644 index 0000000000000..574c454557ec8 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyBuffer.scala @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection + +import scala.reflect.ClassTag + +/** + * An append-only buffer that keeps track of its estimated size in bytes. + */ +private[spark] class SizeTrackingAppendOnlyBuffer[T: ClassTag] + extends PrimitiveVector[T] + with SizeTracker { + + override def +=(value: T): Unit = { + super.+=(value) + super.afterUpdate() + } + + override def resize(newLength: Int): PrimitiveVector[T] = { + super.resize(newLength) + resetSamples() + this + } +} diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala index 204330dad48b9..de61e1d17fe10 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala @@ -17,85 +17,24 @@ package org.apache.spark.util.collection -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.util.SizeEstimator -import org.apache.spark.util.collection.SizeTrackingAppendOnlyMap.Sample - /** - * Append-only map that keeps track of its estimated size in bytes. - * We sample with a slow exponential back-off using the SizeEstimator to amortize the time, - * as each call to SizeEstimator can take a sizable amount of time (order of a few milliseconds). + * An append-only map that keeps track of its estimated size in bytes. */ -private[spark] class SizeTrackingAppendOnlyMap[K, V] extends AppendOnlyMap[K, V] { - - /** - * Controls the base of the exponential which governs the rate of sampling. - * E.g., a value of 2 would mean we sample at 1, 2, 4, 8, ... elements. - */ - private val SAMPLE_GROWTH_RATE = 1.1 - - /** All samples taken since last resetSamples(). Only the last two are used for extrapolation. */ - private val samples = new ArrayBuffer[Sample]() - - /** Total number of insertions and updates into the map since the last resetSamples(). */ - private var numUpdates: Long = _ - - /** The value of 'numUpdates' at which we will take our next sample. */ - private var nextSampleNum: Long = _ - - /** The average number of bytes per update between our last two samples. */ - private var bytesPerUpdate: Double = _ - - resetSamples() - - /** Called after the map grows in size, as this can be a dramatic change for small objects. */ - def resetSamples() { - numUpdates = 1 - nextSampleNum = 1 - samples.clear() - takeSample() - } +private[spark] class SizeTrackingAppendOnlyMap[K, V] extends AppendOnlyMap[K, V] with SizeTracker { override def update(key: K, value: V): Unit = { super.update(key, value) - numUpdates += 1 - if (nextSampleNum == numUpdates) { takeSample() } + super.afterUpdate() } override def changeValue(key: K, updateFunc: (Boolean, V) => V): V = { val newValue = super.changeValue(key, updateFunc) - numUpdates += 1 - if (nextSampleNum == numUpdates) { takeSample() } + super.afterUpdate() newValue } - /** Takes a new sample of the current map's size. */ - def takeSample() { - samples += Sample(SizeEstimator.estimate(this), numUpdates) - // Only use the last two samples to extrapolate. If fewer than 2 samples, assume no change. - bytesPerUpdate = math.max(0, samples.toSeq.reverse match { - case latest :: previous :: tail => - (latest.size - previous.size).toDouble / (latest.numUpdates - previous.numUpdates) - case _ => - 0 - }) - nextSampleNum = math.ceil(numUpdates * SAMPLE_GROWTH_RATE).toLong - } - - override protected def growTable() { + override protected def growTable(): Unit = { super.growTable() resetSamples() } - - /** Estimates the current size of the map in bytes. O(1) time. */ - def estimateSize(): Long = { - assert(samples.nonEmpty) - val extrapolatedDelta = bytesPerUpdate * (numUpdates - samples.last.numUpdates) - (samples.last.size + extrapolatedDelta).toLong - } -} - -private object SizeTrackingAppendOnlyMap { - case class Sample(size: Long, numUpdates: Long) } diff --git a/core/src/test/scala/org/apache/spark/util/SizeTrackingAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeTrackingAppendOnlyMapSuite.scala deleted file mode 100644 index 93f0c6a8e6408..0000000000000 --- a/core/src/test/scala/org/apache/spark/util/SizeTrackingAppendOnlyMapSuite.scala +++ /dev/null @@ -1,120 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.util - -import scala.util.Random - -import org.scalatest.{BeforeAndAfterAll, FunSuite} - -import org.apache.spark.util.SizeTrackingAppendOnlyMapSuite.LargeDummyClass -import org.apache.spark.util.collection.{AppendOnlyMap, SizeTrackingAppendOnlyMap} - -class SizeTrackingAppendOnlyMapSuite extends FunSuite with BeforeAndAfterAll { - val NORMAL_ERROR = 0.20 - val HIGH_ERROR = 0.30 - - test("fixed size insertions") { - testWith[Int, Long](10000, i => (i, i.toLong)) - testWith[Int, (Long, Long)](10000, i => (i, (i.toLong, i.toLong))) - testWith[Int, LargeDummyClass](10000, i => (i, new LargeDummyClass())) - } - - test("variable size insertions") { - val rand = new Random(123456789) - def randString(minLen: Int, maxLen: Int): String = { - "a" * (rand.nextInt(maxLen - minLen) + minLen) - } - testWith[Int, String](10000, i => (i, randString(0, 10))) - testWith[Int, String](10000, i => (i, randString(0, 100))) - testWith[Int, String](10000, i => (i, randString(90, 100))) - } - - test("updates") { - val rand = new Random(123456789) - def randString(minLen: Int, maxLen: Int): String = { - "a" * (rand.nextInt(maxLen - minLen) + minLen) - } - testWith[String, Int](10000, i => (randString(0, 10000), i)) - } - - def testWith[K, V](numElements: Int, makeElement: (Int) => (K, V)) { - val map = new SizeTrackingAppendOnlyMap[K, V]() - for (i <- 0 until numElements) { - val (k, v) = makeElement(i) - map(k) = v - expectWithinError(map, map.estimateSize(), if (i < 32) HIGH_ERROR else NORMAL_ERROR) - } - } - - def expectWithinError(obj: AnyRef, estimatedSize: Long, error: Double) { - val betterEstimatedSize = SizeEstimator.estimate(obj) - assert(betterEstimatedSize * (1 - error) < estimatedSize, - s"Estimated size $estimatedSize was less than expected size $betterEstimatedSize") - assert(betterEstimatedSize * (1 + 2 * error) > estimatedSize, - s"Estimated size $estimatedSize was greater than expected size $betterEstimatedSize") - } -} - -object SizeTrackingAppendOnlyMapSuite { - // Speed test, for reproducibility of results. - // These could be highly non-deterministic in general, however. - // Results: - // AppendOnlyMap: 31 ms - // SizeTracker: 54 ms - // SizeEstimator: 1500 ms - def main(args: Array[String]) { - val numElements = 100000 - - val baseTimes = for (i <- 0 until 10) yield time { - val map = new AppendOnlyMap[Int, LargeDummyClass]() - for (i <- 0 until numElements) { - map(i) = new LargeDummyClass() - } - } - - val sampledTimes = for (i <- 0 until 10) yield time { - val map = new SizeTrackingAppendOnlyMap[Int, LargeDummyClass]() - for (i <- 0 until numElements) { - map(i) = new LargeDummyClass() - map.estimateSize() - } - } - - val unsampledTimes = for (i <- 0 until 3) yield time { - val map = new AppendOnlyMap[Int, LargeDummyClass]() - for (i <- 0 until numElements) { - map(i) = new LargeDummyClass() - SizeEstimator.estimate(map) - } - } - - println("Base: " + baseTimes) - println("SizeTracker (sampled): " + sampledTimes) - println("SizeEstimator (unsampled): " + unsampledTimes) - } - - def time(f: => Unit): Long = { - val start = System.currentTimeMillis() - f - System.currentTimeMillis() - start - } - - private class LargeDummyClass { - val arr = new Array[Int](100) - } -} diff --git a/core/src/test/scala/org/apache/spark/util/collection/SizeTrackerSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/SizeTrackerSuite.scala new file mode 100644 index 0000000000000..83ad339506d97 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/collection/SizeTrackerSuite.scala @@ -0,0 +1,204 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection + +import scala.reflect.ClassTag +import scala.util.Random + +import org.scalatest.FunSuite + +import org.apache.spark.util.SizeEstimator + +class SizeTrackerSuite extends FunSuite { + val NORMAL_ERROR = 0.20 + val HIGH_ERROR = 0.30 + + import SizeTrackerSuite._ + + test("buffer fixed size insertions") { + testBuffer[Long](10000, i => i.toLong) + testBuffer[(Long, Long)](10000, i => (i.toLong, i.toLong)) + testBuffer[LargeDummyClass](10000, i => new LargeDummyClass) + } + + test("buffer variable size insertions") { + val rand = new Random(123456789) + def randString(minLen: Int, maxLen: Int): String = { + "a" * (rand.nextInt(maxLen - minLen) + minLen) + } + testBuffer[String](10000, i => randString(0, 10)) + testBuffer[String](10000, i => randString(0, 100)) + testBuffer[String](10000, i => randString(90, 100)) + } + + test("map fixed size insertions") { + testMap[Int, Long](10000, i => (i, i.toLong)) + testMap[Int, (Long, Long)](10000, i => (i, (i.toLong, i.toLong))) + testMap[Int, LargeDummyClass](10000, i => (i, new LargeDummyClass)) + } + + test("map variable size insertions") { + val rand = new Random(123456789) + def randString(minLen: Int, maxLen: Int): String = { + "a" * (rand.nextInt(maxLen - minLen) + minLen) + } + testMap[Int, String](10000, i => (i, randString(0, 10))) + testMap[Int, String](10000, i => (i, randString(0, 100))) + testMap[Int, String](10000, i => (i, randString(90, 100))) + } + + test("map updates") { + val rand = new Random(123456789) + def randString(minLen: Int, maxLen: Int): String = { + "a" * (rand.nextInt(maxLen - minLen) + minLen) + } + testMap[String, Int](10000, i => (randString(0, 10000), i)) + } + + def testBuffer[T: ClassTag](numElements: Int, makeElement: Int => T) { + val buffer = new SizeTrackingAppendOnlyBuffer[T] + for (i <- 0 until numElements) { + val item = makeElement(i) + buffer += item + expectWithinError(buffer, buffer.estimateSize(), if (i < 32) HIGH_ERROR else NORMAL_ERROR) + } + } + + def testMap[K, V](numElements: Int, makeElement: (Int) => (K, V)) { + val map = new SizeTrackingAppendOnlyMap[K, V] + for (i <- 0 until numElements) { + val (k, v) = makeElement(i) + map(k) = v + expectWithinError(map, map.estimateSize(), if (i < 32) HIGH_ERROR else NORMAL_ERROR) + } + } + + def expectWithinError(obj: AnyRef, estimatedSize: Long, error: Double) { + val betterEstimatedSize = SizeEstimator.estimate(obj) + assert(betterEstimatedSize * (1 - error) < estimatedSize, + s"Estimated size $estimatedSize was less than expected size $betterEstimatedSize") + assert(betterEstimatedSize * (1 + 2 * error) > estimatedSize, + s"Estimated size $estimatedSize was greater than expected size $betterEstimatedSize") + } +} + +private object SizeTrackerSuite { + + /** + * Run speed tests for size tracking collections. + */ + def main(args: Array[String]): Unit = { + if (args.size < 1) { + println("Usage: SizeTrackerSuite [num elements]") + System.exit(1) + } + val numElements = args(0).toInt + bufferSpeedTest(numElements) + mapSpeedTest(numElements) + } + + /** + * Speed test for SizeTrackingAppendOnlyBuffer. + * + * Results for 100000 elements (possibly non-deterministic): + * PrimitiveVector 15 ms + * SizeTracker 51 ms + * SizeEstimator 2000 ms + */ + def bufferSpeedTest(numElements: Int): Unit = { + val baseTimes = for (i <- 0 until 10) yield time { + val buffer = new PrimitiveVector[LargeDummyClass] + for (i <- 0 until numElements) { + buffer += new LargeDummyClass + } + } + val sampledTimes = for (i <- 0 until 10) yield time { + val buffer = new SizeTrackingAppendOnlyBuffer[LargeDummyClass] + for (i <- 0 until numElements) { + buffer += new LargeDummyClass + buffer.estimateSize() + } + } + val unsampledTimes = for (i <- 0 until 3) yield time { + val buffer = new PrimitiveVector[LargeDummyClass] + for (i <- 0 until numElements) { + buffer += new LargeDummyClass + SizeEstimator.estimate(buffer) + } + } + printSpeedTestResult("SizeTrackingAppendOnlyBuffer", baseTimes, sampledTimes, unsampledTimes) + } + + /** + * Speed test for SizeTrackingAppendOnlyMap. + * + * Results for 100000 elements (possibly non-deterministic): + * AppendOnlyMap 30 ms + * SizeTracker 41 ms + * SizeEstimator 1666 ms + */ + def mapSpeedTest(numElements: Int): Unit = { + val baseTimes = for (i <- 0 until 10) yield time { + val map = new AppendOnlyMap[Int, LargeDummyClass] + for (i <- 0 until numElements) { + map(i) = new LargeDummyClass + } + } + val sampledTimes = for (i <- 0 until 10) yield time { + val map = new SizeTrackingAppendOnlyMap[Int, LargeDummyClass] + for (i <- 0 until numElements) { + map(i) = new LargeDummyClass + map.estimateSize() + } + } + val unsampledTimes = for (i <- 0 until 3) yield time { + val map = new AppendOnlyMap[Int, LargeDummyClass] + for (i <- 0 until numElements) { + map(i) = new LargeDummyClass + SizeEstimator.estimate(map) + } + } + printSpeedTestResult("SizeTrackingAppendOnlyMap", baseTimes, sampledTimes, unsampledTimes) + } + + def printSpeedTestResult( + testName: String, + baseTimes: Seq[Long], + sampledTimes: Seq[Long], + unsampledTimes: Seq[Long]): Unit = { + println(s"Average times for $testName (ms):") + println(" Base - " + averageTime(baseTimes)) + println(" SizeTracker (sampled) - " + averageTime(sampledTimes)) + println(" SizeEstimator (unsampled) - " + averageTime(unsampledTimes)) + println() + } + + def time(f: => Unit): Long = { + val start = System.currentTimeMillis() + f + System.currentTimeMillis() - start + } + + def averageTime(v: Seq[Long]): Long = { + v.sum / v.size + } + + private class LargeDummyClass { + val arr = new Array[Int](100) + } +} From 97ea49900d21dac108d2b0f70395f5824d296724 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 19 Jun 2014 18:53:42 -0700 Subject: [PATCH 02/55] Change BlockManager interface to use Arrays ... rather than ArrayBuffer. We only ever iterate through it anyway, so there is really no reason for it to be a mutable buffer of any sort. This change is introduced so that we can eventually directly pass our SizeTrackingAppendOnlyBuffer's underlying array to BlockManager, instead of having to awkwardly make it an ArrayBuffer first. --- .../scala/org/apache/spark/CacheManager.scala | 3 +-- .../apache/spark/storage/BlockManager.scala | 26 ++++++++----------- .../org/apache/spark/storage/BlockStore.scala | 2 +- .../org/apache/spark/storage/DiskStore.scala | 4 +-- .../apache/spark/storage/MemoryStore.scala | 25 ++++++++---------- .../apache/spark/storage/TachyonStore.scala | 4 +-- .../receiver/ReceiverSupervisorImpl.scala | 3 +-- 7 files changed, 27 insertions(+), 40 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 3f667a4a0f9c5..8e5b787e34c90 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -142,8 +142,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { * to the BlockManager as an iterator and expect to read it back later. This is because * we may end up dropping a partition from memory store before getting it back, e.g. * when the entirety of the RDD does not fit in memory. */ - val elements = new ArrayBuffer[Any] - elements ++= values + val elements = values.toArray[Any] updatedBlocks ++= blockManager.put(key, elements, storageLevel, tellMaster = true) elements.iterator.asInstanceOf[Iterator[T]] } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index d2f7baf928b62..0444ffa346638 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -37,7 +37,7 @@ import org.apache.spark.util._ private[spark] sealed trait BlockValues private[spark] case class ByteBufferValues(buffer: ByteBuffer) extends BlockValues private[spark] case class IteratorValues(iterator: Iterator[Any]) extends BlockValues -private[spark] case class ArrayBufferValues(buffer: ArrayBuffer[Any]) extends BlockValues +private[spark] case class ArrayValues(buffer: Array[Any]) extends BlockValues private[spark] class BlockManager( executorId: String, @@ -451,16 +451,12 @@ private[spark] class BlockManager( val values = dataDeserialize(blockId, bytes) if (level.deserialized) { // Cache the values before returning them - // TODO: Consider creating a putValues that also takes in a iterator? - val valuesBuffer = new ArrayBuffer[Any] - valuesBuffer ++= values - memoryStore.putValues(blockId, valuesBuffer, level, returnValues = true).data - match { - case Left(values2) => - return Some(values2) - case _ => - throw new SparkException("Memory store did not return an iterator") - } + memoryStore.putValues(blockId, values, level, returnValues = true).data match { + case Left(values2) => + return Some(values2) + case _ => + throw new SparkException("Memory store did not return an iterator") + } } else { return Some(values) } @@ -576,11 +572,11 @@ private[spark] class BlockManager( */ def put( blockId: BlockId, - values: ArrayBuffer[Any], + values: Array[Any], level: StorageLevel, tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { require(values != null, "Values is null") - doPut(blockId, ArrayBufferValues(values), level, tellMaster) + doPut(blockId, ArrayValues(values), level, tellMaster) } /** @@ -682,7 +678,7 @@ private[spark] class BlockManager( val result = data match { case IteratorValues(iterator) => blockStore.putValues(blockId, iterator, level, returnValues) - case ArrayBufferValues(array) => + case ArrayValues(array) => blockStore.putValues(blockId, array, level, returnValues) case ByteBufferValues(bytes) => bytes.rewind() @@ -814,7 +810,7 @@ private[spark] class BlockManager( */ def dropFromMemory( blockId: BlockId, - data: Either[ArrayBuffer[Any], ByteBuffer]): Option[BlockStatus] = { + data: Either[Array[Any], ByteBuffer]): Option[BlockStatus] = { logInfo(s"Dropping block $blockId from memory") val info = blockInfo.get(blockId).orNull diff --git a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala index b9b53b1a2f118..68fd2b2156ee7 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala @@ -45,7 +45,7 @@ private[spark] abstract class BlockStore(val blockManager: BlockManager) extends def putValues( blockId: BlockId, - values: ArrayBuffer[Any], + values: Array[Any], level: StorageLevel, returnValues: Boolean): PutResult diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index ebff0cb5ba153..4787e034f5f83 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -21,8 +21,6 @@ import java.io.{FileOutputStream, RandomAccessFile} import java.nio.ByteBuffer import java.nio.channels.FileChannel.MapMode -import scala.collection.mutable.ArrayBuffer - import org.apache.spark.Logging import org.apache.spark.serializer.Serializer import org.apache.spark.util.Utils @@ -59,7 +57,7 @@ private class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManage override def putValues( blockId: BlockId, - values: ArrayBuffer[Any], + values: Array[Any], level: StorageLevel, returnValues: Boolean): PutResult = { putValues(blockId, values.toIterator, level, returnValues) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 71f66c826c5b3..c12a9d7021223 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -27,7 +27,7 @@ import org.apache.spark.util.{SizeEstimator, Utils} private case class MemoryEntry(value: Any, size: Long, deserialized: Boolean) /** - * Stores blocks in memory, either as ArrayBuffers of deserialized Java objects or as + * Stores blocks in memory, either as Arrays of deserialized Java objects or as * serialized ByteBuffers. */ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) @@ -55,8 +55,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) bytes.rewind() if (level.deserialized) { val values = blockManager.dataDeserialize(blockId, bytes) - val elements = new ArrayBuffer[Any] - elements ++= values + val elements = values.toArray val sizeEstimate = SizeEstimator.estimate(elements.asInstanceOf[AnyRef]) val putAttempt = tryToPut(blockId, elements, sizeEstimate, deserialized = true) PutResult(sizeEstimate, Left(values.toIterator), putAttempt.droppedBlocks) @@ -68,7 +67,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) override def putValues( blockId: BlockId, - values: ArrayBuffer[Any], + values: Array[Any], level: StorageLevel, returnValues: Boolean): PutResult = { if (level.deserialized) { @@ -87,9 +86,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) values: Iterator[Any], level: StorageLevel, returnValues: Boolean): PutResult = { - val valueEntries = new ArrayBuffer[Any]() - valueEntries ++= values - putValues(blockId, valueEntries, level, returnValues) + putValues(blockId, values.toArray, level, returnValues) } override def getBytes(blockId: BlockId): Option[ByteBuffer] = { @@ -99,7 +96,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) if (entry == null) { None } else if (entry.deserialized) { - Some(blockManager.dataSerialize(blockId, entry.value.asInstanceOf[ArrayBuffer[Any]].iterator)) + Some(blockManager.dataSerialize(blockId, entry.value.asInstanceOf[Array[Any]].iterator)) } else { Some(entry.value.asInstanceOf[ByteBuffer].duplicate()) // Doesn't actually copy the data } @@ -112,7 +109,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) if (entry == null) { None } else if (entry.deserialized) { - Some(entry.value.asInstanceOf[ArrayBuffer[Any]].iterator) + Some(entry.value.asInstanceOf[Array[Any]].iterator) } else { val buffer = entry.value.asInstanceOf[ByteBuffer].duplicate() // Doesn't actually copy data Some(blockManager.dataDeserialize(blockId, buffer)) @@ -149,8 +146,8 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) /** * Try to put in a set of values, if we can free up enough space. The value should either be - * an ArrayBuffer if deserialized is true or a ByteBuffer otherwise. Its (possibly estimated) - * size must also be passed by the caller. + * an Array if deserialized is true or a ByteBuffer otherwise. Its (possibly estimated) size + * must also be passed by the caller. * * Lock on the object putLock to ensure that all the put requests and its associated block * dropping is done by only on thread at a time. Otherwise while one thread is dropping @@ -193,7 +190,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) // Tell the block manager that we couldn't put it in memory so that it can drop it to // disk if the block allows disk storage. val data = if (deserialized) { - Left(value.asInstanceOf[ArrayBuffer[Any]]) + Left(value.asInstanceOf[Array[Any]]) } else { Right(value.asInstanceOf[ByteBuffer].duplicate()) } @@ -227,7 +224,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) if (maxMemory - currentMemory < space) { val rddToAdd = getRddId(blockIdToAdd) - val selectedBlocks = new ArrayBuffer[BlockId]() + val selectedBlocks = new ArrayBuffer[BlockId] var selectedMemory = 0L // This is synchronized to ensure that the set of entries is not changed @@ -254,7 +251,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) // future safety. if (entry != null) { val data = if (entry.deserialized) { - Left(entry.value.asInstanceOf[ArrayBuffer[Any]]) + Left(entry.value.asInstanceOf[Array[Any]]) } else { Right(entry.value.asInstanceOf[ByteBuffer].duplicate()) } diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala index d8ff4ff6bd42c..99faf4c52f552 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala @@ -20,8 +20,6 @@ package org.apache.spark.storage import java.io.IOException import java.nio.ByteBuffer -import scala.collection.mutable.ArrayBuffer - import tachyon.client.{ReadType, WriteType} import org.apache.spark.Logging @@ -47,7 +45,7 @@ private class TachyonStore( override def putValues( blockId: BlockId, - values: ArrayBuffer[Any], + values: Array[Any], level: StorageLevel, returnValues: Boolean): PutResult = { putValues(blockId, values.toIterator, level, returnValues) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index ce8316bb14891..4579ffea678bd 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -110,8 +110,7 @@ private[streaming] class ReceiverSupervisorImpl( ) { val blockId = optionalBlockId.getOrElse(nextBlockId) val time = System.currentTimeMillis - blockManager.put(blockId, arrayBuffer.asInstanceOf[ArrayBuffer[Any]], - storageLevel, tellMaster = true) + blockManager.put(blockId, arrayBuffer.toArray[Any], storageLevel, tellMaster = true) logDebug("Pushed block " + blockId + " in " + (System.currentTimeMillis - time) + " ms") reportPushedBlock(blockId, arrayBuffer.size, optionalMetadata) } From bbd3eea466b284c828962db65248de349a2964da Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 20 Jun 2014 12:24:37 -0700 Subject: [PATCH 03/55] Fix CacheManagerSuite to use Array In addition, avoid using EasyMock for one of our tests, which expects BlockManager#put to be called with an Array[Any] parameter. Even with all the EasyMock matchers, it is impossible to match an Array[Any] because of a combination of the following: (1) the fact that Arrays are not covariant, (2) EasyMock provides `aryEq` matchers for all the Java primitive types, which conflict with Any, and (3) EasyMock's super general matchers like `anyObject` or `isA` also do not match for some reason. --- .../org/apache/spark/CacheManagerSuite.scala | 25 ++++++++----------- 1 file changed, 11 insertions(+), 14 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index 4f178db40f638..2ae1a09bdbce3 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark -import scala.collection.mutable.ArrayBuffer - import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.mock.EasyMockSugar @@ -51,22 +49,21 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } test("get uncached rdd") { - expecting { - blockManager.get(RDDBlockId(0, 0)).andReturn(None) - blockManager.put(RDDBlockId(0, 0), ArrayBuffer[Any](1, 2, 3, 4), StorageLevel.MEMORY_ONLY, - true).andStubReturn(Seq[(BlockId, BlockStatus)]()) - } - - whenExecuting(blockManager) { - val context = new TaskContext(0, 0, 0) - val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) - assert(value.toList === List(1, 2, 3, 4)) - } + // Do not mock this test, because matching Array[Any] in blockManager.put is a losing battle. + // You have been warned. + blockManager = sc.env.blockManager + cacheManager = sc.env.cacheManager + val context = new TaskContext(0, 0, 0) + val computeValue = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) + val getValue = blockManager.get(RDDBlockId(rdd.id, split.index)) + assert(computeValue.toList === List(1, 2, 3, 4)) + assert(getValue.isDefined, "Block cached from getOrCompute is not found!") + assert(getValue.get.toArray === List(1, 2, 3, 4)) } test("get cached rdd") { expecting { - blockManager.get(RDDBlockId(0, 0)).andReturn(Some(ArrayBuffer(5, 6, 7).iterator)) + blockManager.get(RDDBlockId(0, 0)).andReturn(Some(Array(5, 6, 7).iterator)) } whenExecuting(blockManager) { From 776aec9e20f4a0c4beffe45ca07511bcd3fcba32 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 20 Jun 2014 15:58:46 -0700 Subject: [PATCH 04/55] Prevent OOM if a single RDD partition is too large The idea is to always use at least a fixed amount of memory (M bytes) to unroll RDD partitions. This space is not reserved, but instead allocated dynamically by dropping existing blocks when necessary. We maintain this buffer as a global quantity shared across all cores. The way we synchronize the usage of this buffer is very similar to the way we share memory across all threads for shuffle aggregation. In particular, each thread cautiously requests for more memory periodically, and if there is not enough global memory to grant the request, the thread concedes and spills. However, this relies on the accuracy of size estimation, which is not guaranteed. Therefore, as in the shuffle case, we need an equivalent spark.storage.safetyFraction in case size estimation is slightly off. We expose M to the user as spark.storage.bufferFraction, with a default value of 0.2. --- .../scala/org/apache/spark/CacheManager.scala | 86 +++++++++++++++++-- .../scala/org/apache/spark/SparkEnv.scala | 6 +- .../org/apache/spark/executor/Executor.scala | 8 +- .../apache/spark/storage/BlockManager.scala | 16 +++- .../org/apache/spark/storage/DiskStore.scala | 2 +- .../apache/spark/storage/MemoryStore.scala | 8 +- .../apache/spark/storage/TachyonStore.scala | 2 +- .../collection/ExternalAppendOnlyMap.scala | 4 +- .../util/collection/PrimitiveVector.scala | 10 +++ .../SizeTrackingAppendOnlyBuffer.scala | 4 + 10 files changed, 125 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 8e5b787e34c90..f30a28b2cfeed 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -17,10 +17,12 @@ package org.apache.spark -import scala.collection.mutable.{ArrayBuffer, HashSet} +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer import org.apache.spark.rdd.RDD import org.apache.spark.storage._ +import org.apache.spark.util.collection.SizeTrackingAppendOnlyBuffer /** * Spark class responsible for passing RDDs partition contents to the BlockManager and making @@ -29,7 +31,14 @@ import org.apache.spark.storage._ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { /** Keys of RDD partitions that are being computed/loaded. */ - private val loading = new HashSet[RDDBlockId]() + private val loading = new mutable.HashSet[RDDBlockId] + + /** + * The amount of space ensured for unrolling partitions, shared across all cores. + * This space is not reserved in advance, but allocated dynamically by dropping existing blocks. + * It must be a lazy val in order to access a mocked BlockManager's conf in tests properly. + */ + private lazy val globalBufferMemory = BlockManager.getBufferMemory(blockManager.conf) /** Gets or computes an RDD partition. Used by RDD.iterator() when an RDD is cached. */ def getOrCompute[T]( @@ -142,9 +151,76 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { * to the BlockManager as an iterator and expect to read it back later. This is because * we may end up dropping a partition from memory store before getting it back, e.g. * when the entirety of the RDD does not fit in memory. */ - val elements = values.toArray[Any] - updatedBlocks ++= blockManager.put(key, elements, storageLevel, tellMaster = true) - elements.iterator.asInstanceOf[Iterator[T]] + + var count = 0 // The number of elements unrolled so far + var dropPartition = false // Whether to drop the new partition from memory + var previousSize = 0L // Previous estimate of the size of our buffer + val memoryRequestPeriod = 1000 // How frequently we request for more memory for our buffer + + val threadId = Thread.currentThread().getId + val cacheMemoryMap = SparkEnv.get.cacheMemoryMap + var buffer = new SizeTrackingAppendOnlyBuffer[Any] + + /* While adding values to the in-memory buffer, periodically check whether the memory + * restrictions for unrolling partitions are still satisfied. If not, stop immediately, + * and persist the partition to disk if specified by the storage level. This check is + * a safeguard against the scenario when a single partition does not fit in memory. */ + while (values.hasNext && !dropPartition) { + buffer += values.next() + count += 1 + if (count % memoryRequestPeriod == 1) { + // Calculate the amount of memory to request from the global memory pool + val currentSize = buffer.estimateSize() + val delta = math.max(currentSize - previousSize, 0) + val memoryToRequest = currentSize + delta + previousSize = currentSize + + // Atomically check whether there is sufficient memory in the global pool to continue + cacheMemoryMap.synchronized { + val previouslyOccupiedMemory = cacheMemoryMap.get(threadId).getOrElse(0L) + val otherThreadsMemory = cacheMemoryMap.values.sum - previouslyOccupiedMemory + + // Request for memory for the local buffer, and return whether request is granted + def requestForMemory(): Boolean = { + val availableMemory = blockManager.memoryStore.freeMemory - otherThreadsMemory + val granted = availableMemory > memoryToRequest + if (granted) { cacheMemoryMap(threadId) = memoryToRequest } + granted + } + + // If the first request is not granted, try again after ensuring free space + // If there is still not enough space, give up and drop the partition + if (!requestForMemory()) { + val result = blockManager.memoryStore.ensureFreeSpace(key, globalBufferMemory) + updatedBlocks ++= result.droppedBlocks + dropPartition = !requestForMemory() + } + } + } + } + + if (!dropPartition) { + // We have successfully unrolled the entire partition, so cache it in memory + updatedBlocks ++= blockManager.put(key, buffer.array, storageLevel, tellMaster = true) + buffer.iterator.asInstanceOf[Iterator[T]] + } else { + // We have exceeded our collective quota. This partition will not be cached in memory. + val persistToDisk = storageLevel.useDisk + logWarning(s"Failed to cache $key in memory! There is not enough space to unroll the " + + s"entire partition. " + (if (persistToDisk) "Persisting to disk instead." else "")) + var newValues = (buffer.iterator ++ values).asInstanceOf[Iterator[T]] + if (persistToDisk) { + val newLevel = StorageLevel(storageLevel.useDisk, useMemory = false, + storageLevel.useOffHeap, storageLevel.deserialized, storageLevel.replication) + newValues = putInBlockManager[T](key, newValues, newLevel, updatedBlocks) + // Free up buffer for other threads + buffer = null + cacheMemoryMap.synchronized { + cacheMemoryMap(threadId) = 0 + } + } + newValues + } } } diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 8dfa8cc4b5b3f..0d6bce064525b 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -67,10 +67,14 @@ class SparkEnv ( val metricsSystem: MetricsSystem, val conf: SparkConf) extends Logging { - // A mapping of thread ID to amount of memory used for shuffle in bytes + // A mapping of thread ID to amount of memory, in bytes, used for shuffle // All accesses should be manually synchronized val shuffleMemoryMap = mutable.HashMap[Long, Long]() + // A mapping of thread ID to amount of memory, in bytes, used for unrolling an RDD partition + // All accesses should be manually synchronized + val cacheMemoryMap = mutable.HashMap[Long, Long]() + private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() // A general, soft-reference map for metadata needed during HadoopRDD split computation diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index baee7a216a7c3..1c95f4d9ba136 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -259,11 +259,11 @@ private[spark] class Executor( } } } finally { - // TODO: Unregister shuffle memory only for ResultTask + val threadId = Thread.currentThread().getId val shuffleMemoryMap = env.shuffleMemoryMap - shuffleMemoryMap.synchronized { - shuffleMemoryMap.remove(Thread.currentThread().getId) - } + val cacheMemoryMap = env.cacheMemoryMap + shuffleMemoryMap.synchronized { shuffleMemoryMap.remove(threadId) } + cacheMemoryMap.synchronized { cacheMemoryMap.remove(threadId) } runningTasks.remove(taskId) } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 0444ffa346638..002f03ff622e4 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -61,9 +61,9 @@ private[spark] class BlockManager( // Actual storage of where blocks are kept private var tachyonInitialized = false - private[storage] val memoryStore = new MemoryStore(this, maxMemory) - private[storage] val diskStore = new DiskStore(this, diskBlockManager) - private[storage] lazy val tachyonStore: TachyonStore = { + private[spark] val memoryStore = new MemoryStore(this, maxMemory) + private[spark] val diskStore = new DiskStore(this, diskBlockManager) + private[spark] lazy val tachyonStore: TachyonStore = { val storeDir = conf.get("spark.tachyonStore.baseDir", "/tmp_spark_tachyon") val appFolderName = conf.get("spark.tachyonStore.folderName") val tachyonStorePath = s"$storeDir/$appFolderName/${this.executorId}" @@ -1049,9 +1049,17 @@ private[spark] class BlockManager( private[spark] object BlockManager extends Logging { private val ID_GENERATOR = new IdGenerator + /** Return the total amount of storage memory available. */ private def getMaxMemory(conf: SparkConf): Long = { val memoryFraction = conf.getDouble("spark.storage.memoryFraction", 0.6) - (Runtime.getRuntime.maxMemory * memoryFraction).toLong + val safetyFraction = conf.getDouble("spark.storage.safetyFraction", 0.9) + (Runtime.getRuntime.maxMemory * memoryFraction * safetyFraction).toLong + } + + /** Return the amount of storage memory used for unrolling RDD partitions. */ + def getBufferMemory(conf: SparkConf): Long = { + val bufferFraction = conf.getDouble("spark.storage.bufferFraction", 0.2) + (getMaxMemory(conf) * bufferFraction).toLong } def getHeartBeatFrequency(conf: SparkConf): Long = diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index 4787e034f5f83..422179b44b189 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -28,7 +28,7 @@ import org.apache.spark.util.Utils /** * Stores BlockManager blocks on disk. */ -private class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManager) +private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManager) extends BlockStore(blockManager) with Logging { val minMemoryMapBytes = blockManager.conf.getLong("spark.storage.memoryMapThreshold", 2 * 4096L) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index c12a9d7021223..b4363b312bc64 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -30,7 +30,7 @@ private case class MemoryEntry(value: Any, size: Long, deserialized: Boolean) * Stores blocks in memory, either as Arrays of deserialized Java objects or as * serialized ByteBuffers. */ -private class MemoryStore(blockManager: BlockManager, maxMemory: Long) +private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) extends BlockStore(blockManager) { private val entries = new LinkedHashMap[BlockId, MemoryEntry](32, 0.75f, true) @@ -212,7 +212,9 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) * * Return whether there is enough free space, along with the blocks dropped in the process. */ - private def ensureFreeSpace(blockIdToAdd: BlockId, space: Long): ResultWithDroppedBlocks = { + private[spark] def ensureFreeSpace( + blockIdToAdd: BlockId, + space: Long): ResultWithDroppedBlocks = { logInfo(s"ensureFreeSpace($space) called with curMem=$currentMemory, maxMem=$maxMemory") val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] @@ -274,6 +276,6 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } } -private case class ResultWithDroppedBlocks( +private[spark] case class ResultWithDroppedBlocks( success: Boolean, droppedBlocks: Seq[(BlockId, BlockStatus)]) diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala index 99faf4c52f552..c843a50b70304 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala @@ -28,7 +28,7 @@ import org.apache.spark.util.Utils /** * Stores BlockManager blocks on Tachyon. */ -private class TachyonStore( +private[spark] class TachyonStore( blockManager: BlockManager, tachyonManager: TachyonBlockManager) extends BlockStore(blockManager: BlockManager) with Logging { diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 288badd3160f8..40e7521838f2b 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -130,9 +130,9 @@ class ExternalAppendOnlyMap[K, V, C]( // this map to grow and, if possible, allocate the required amount shuffleMemoryMap.synchronized { val threadId = Thread.currentThread().getId - val previouslyOccupiedMemory = shuffleMemoryMap.get(threadId) + val previouslyOccupiedMemory = shuffleMemoryMap.get(threadId).getOrElse(0L) val availableMemory = maxMemoryThreshold - - (shuffleMemoryMap.values.sum - previouslyOccupiedMemory.getOrElse(0L)) + (shuffleMemoryMap.values.sum - previouslyOccupiedMemory) // Assume map growth factor is 2x shouldSpill = availableMemory < mapSize * 2 diff --git a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala index 3a22d998d1612..fff201f3b518e 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala @@ -50,6 +50,16 @@ class PrimitiveVector[@specialized(Long, Int, Double) V: ClassTag](initialSize: def size: Int = _numElements + def iterator: Iterator[V] = new Iterator[V] { + var index = 0 + override def hasNext: Boolean = index < _numElements + override def next(): V = { + val value = _array(index) + index += 1 + value + } + } + /** Gets the underlying array backing this vector. */ def array: Array[V] = _array diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyBuffer.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyBuffer.scala index 574c454557ec8..2ea4b1d996fe4 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyBuffer.scala @@ -36,4 +36,8 @@ private[spark] class SizeTrackingAppendOnlyBuffer[T: ClassTag] resetSamples() this } + + override def array: Array[T] = { + super.iterator.toArray + } } From f94f5af0091ffcb4871024e28c149bc8c73c3c87 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 20 Jun 2014 18:50:00 -0700 Subject: [PATCH 05/55] Update a few comments (minor) --- .../spark/util/collection/SizeTracker.scala | 23 +++++++++++++------ 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTracker.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTracker.scala index a40359038545c..70b2e63bf1df3 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/SizeTracker.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTracker.scala @@ -22,9 +22,9 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.util.SizeEstimator /** - * A general interface for collections that keeps track of its estimated size in bytes. + * A general interface for collections to keep track of their estimated sizes in bytes. * We sample with a slow exponential back-off using the SizeEstimator to amortize the time, - * as each call to SizeEstimator can take a sizable amount of time (order of a few milliseconds). + * as each call to SizeEstimator is somewhat expensive (order of a few milliseconds). */ private[spark] trait SizeTracker { @@ -36,7 +36,7 @@ private[spark] trait SizeTracker { */ private val SAMPLE_GROWTH_RATE = 1.1 - /** All samples taken since last resetSamples(). Only the last two are used for extrapolation. */ + /** Samples taken since last resetSamples(). Only the last two are kept for extrapolation. */ private val samples = new ArrayBuffer[Sample] /** The average number of bytes per update between our last two samples. */ @@ -50,7 +50,10 @@ private[spark] trait SizeTracker { resetSamples() - /** Called after the collection undergoes a dramatic change in size. */ + /** + * Reset samples collected so far. + * This should be called after the collection undergoes a dramatic change in size. + */ protected def resetSamples(): Unit = { numUpdates = 1 nextSampleNum = 1 @@ -58,7 +61,9 @@ private[spark] trait SizeTracker { takeSample() } - /** Callback to be invoked after an update. */ + /** + * Callback to be invoked after every update. + */ protected def afterUpdate(): Unit = { numUpdates += 1 if (nextSampleNum == numUpdates) { @@ -66,7 +71,9 @@ private[spark] trait SizeTracker { } } - /** Takes a new sample of the current collection's size. */ + /** + * Take a new sample of the current collection's size. + */ private def takeSample(): Unit = { samples += Sample(SizeEstimator.estimate(this), numUpdates) // Only use the last two samples to extrapolate @@ -83,7 +90,9 @@ private[spark] trait SizeTracker { nextSampleNum = math.ceil(numUpdates * SAMPLE_GROWTH_RATE).toLong } - /** Estimates the current size of the collection in bytes. O(1) time. */ + /** + * Estimate the current size of the collection in bytes. O(1) time. + */ def estimateSize(): Long = { assert(samples.nonEmpty) val extrapolatedDelta = bytesPerUpdate * (numUpdates - samples.last.numUpdates) From df472651a4fd776ab39d40e307be6a5b108b193c Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 20 Jun 2014 22:00:18 -0700 Subject: [PATCH 06/55] Fix binary incompatibility --- project/MimaExcludes.scala | 25 ++++++++++++++++++------- 1 file changed, 18 insertions(+), 7 deletions(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 042fdfcc47261..09a91fe7317bd 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -53,19 +53,30 @@ object MimaExcludes { "org.apache.spark.api.java.JavaRDDLike.countApproxDistinct$default$1"), ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.api.java.JavaDoubleRDD.countApproxDistinct$default$1"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.java.JavaRDDLike.partitions"), ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.storage.MemoryStore.Entry"), ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.rdd.PairRDDFunctions.org$apache$spark$rdd$PairRDDFunctions$$" - + "createZero$1") - ) ++ - Seq( // Ignore some private methods in ALS. + "org.apache.spark.rdd.PairRDDFunctions." + + "org$apache$spark$rdd$PairRDDFunctions$$createZero$1"), + // The BlockStore interface now uses Arrays instead of ArrayBuffers + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.storage.MemoryStore.putValues"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.storage.DiskStore.putValues"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.storage.TachyonSTore.putValues"), + // Ignore some private methods in ALS. + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.mllib.recommendation.ALS." + + "org$apache$spark$mllib$recommendation$ALS$^dateFeatures"), + // The only public constructor is the one without arguments. ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$^dateFeatures"), - ProblemFilters.exclude[MissingMethodProblem]( // The only public constructor is the one without arguments. "org.apache.spark.mllib.recommendation.ALS.this"), ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$$$default$7") + "org.apache.spark.mllib.recommendation.ALS." + + "org$apache$spark$mllib$recommendation$ALS$$$default$7") ) ++ MimaBuild.excludeSparkClass("rdd.ZippedRDD") ++ MimaBuild.excludeSparkClass("rdd.ZippedPartition") ++ From 50cae44a042f758f2478bd501881d6cb30b8ca3e Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sat, 21 Jun 2014 13:40:07 -0700 Subject: [PATCH 07/55] Remove now duplicate mima exclude --- project/MimaExcludes.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 181614354994a..7389ef506ae11 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -36,8 +36,10 @@ object MimaExcludes { case v if v.startsWith("1.1") => Seq(MimaBuild.excludeSparkPackage("graphx")) ++ Seq( - // Adding new method to JavaRDLike trait - we should probably mark this as a developer API. - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.partitions"), + // Adding new method to JavaRDLike trait + // We should probably mark this as a developer API. + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.java.JavaRDDLike.partitions"), // We made a mistake earlier (ed06500d3) in the Java API to use default parameter values // for countApproxDistinct* functions, which does not work in Java. We later removed // them, and use the following to tell Mima to not care about them. @@ -55,8 +57,6 @@ object MimaExcludes { "org.apache.spark.api.java.JavaRDDLike.countApproxDistinct$default$1"), ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.api.java.JavaDoubleRDD.countApproxDistinct$default$1"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.JavaRDDLike.partitions"), ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.storage.MemoryStore.Entry"), ProblemFilters.exclude[MissingMethodProblem]( From 11eb921f74e962ec372a1e90cdeaba458963a85a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sat, 21 Jun 2014 13:56:08 -0700 Subject: [PATCH 08/55] Clarify comment (minor) --- core/src/test/scala/org/apache/spark/CacheManagerSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index 2ae1a09bdbce3..20798a34755c5 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -49,8 +49,8 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } test("get uncached rdd") { - // Do not mock this test, because matching Array[Any] in blockManager.put is a losing battle. - // You have been warned. + // Do not mock this test, because attempting to match Array[Any], which is not covariant, + // in blockManager.put is a losing battle. You have been warned. blockManager = sc.env.blockManager cacheManager = sc.env.cacheManager val context = new TaskContext(0, 0, 0) From 87aa75ccf937c6924c2ca88edfc37af5aa42e59f Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sat, 21 Jun 2014 14:54:45 -0700 Subject: [PATCH 09/55] Fix mima excludes again (typo) --- project/MimaExcludes.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 7389ef506ae11..7b42b35a06380 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -68,7 +68,7 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleMethTypeProblem]( "org.apache.spark.storage.DiskStore.putValues"), ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.storage.TachyonSTore.putValues"), + "org.apache.spark.storage.TachyonStore.putValues"), // Ignore some private methods in ALS. ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.mllib.recommendation.ALS." + From ea02eec1d31191d7713fbf4001b4ffaf320a0e3b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 3 Jul 2014 17:54:38 -0700 Subject: [PATCH 10/55] Fix tests --- core/src/test/scala/org/apache/spark/CacheManagerSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index ba9eda7fb36f5..3aaa10c459bb8 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -59,12 +59,12 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar val getValue = blockManager.get(RDDBlockId(rdd.id, split.index)) assert(computeValue.toList === List(1, 2, 3, 4)) assert(getValue.isDefined, "Block cached from getOrCompute is not found!") - assert(getValue.get.toArray === List(1, 2, 3, 4)) + assert(getValue.get.data.toArray === List(1, 2, 3, 4)) } test("get cached rdd") { expecting { - val result = new BlockResult(ArrayBuffer(5, 6, 7).iterator, DataReadMethod.Memory, 12) + val result = new BlockResult(Array(5, 6, 7).iterator, DataReadMethod.Memory, 12) blockManager.get(RDDBlockId(0, 0)).andReturn(Some(result)) } From d5dd3b4aac6b3992591813f3bd1fcfb9f2adbec2 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 3 Jul 2014 19:53:31 -0700 Subject: [PATCH 11/55] Free buffer memory in finally --- .../scala/org/apache/spark/CacheManager.scala | 113 ++++++++++-------- 1 file changed, 60 insertions(+), 53 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index ee25458f2ad48..f0eef9bb27945 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -163,65 +163,72 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { val cacheMemoryMap = SparkEnv.get.cacheMemoryMap var buffer = new SizeTrackingAppendOnlyBuffer[Any] - /* While adding values to the in-memory buffer, periodically check whether the memory - * restrictions for unrolling partitions are still satisfied. If not, stop immediately, - * and persist the partition to disk if specified by the storage level. This check is - * a safeguard against the scenario when a single partition does not fit in memory. */ - while (values.hasNext && !dropPartition) { - buffer += values.next() - count += 1 - if (count % memoryRequestPeriod == 1) { - // Calculate the amount of memory to request from the global memory pool - val currentSize = buffer.estimateSize() - val delta = math.max(currentSize - previousSize, 0) - val memoryToRequest = currentSize + delta - previousSize = currentSize - - // Atomically check whether there is sufficient memory in the global pool to continue - cacheMemoryMap.synchronized { - val previouslyOccupiedMemory = cacheMemoryMap.get(threadId).getOrElse(0L) - val otherThreadsMemory = cacheMemoryMap.values.sum - previouslyOccupiedMemory - - // Request for memory for the local buffer, and return whether request is granted - def requestForMemory(): Boolean = { - val availableMemory = blockManager.memoryStore.freeMemory - otherThreadsMemory - val granted = availableMemory > memoryToRequest - if (granted) { cacheMemoryMap(threadId) = memoryToRequest } - granted - } - - // If the first request is not granted, try again after ensuring free space - // If there is still not enough space, give up and drop the partition - if (!requestForMemory()) { - val result = blockManager.memoryStore.ensureFreeSpace(key, globalBufferMemory) - updatedBlocks ++= result.droppedBlocks - dropPartition = !requestForMemory() + try { + /* While adding values to the in-memory buffer, periodically check whether the memory + * restrictions for unrolling partitions are still satisfied. If not, stop immediately, + * and persist the partition to disk if specified by the storage level. This check is + * a safeguard against the scenario when a single partition does not fit in memory. */ + while (values.hasNext && !dropPartition) { + buffer += values.next() + count += 1 + if (count % memoryRequestPeriod == 1) { + // Calculate the amount of memory to request from the global memory pool + val currentSize = buffer.estimateSize() + val delta = math.max(currentSize - previousSize, 0) + val memoryToRequest = currentSize + delta + previousSize = currentSize + + // Atomically check whether there is sufficient memory in the global pool to continue + cacheMemoryMap.synchronized { + val previouslyOccupiedMemory = cacheMemoryMap.get(threadId).getOrElse(0L) + val otherThreadsMemory = cacheMemoryMap.values.sum - previouslyOccupiedMemory + + // Request for memory for the local buffer, and return whether request is granted + def requestForMemory(): Boolean = { + val availableMemory = blockManager.memoryStore.freeMemory - otherThreadsMemory + val granted = availableMemory > memoryToRequest + if (granted) { cacheMemoryMap(threadId) = memoryToRequest } + granted + } + + // If the first request is not granted, try again after ensuring free space + // If there is still not enough space, give up and drop the partition + if (!requestForMemory()) { + val result = blockManager.memoryStore.ensureFreeSpace(key, globalBufferMemory) + updatedBlocks ++= result.droppedBlocks + dropPartition = !requestForMemory() + } } } } - } - if (!dropPartition) { - // We have successfully unrolled the entire partition, so cache it in memory - updatedBlocks ++= blockManager.put(key, buffer.array, storageLevel, tellMaster = true) - buffer.iterator.asInstanceOf[Iterator[T]] - } else { - // We have exceeded our collective quota. This partition will not be cached in memory. - val persistToDisk = storageLevel.useDisk - logWarning(s"Failed to cache $key in memory! There is not enough space to unroll the " + - s"entire partition. " + (if (persistToDisk) "Persisting to disk instead." else "")) - var newValues = (buffer.iterator ++ values).asInstanceOf[Iterator[T]] - if (persistToDisk) { - val newLevel = StorageLevel(storageLevel.useDisk, useMemory = false, - storageLevel.useOffHeap, storageLevel.deserialized, storageLevel.replication) - newValues = putInBlockManager[T](key, newValues, newLevel, updatedBlocks) - // Free up buffer for other threads - buffer = null - cacheMemoryMap.synchronized { - cacheMemoryMap(threadId) = 0 + if (!dropPartition) { + // We have successfully unrolled the entire partition, so cache it in memory + updatedBlocks ++= blockManager.put(key, buffer.array, storageLevel, tellMaster = true) + buffer.iterator.asInstanceOf[Iterator[T]] + } else { + // We have exceeded our collective quota. This partition will not be cached in memory. + val persistToDisk = storageLevel.useDisk + logWarning(s"Failed to cache $key in memory! There is not enough space to unroll the " + + s"entire partition. " + (if (persistToDisk) "Persisting to disk instead." else "")) + var newValues = (buffer.iterator ++ values).asInstanceOf[Iterator[T]] + if (persistToDisk) { + val newLevel = StorageLevel( + storageLevel.useDisk, + useMemory = false, + storageLevel.useOffHeap, + deserialized = false, + storageLevel.replication) + newValues = putInBlockManager[T](key, newValues, newLevel, updatedBlocks) } + newValues + } + } finally { + // Free up buffer for other threads + buffer = null + cacheMemoryMap.synchronized { + cacheMemoryMap(threadId) = 0 } - newValues } } } From 195abd771129f33a429c0b4437ba4f6e71b7259c Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 7 Jul 2014 16:49:58 -0700 Subject: [PATCH 12/55] Refactor: move unfold logic to MemoryStore This logic is also needed in other parts of the MemoryStore, e.g. when we try to store deserialized bytes in memory. The unfolding logic is specific to the memory case, so it makes sense for it to reside in MemoryStore, as opposed to the higher level CacheManager. --- .../scala/org/apache/spark/CacheManager.scala | 103 +++++------------- .../apache/spark/storage/MemoryStore.scala | 96 +++++++++++++++- 2 files changed, 119 insertions(+), 80 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index f0eef9bb27945..01c1b9f02898e 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -34,13 +34,6 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { /** Keys of RDD partitions that are being computed/loaded. */ private val loading = new mutable.HashSet[RDDBlockId] - /** - * The amount of space ensured for unrolling partitions, shared across all cores. - * This space is not reserved in advance, but allocated dynamically by dropping existing blocks. - * It must be a lazy val in order to access a mocked BlockManager's conf in tests properly. - */ - private lazy val globalBufferMemory = BlockManager.getBufferMemory(blockManager.conf) - /** Gets or computes an RDD partition. Used by RDD.iterator() when an RDD is cached. */ def getOrCompute[T]( rdd: RDD[T], @@ -137,10 +130,12 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { updatedBlocks: ArrayBuffer[(BlockId, BlockStatus)]): Iterator[T] = { if (!storageLevel.useMemory) { - /* This RDD is not to be cached in memory, so we can just pass the computed values - * as an iterator directly to the BlockManager, rather than first fully unrolling + /* + * This RDD is not to be cached in memory, so we can just pass the computed values + * as an iterator directly to the BlockManager, rather than first fully unfolding * it in memory. The latter option potentially uses much more memory and risks OOM - * exceptions that can be avoided. */ + * exceptions that can be avoided. + */ updatedBlocks ++= blockManager.put(key, values, storageLevel, tellMaster = true) blockManager.get(key) match { case Some(v) => v.data.asInstanceOf[Iterator[T]] @@ -149,69 +144,28 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { throw new BlockException(key, s"Block manager failed to return cached value for $key!") } } else { - /* This RDD is to be cached in memory. In this case we cannot pass the computed values + /* + * This RDD is to be cached in memory. In this case we cannot pass the computed values * to the BlockManager as an iterator and expect to read it back later. This is because * we may end up dropping a partition from memory store before getting it back, e.g. - * when the entirety of the RDD does not fit in memory. */ - - var count = 0 // The number of elements unrolled so far - var dropPartition = false // Whether to drop the new partition from memory - var previousSize = 0L // Previous estimate of the size of our buffer - val memoryRequestPeriod = 1000 // How frequently we request for more memory for our buffer - - val threadId = Thread.currentThread().getId - val cacheMemoryMap = SparkEnv.get.cacheMemoryMap - var buffer = new SizeTrackingAppendOnlyBuffer[Any] - - try { - /* While adding values to the in-memory buffer, periodically check whether the memory - * restrictions for unrolling partitions are still satisfied. If not, stop immediately, - * and persist the partition to disk if specified by the storage level. This check is - * a safeguard against the scenario when a single partition does not fit in memory. */ - while (values.hasNext && !dropPartition) { - buffer += values.next() - count += 1 - if (count % memoryRequestPeriod == 1) { - // Calculate the amount of memory to request from the global memory pool - val currentSize = buffer.estimateSize() - val delta = math.max(currentSize - previousSize, 0) - val memoryToRequest = currentSize + delta - previousSize = currentSize - - // Atomically check whether there is sufficient memory in the global pool to continue - cacheMemoryMap.synchronized { - val previouslyOccupiedMemory = cacheMemoryMap.get(threadId).getOrElse(0L) - val otherThreadsMemory = cacheMemoryMap.values.sum - previouslyOccupiedMemory - - // Request for memory for the local buffer, and return whether request is granted - def requestForMemory(): Boolean = { - val availableMemory = blockManager.memoryStore.freeMemory - otherThreadsMemory - val granted = availableMemory > memoryToRequest - if (granted) { cacheMemoryMap(threadId) = memoryToRequest } - granted - } - - // If the first request is not granted, try again after ensuring free space - // If there is still not enough space, give up and drop the partition - if (!requestForMemory()) { - val result = blockManager.memoryStore.ensureFreeSpace(key, globalBufferMemory) - updatedBlocks ++= result.droppedBlocks - dropPartition = !requestForMemory() - } - } - } - } - - if (!dropPartition) { - // We have successfully unrolled the entire partition, so cache it in memory - updatedBlocks ++= blockManager.put(key, buffer.array, storageLevel, tellMaster = true) - buffer.iterator.asInstanceOf[Iterator[T]] - } else { - // We have exceeded our collective quota. This partition will not be cached in memory. + * when the entirety of the RDD does not fit in memory. + * + * In addition, we must be careful to not unfold the entire partition in memory at once. + * Otherwise, we may cause an OOM exception if the JVM does not have enough space for this + * single partition. Instead, we unfold the values cautiously, potentially aborting and + * dropping the partition to disk if applicable. + */ + blockManager.memoryStore.unfoldSafely(key, values, storageLevel, updatedBlocks) match { + case Left(arrayValues) => + // We have successfully unfolded the entire partition, so cache it in memory + updatedBlocks ++= blockManager.put(key, arrayValues, storageLevel, tellMaster = true) + arrayValues.iterator.asInstanceOf[Iterator[T]] + case Right(iteratorValues) => + // There is not enough space to cache this partition in memory + var returnValues = iteratorValues.asInstanceOf[Iterator[T]] val persistToDisk = storageLevel.useDisk - logWarning(s"Failed to cache $key in memory! There is not enough space to unroll the " + + logWarning(s"Failed to cache $key in memory! There is not enough space to unfold the " + s"entire partition. " + (if (persistToDisk) "Persisting to disk instead." else "")) - var newValues = (buffer.iterator ++ values).asInstanceOf[Iterator[T]] if (persistToDisk) { val newLevel = StorageLevel( storageLevel.useDisk, @@ -219,16 +173,9 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { storageLevel.useOffHeap, deserialized = false, storageLevel.replication) - newValues = putInBlockManager[T](key, newValues, newLevel, updatedBlocks) + returnValues = putInBlockManager[T](key, returnValues, newLevel, updatedBlocks) } - newValues - } - } finally { - // Free up buffer for other threads - buffer = null - cacheMemoryMap.synchronized { - cacheMemoryMap(threadId) = 0 - } + returnValues } } } diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index b4363b312bc64..2d0d5b76f0d23 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -22,7 +22,9 @@ import java.util.LinkedHashMap import scala.collection.mutable.ArrayBuffer +import org.apache.spark.SparkEnv import org.apache.spark.util.{SizeEstimator, Utils} +import org.apache.spark.util.collection.SizeTrackingAppendOnlyBuffer private case class MemoryEntry(value: Any, size: Long, deserialized: Boolean) @@ -34,11 +36,20 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) extends BlockStore(blockManager) { private val entries = new LinkedHashMap[BlockId, MemoryEntry](32, 0.75f, true) + @volatile private var currentMemory = 0L - // Object used to ensure that only one thread is putting blocks and if necessary, dropping - // blocks from the memory store. + + // Object used to ensure that only one thread is putting blocks and if necessary, + // dropping blocks from the memory store. private val putLock = new Object() + /** + * The amount of space ensured for unfolding values in memory, shared across all cores. + * This space is not reserved in advance, but allocated dynamically by dropping existing blocks. + * It must be a lazy val in order to access a mocked BlockManager's conf in tests properly. + */ + private lazy val globalBufferMemory = BlockManager.getBufferMemory(blockManager.conf) + logInfo("MemoryStore started with capacity %s".format(Utils.bytesToString(maxMemory))) def freeMemory: Long = maxMemory - currentMemory @@ -137,6 +148,87 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) logInfo("MemoryStore cleared") } + /** + * Unfold the given block in memory safely. + * + * The safety of this operation refers to avoiding potential OOM exceptions caused by + * unfolding the entirety of the block in memory at once. This is achieved by periodically + * checking whether the memory restrictions for unfolding blocks are still satisfied, + * stopping immediately if not. This check is a safeguard against the scenario in which + * there is not enough free memory to accommodate the entirety of a single block. + * + * This method returns either a fully unfolded array or a partially unfolded iterator. + */ + def unfoldSafely( + blockId: BlockId, + values: Iterator[Any], + storageLevel: StorageLevel, + droppedBlocks: ArrayBuffer[(BlockId, BlockStatus)]) + : Either[Array[Any], Iterator[Any]] = { + + var count = 0 // The number of elements unfolded so far + var enoughMemory = true // Whether there is enough memory to unfold this block + var previousSize = 0L // Previous estimate of the size of our buffer + val memoryRequestPeriod = 1000 // How frequently we request for more memory for our buffer + + val threadId = Thread.currentThread().getId + val cacheMemoryMap = SparkEnv.get.cacheMemoryMap + var buffer = new SizeTrackingAppendOnlyBuffer[Any] + + try { + while (values.hasNext && enoughMemory) { + buffer += values.next() + count += 1 + if (count % memoryRequestPeriod == 1) { + // Calculate the amount of memory to request from the global memory pool + val currentSize = buffer.estimateSize() + val delta = math.max(currentSize - previousSize, 0) + val memoryToRequest = currentSize + delta + previousSize = currentSize + + // Atomically check whether there is sufficient memory in the global pool to continue + cacheMemoryMap.synchronized { + val previouslyOccupiedMemory = cacheMemoryMap.get(threadId).getOrElse(0L) + val otherThreadsMemory = cacheMemoryMap.values.sum - previouslyOccupiedMemory + + // Request for memory for the local buffer, and return whether request is granted + def requestForMemory(): Boolean = { + val availableMemory = freeMemory - otherThreadsMemory + val granted = availableMemory > memoryToRequest + if (granted) { cacheMemoryMap(threadId) = memoryToRequest } + granted + } + + // If the first request is not granted, try again after ensuring free space + // If there is still not enough space, give up and drop the partition + if (!requestForMemory()) { + val result = ensureFreeSpace(blockId, globalBufferMemory) + droppedBlocks ++= result.droppedBlocks + enoughMemory = requestForMemory() + } + } + } + } + + if (enoughMemory) { + // We successfully unfolded the entirety of this block + Left(buffer.array) + } else { + // We ran out of space while unfolding the values for this block + Right(buffer.iterator ++ values) + } + + } finally { + // Unless we return an iterator that depends on the buffer, free up space for other threads + if (enoughMemory) { + buffer = null + cacheMemoryMap.synchronized { + cacheMemoryMap(threadId) = 0 + } + } + } + } + /** * Return the RDD ID that a given block ID is from, or None if it is not an RDD block. */ From 5961f50e8c118c13e58a8685e3606b6fb814813c Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 7 Jul 2014 16:52:34 -0700 Subject: [PATCH 13/55] Fix tests We no longer use ArrayBuffer in BlockManager, so the expected size estimation is off. Thanks @kayousterhout for catching this. --- .../org/apache/spark/storage/BlockManagerSuite.scala | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 23cb6905bfdeb..448920f2ae90a 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -421,13 +421,9 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr, mapOutputTracker) val list1 = List(new Array[Byte](200), new Array[Byte](200)) - val list1ForSizeEstimate = new ArrayBuffer[Any] - list1ForSizeEstimate ++= list1.iterator - val list1SizeEstimate = SizeEstimator.estimate(list1ForSizeEstimate) val list2 = List(new Array[Byte](50), new Array[Byte](100), new Array[Byte](150)) - val list2ForSizeEstimate = new ArrayBuffer[Any] - list2ForSizeEstimate ++= list2.iterator - val list2SizeEstimate = SizeEstimator.estimate(list2ForSizeEstimate) + val list1SizeEstimate = SizeEstimator.estimate(list1.iterator.toArray) + val list2SizeEstimate = SizeEstimator.estimate(list2.iterator.toArray) store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) store.put("list2memory", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) store.put("list2disk", list2.iterator, StorageLevel.DISK_ONLY, tellMaster = true) From d68f31e1352620c8c850e06d34642025ad1846a8 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 8 Jul 2014 12:13:30 -0700 Subject: [PATCH 14/55] Safely unfold blocks for all memory puts MemoryStore still has potential for OOM'ing. More specifically, we currently still unfold the entire block if we attempt to put (1) an iterator of values or (2) bytes in deserialized form directly in MemoryStore. As of this commit, all direct puts into MemoryStore go through the safe unfold code path. Note that tests are still failing, because our implementation of size estimation relies on SizeTrackingAppendOnlyBuffer, which adds some constant overhead in memory. As a result, for small blocks, we try to request more memory (more than double, in many cases) than we actually need. These will be fixed in the next commit. --- .../scala/org/apache/spark/CacheManager.scala | 12 +++--- .../apache/spark/storage/BlockManager.scala | 6 --- .../apache/spark/storage/MemoryStore.scala | 41 +++++++++++++++---- 3 files changed, 39 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 01c1b9f02898e..6ea0f1404e951 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -162,15 +162,15 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { arrayValues.iterator.asInstanceOf[Iterator[T]] case Right(iteratorValues) => // There is not enough space to cache this partition in memory + logWarning(s"Not enough space to cache $key in memory! " + + s"Free memory is ${blockManager.memoryStore.freeMemory}B.") var returnValues = iteratorValues.asInstanceOf[Iterator[T]] - val persistToDisk = storageLevel.useDisk - logWarning(s"Failed to cache $key in memory! There is not enough space to unfold the " + - s"entire partition. " + (if (persistToDisk) "Persisting to disk instead." else "")) - if (persistToDisk) { + if (storageLevel.useDisk) { + logWarning(s"Persisting $key to disk instead.") val newLevel = StorageLevel( - storageLevel.useDisk, + useDisk = true, useMemory = false, - storageLevel.useOffHeap, + useOffHeap = false, deserialized = false, storageLevel.replication) returnValues = putInBlockManager[T](key, returnValues, newLevel, updatedBlocks) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 09fc793a357a8..90cd2a68bd991 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -1071,12 +1071,6 @@ private[spark] object BlockManager extends Logging { (Runtime.getRuntime.maxMemory * memoryFraction * safetyFraction).toLong } - /** Return the amount of storage memory used for unrolling RDD partitions. */ - def getBufferMemory(conf: SparkConf): Long = { - val bufferFraction = conf.getDouble("spark.storage.bufferFraction", 0.2) - (getMaxMemory(conf) * bufferFraction).toLong - } - def getHeartBeatFrequency(conf: SparkConf): Long = conf.getLong("spark.storage.blockManagerTimeoutIntervalMs", 60000) / 4 diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 2d0d5b76f0d23..d8d08748107e9 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -35,6 +35,7 @@ private case class MemoryEntry(value: Any, size: Long, deserialized: Boolean) private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) extends BlockStore(blockManager) { + private val conf = blockManager.conf private val entries = new LinkedHashMap[BlockId, MemoryEntry](32, 0.75f, true) @volatile private var currentMemory = 0L @@ -46,9 +47,11 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) /** * The amount of space ensured for unfolding values in memory, shared across all cores. * This space is not reserved in advance, but allocated dynamically by dropping existing blocks. - * It must be a lazy val in order to access a mocked BlockManager's conf in tests properly. */ - private lazy val globalBufferMemory = BlockManager.getBufferMemory(blockManager.conf) + private val globalBufferMemory = { + val bufferFraction = conf.getDouble("spark.storage.bufferFraction", 0.2) + (maxMemory * bufferFraction).toLong + } logInfo("MemoryStore started with capacity %s".format(Utils.bytesToString(maxMemory))) @@ -66,10 +69,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) bytes.rewind() if (level.deserialized) { val values = blockManager.dataDeserialize(blockId, bytes) - val elements = values.toArray - val sizeEstimate = SizeEstimator.estimate(elements.asInstanceOf[AnyRef]) - val putAttempt = tryToPut(blockId, elements, sizeEstimate, deserialized = true) - PutResult(sizeEstimate, Left(values.toIterator), putAttempt.droppedBlocks) + putValues(blockId, values, level, returnValues = true) } else { val putAttempt = tryToPut(blockId, bytes, bytes.limit, deserialized = false) PutResult(bytes.limit(), Right(bytes.duplicate()), putAttempt.droppedBlocks) @@ -97,7 +97,32 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) values: Iterator[Any], level: StorageLevel, returnValues: Boolean): PutResult = { - putValues(blockId, values.toArray, level, returnValues) + val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] + val unfoldedValues = unfoldSafely(blockId, values, level, droppedBlocks) + unfoldedValues match { + case Left(arrayValues) => + // Values are fully unfolded in memory, so store them as an array + val result = putValues(blockId, arrayValues, level, returnValues) + droppedBlocks ++= result.droppedBlocks + PutResult(result.size, result.data, droppedBlocks) + case Right(iteratorValues) => + // Not enough space to unfold this block; drop to disk if applicable + logWarning(s"Not enough space to store $blockId in memory! Free memory is ${freeMemory}B.") + if (level.useDisk) { + logWarning(s"Persisting $blockId to disk instead.") + val newLevel = StorageLevel( + useDisk = true, + useMemory = false, + useOffHeap = false, + deserialized = false, + level.replication) + val result = blockManager.diskStore.putValues( + blockId, iteratorValues, newLevel, returnValues) + PutResult(result.size, result.data, droppedBlocks) + } else { + PutResult(0, null, droppedBlocks) + } + } } override def getBytes(blockId: BlockId): Option[ByteBuffer] = { @@ -182,7 +207,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) if (count % memoryRequestPeriod == 1) { // Calculate the amount of memory to request from the global memory pool val currentSize = buffer.estimateSize() - val delta = math.max(currentSize - previousSize, 0) + val delta = if (previousSize > 0) math.max(currentSize - previousSize, 0) else 0 val memoryToRequest = currentSize + delta previousSize = currentSize From 08715355c3005099e9f6e002ddb245b2d300f707 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 8 Jul 2014 13:47:13 -0700 Subject: [PATCH 15/55] Fix tests in BlockManagerSuite MemoryStore uses SizeTrackingAppendOnlyBuffer to safely unroll blocks. This buffer, however, adds a constant amount of overhead, which becomes quite significant for small blocks. In our tests, this overhead amounts to more than double the block size itself in many cases. The result is that we end up attempting to request more memory than the block needs. This causes the MemoryStore to decide there is not enough room to unfold the block even though there is in fact room for it. The fix scales everything up tenfold such that the constant overhead becomes less significant. In addition, we need to increase the buffer fraction to preserve the original LRU semantics of the tests. Note that, in general, we cannot liberally increase the buffer fraction, because we may end up dropping many old blocks just to cache a new small one. --- .../scala/org/apache/spark/SparkEnv.scala | 8 + .../apache/spark/storage/BlockManager.scala | 14 +- .../apache/spark/storage/MemoryStore.scala | 2 +- .../spark/storage/BlockManagerSuite.scala | 229 +++++++++--------- 4 files changed, 135 insertions(+), 118 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index eb97f1adbbd86..50f5ea39055a7 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -138,6 +138,14 @@ object SparkEnv extends Logging { env.get() } + /** + * Create an empty SparkEnv. For testing only. + */ + private[spark] def empty(conf: SparkConf): SparkEnv = { + new SparkEnv( + "test", null, null, null, null, null, null, null, null, null, null, null, null, null, conf) + } + private[spark] def create( conf: SparkConf, executorId: String, diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 90cd2a68bd991..f32500f0146cb 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -463,11 +463,15 @@ private[spark] class BlockManager( val values = dataDeserialize(blockId, bytes) if (level.deserialized) { // Cache the values before returning them - memoryStore.putValues(blockId, values, level, returnValues = true).data match { - case Left(values2) => - return Some(new BlockResult(values2, DataReadMethod.Disk, info.size)) - case _ => - throw new SparkException("Memory store did not return an iterator") + val putResult = memoryStore.putValues(blockId, values, level, returnValues = true) + putResult.data match { + case Left(it) => + return Some(new BlockResult(it, DataReadMethod.Disk, info.size)) + case Right(b) => + return Some(new BlockResult( + dataDeserialize(blockId, b), + DataReadMethod.Disk, + info.size)) } } else { return Some(new BlockResult(values, DataReadMethod.Disk, info.size)) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index d8d08748107e9..78d16a4863567 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -120,7 +120,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) blockId, iteratorValues, newLevel, returnValues) PutResult(result.size, result.data, droppedBlocks) } else { - PutResult(0, null, droppedBlocks) + PutResult(0, Left(iteratorValues), droppedBlocks) } } } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 448920f2ae90a..c0e8033713cdb 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -31,7 +31,7 @@ import org.scalatest.concurrent.Timeouts._ import org.scalatest.Matchers import org.scalatest.time.SpanSugar._ -import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf, SparkContext} +import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf, SparkEnv} import org.apache.spark.executor.DataReadMethod import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} @@ -43,7 +43,9 @@ import scala.language.postfixOps class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter with PrivateMethodTester { + private val conf = new SparkConf(false) + val env: SparkEnv = SparkEnv.empty(conf) var store: BlockManager = null var store2: BlockManager = null var actorSystem: ActorSystem = null @@ -62,20 +64,23 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter def rdd(rddId: Int, splitId: Int) = RDDBlockId(rddId, splitId) before { - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("test", "localhost", 0, conf = conf, - securityManager = securityMgr) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem( + "test", "localhost", 0, conf = conf, securityManager = securityMgr) this.actorSystem = actorSystem - conf.set("spark.driver.port", boundPort.toString) - - master = new BlockManagerMaster( - actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), - conf) // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case oldArch = System.setProperty("os.arch", "amd64") conf.set("os.arch", "amd64") conf.set("spark.test.useCompressedOops", "true") conf.set("spark.storage.disableBlockManagerHeartBeat", "true") + conf.set("spark.driver.port", boundPort.toString) + conf.set("spark.storage.bufferFraction", "0.4") + SparkEnv.set(env) + + master = new BlockManagerMaster( + actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), + conf) + val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() } @@ -447,11 +452,11 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("in-memory LRU storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = new BlockManager( + "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY) store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY) @@ -467,11 +472,11 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("in-memory LRU storage with serialization") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = new BlockManager( + "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_SER) store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY_SER) store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY_SER) @@ -487,11 +492,11 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("in-memory LRU for partitions of same RDD") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = new BlockManager( + "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle(rdd(0, 1), a1, StorageLevel.MEMORY_ONLY) store.putSingle(rdd(0, 2), a2, StorageLevel.MEMORY_ONLY) store.putSingle(rdd(0, 3), a3, StorageLevel.MEMORY_ONLY) @@ -507,11 +512,11 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("in-memory LRU for partitions of multiple RDDs") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - store.putSingle(rdd(0, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY) - store.putSingle(rdd(0, 2), new Array[Byte](400), StorageLevel.MEMORY_ONLY) - store.putSingle(rdd(1, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY) + store = new BlockManager( + "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + store.putSingle(rdd(0, 1), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) + store.putSingle(rdd(0, 2), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) + store.putSingle(rdd(1, 1), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) // At this point rdd_1_1 should've replaced rdd_0_1 assert(store.memoryStore.contains(rdd(1, 1)), "rdd_1_1 was not in store") assert(!store.memoryStore.contains(rdd(0, 1)), "rdd_0_1 was in store") @@ -519,8 +524,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter // Do a get() on rdd_0_2 so that it is the most recently used item assert(store.getSingle(rdd(0, 2)).isDefined, "rdd_0_2 was not in store") // Put in more partitions from RDD 0; they should replace rdd_1_1 - store.putSingle(rdd(0, 3), new Array[Byte](400), StorageLevel.MEMORY_ONLY) - store.putSingle(rdd(0, 4), new Array[Byte](400), StorageLevel.MEMORY_ONLY) + store.putSingle(rdd(0, 3), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) + store.putSingle(rdd(0, 4), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) // Now rdd_1_1 should be dropped to add rdd_0_3, but then rdd_0_2 should *not* be dropped // when we try to add rdd_0_4. assert(!store.memoryStore.contains(rdd(1, 1)), "rdd_1_1 was in store") @@ -565,11 +570,11 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("disk and memory storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + store = new BlockManager("", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK) store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK) store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK) @@ -581,11 +586,11 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("disk and memory storage with getLocalBytes") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = new BlockManager( + "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK) store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK) store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK) @@ -597,11 +602,11 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("disk and memory storage with serialization") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = new BlockManager( + "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK_SER) store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK_SER) store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK_SER) @@ -613,11 +618,11 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("disk and memory storage with serialization and getLocalBytes") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = new BlockManager( + "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK_SER) store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK_SER) store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK_SER) @@ -629,12 +634,12 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("LRU with mixed storage levels") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) - val a4 = new Array[Byte](400) + store = new BlockManager( + "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) + val a4 = new Array[Byte](4000) // First store a1 and a2, both in memory, and a3, on disk only store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_SER) store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY_SER) @@ -652,11 +657,11 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("in-memory LRU with streams") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val list1 = List(new Array[Byte](200), new Array[Byte](200)) - val list2 = List(new Array[Byte](200), new Array[Byte](200)) - val list3 = List(new Array[Byte](200), new Array[Byte](200)) + store = new BlockManager( + "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + val list1 = List(new Array[Byte](2000), new Array[Byte](2000)) + val list2 = List(new Array[Byte](2000), new Array[Byte](2000)) + val list3 = List(new Array[Byte](2000), new Array[Byte](2000)) store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) store.put("list2", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) store.put("list3", list3.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) @@ -677,12 +682,12 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("LRU with mixed storage levels and streams") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val list1 = List(new Array[Byte](200), new Array[Byte](200)) - val list2 = List(new Array[Byte](200), new Array[Byte](200)) - val list3 = List(new Array[Byte](200), new Array[Byte](200)) - val list4 = List(new Array[Byte](200), new Array[Byte](200)) + store = new BlockManager( + "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + val list1 = List(new Array[Byte](2000), new Array[Byte](2000)) + val list2 = List(new Array[Byte](2000), new Array[Byte](2000)) + val list3 = List(new Array[Byte](2000), new Array[Byte](2000)) + val list4 = List(new Array[Byte](2000), new Array[Byte](2000)) // First store list1 and list2, both in memory, and list3, on disk only store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) store.put("list2", list2.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) @@ -727,11 +732,11 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("overly large block") { - store = new BlockManager("", actorSystem, master, serializer, 500, conf, - securityMgr, mapOutputTracker) - store.putSingle("a1", new Array[Byte](1000), StorageLevel.MEMORY_ONLY) + store = new BlockManager( + "", actorSystem, master, serializer, 5000, conf, securityMgr, mapOutputTracker) + store.putSingle("a1", new Array[Byte](10000), StorageLevel.MEMORY_ONLY) assert(store.getSingle("a1") === None, "a1 was in store") - store.putSingle("a2", new Array[Byte](1000), StorageLevel.MEMORY_AND_DISK) + store.putSingle("a2", new Array[Byte](10000), StorageLevel.MEMORY_AND_DISK) assert(store.memoryStore.getValues("a2") === None, "a2 was in memory store") assert(store.getSingle("a2").isDefined, "a2 was not in store") } @@ -739,8 +744,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter test("block compression") { try { conf.set("spark.shuffle.compress", "true") - store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) + store = new BlockManager( + "exec1", actorSystem, master, serializer, 20000, conf, securityMgr, mapOutputTracker) store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) <= 100, "shuffle_0_0_0 was not compressed") @@ -748,52 +753,52 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter store = null conf.set("spark.shuffle.compress", "false") - store = new BlockManager("exec2", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) - assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) >= 1000, + store = new BlockManager( + "exec2", actorSystem, master, serializer, 20000, conf, securityMgr, mapOutputTracker) + store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) >= 10000, "shuffle_0_0_0 was compressed") store.stop() store = null conf.set("spark.broadcast.compress", "true") - store = new BlockManager("exec3", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) - assert(store.memoryStore.getSize(BroadcastBlockId(0)) <= 100, + store = new BlockManager( + "exec3", actorSystem, master, serializer, 20000, conf, securityMgr, mapOutputTracker) + store.putSingle(BroadcastBlockId(0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + assert(store.memoryStore.getSize(BroadcastBlockId(0)) <= 1000, "broadcast_0 was not compressed") store.stop() store = null conf.set("spark.broadcast.compress", "false") - store = new BlockManager("exec4", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) - assert(store.memoryStore.getSize(BroadcastBlockId(0)) >= 1000, "broadcast_0 was compressed") + store = new BlockManager( + "exec4", actorSystem, master, serializer, 20000, conf, securityMgr, mapOutputTracker) + store.putSingle(BroadcastBlockId(0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + assert(store.memoryStore.getSize(BroadcastBlockId(0)) >= 10000, "broadcast_0 was compressed") store.stop() store = null conf.set("spark.rdd.compress", "true") - store = new BlockManager("exec5", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) - assert(store.memoryStore.getSize(rdd(0, 0)) <= 100, "rdd_0_0 was not compressed") + store = new BlockManager( + "exec5", actorSystem, master, serializer, 20000, conf, securityMgr, mapOutputTracker) + store.putSingle(rdd(0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + assert(store.memoryStore.getSize(rdd(0, 0)) <= 1000, "rdd_0_0 was not compressed") store.stop() store = null conf.set("spark.rdd.compress", "false") - store = new BlockManager("exec6", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) - assert(store.memoryStore.getSize(rdd(0, 0)) >= 1000, "rdd_0_0 was compressed") + store = new BlockManager( + "exec6", actorSystem, master, serializer, 20000, conf, securityMgr, mapOutputTracker) + store.putSingle(rdd(0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + assert(store.memoryStore.getSize(rdd(0, 0)) >= 10000, "rdd_0_0 was compressed") store.stop() store = null // Check that any other block types are also kept uncompressed - store = new BlockManager("exec7", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store.putSingle("other_block", new Array[Byte](1000), StorageLevel.MEMORY_ONLY) - assert(store.memoryStore.getSize("other_block") >= 1000, "other_block was compressed") + store = new BlockManager( + "exec7", actorSystem, master, serializer, 20000, conf, securityMgr, mapOutputTracker) + store.putSingle("other_block", new Array[Byte](10000), StorageLevel.MEMORY_ONLY) + assert(store.memoryStore.getSize("other_block") >= 10000, "other_block was compressed") store.stop() store = null } finally { @@ -867,12 +872,12 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(Arrays.equals(mappedAsArray, bytes)) assert(Arrays.equals(notMappedAsArray, bytes)) } - + test("updated block statuses") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val list = List.fill(2)(new Array[Byte](200)) - val bigList = List.fill(8)(new Array[Byte](200)) + store = new BlockManager( + "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + val list = List.fill(2)(new Array[Byte](2000)) + val bigList = List.fill(8)(new Array[Byte](2000)) // 1 updated block (i.e. list1) val updatedBlocks1 = @@ -924,9 +929,9 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("query block statuses") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val list = List.fill(2)(new Array[Byte](200)) + store = new BlockManager( + "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + val list = List.fill(2)(new Array[Byte](2000)) // Tell master. By LRU, only list2 and list3 remains. store.put("list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) @@ -964,9 +969,9 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("get matching blocks") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val list = List.fill(2)(new Array[Byte](10)) + store = new BlockManager( + "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + val list = List.fill(2)(new Array[Byte](100)) // insert some blocks store.put("list1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) @@ -998,14 +1003,14 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("SPARK-1194 regression: fix the same-RDD rule for cache replacement") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - store.putSingle(rdd(0, 0), new Array[Byte](400), StorageLevel.MEMORY_ONLY) - store.putSingle(rdd(1, 0), new Array[Byte](400), StorageLevel.MEMORY_ONLY) + store = new BlockManager( + "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + store.putSingle(rdd(0, 0), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) + store.putSingle(rdd(1, 0), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) // Access rdd_1_0 to ensure it's not least recently used. assert(store.getSingle(rdd(1, 0)).isDefined, "rdd_1_0 was not in store") // According to the same-RDD rule, rdd_1_0 should be replaced here. - store.putSingle(rdd(0, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY) + store.putSingle(rdd(0, 1), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) // rdd_1_0 should have been replaced, even it's not least recently used. assert(store.memoryStore.contains(rdd(0, 0)), "rdd_0_0 was not in store") assert(store.memoryStore.contains(rdd(0, 1)), "rdd_0_1 was not in store") From 078eb83e0b31b0eb2dcd18f389794865f28ec8ea Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 8 Jul 2014 13:55:45 -0700 Subject: [PATCH 16/55] Add check for hasNext in PrimitiveVector.iterator --- .../org/apache/spark/util/collection/PrimitiveVector.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala index fff201f3b518e..7e76d060d6000 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala @@ -54,6 +54,9 @@ class PrimitiveVector[@specialized(Long, Int, Double) V: ClassTag](initialSize: var index = 0 override def hasNext: Boolean = index < _numElements override def next(): V = { + if (!hasNext) { + throw new NoSuchElementException + } val value = _array(index) index += 1 value From ec728d83cb288691fd4132057ba6f6cc38d9006b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 8 Jul 2014 17:59:30 -0700 Subject: [PATCH 17/55] Add tests for safe unfolding of blocks --- .../spark/storage/BlockManagerSuite.scala | 65 +++++++++++++++++++ 1 file changed, 65 insertions(+) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index c0e8033713cdb..10793439abf4a 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -1016,4 +1016,69 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(store.memoryStore.contains(rdd(0, 1)), "rdd_0_1 was not in store") assert(!store.memoryStore.contains(rdd(1, 0)), "rdd_1_0 was in store") } + + test("safely unfold blocks") { + conf.set("spark.storage.bufferFraction", "0.2") + store = new BlockManager( + "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + /* + * After a1 and a2 are cached, there is less than 4000 bytes of free space left. When we try + * to cache a3, we attempt to ensure free space of only 12000 * 0.2 = 2400 bytes, which is + * not enough to drop a1 or a2, as there is already >= 2400 bytes of free space. Therefore, + * we give up and drop a3 on the spot. + */ + store.putSingle("a1", new Array[Byte](4000), StorageLevel.MEMORY_ONLY) + store.putSingle("a2", new Array[Byte](4000), StorageLevel.MEMORY_ONLY) + store.putSingle("a3", new Array[Byte](4000), StorageLevel.MEMORY_ONLY) + // Memory store should contain a1, a2 + assert(store.memoryStore.contains("a1"), "a1 was not in store") + assert(store.memoryStore.contains("a2"), "a2 was not in store") + assert(!store.memoryStore.contains("a3"), "a3 was in store") + /* + * After a4 is cached, and there is less than 2000 bytes of free space left. Now we try + * to cache a5, which is the same size as a3. Ensuring free space of 2400 bytes now drops + * the LRU block (i.e. a1) from memory to accommodate a5. + */ + store.putSingle("a4", new Array[Byte](2000), StorageLevel.MEMORY_ONLY) + store.putSingle("a5", new Array[Byte](4000), StorageLevel.MEMORY_ONLY) + // Memory store should contain a2, a4, a5 + assert(!store.memoryStore.contains("a1"), "a1 was in store") + assert(store.memoryStore.contains("a2"), "a2 was not in store") + assert(!store.memoryStore.contains("a3"), "a3 was in store") + assert(store.memoryStore.contains("a4"), "a4 was not in store") + assert(store.memoryStore.contains("a5"), "a5 was not in store") + } + + test("safely unfold blocks (disk)") { + conf.set("spark.storage.bufferFraction", "0.2") + store = new BlockManager( + "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + /* + * This test is the same as the previous, except it caches each block using MEMORY_AND_DISK. + * The effect is that all dropped blocks now go to disk instead of simply disappear. + */ + store.putSingle("a1", new Array[Byte](4000), StorageLevel.MEMORY_AND_DISK) + store.putSingle("a2", new Array[Byte](4000), StorageLevel.MEMORY_AND_DISK) + store.putSingle("a3", new Array[Byte](4000), StorageLevel.MEMORY_AND_DISK) + // Memory store should contain a1, a2; disk store should contain a3 + assert(store.memoryStore.contains("a1"), "a1 was not in memory store") + assert(store.memoryStore.contains("a2"), "a2 was not in memory store") + assert(!store.memoryStore.contains("a3"), "a3 was in memory store") + assert(!store.diskStore.contains("a1"), "a1 was in disk store") + assert(!store.diskStore.contains("a2"), "a2 was in disk store") + assert(store.diskStore.contains("a3"), "a3 was not in disk store") + store.putSingle("a4", new Array[Byte](2000), StorageLevel.MEMORY_AND_DISK) + store.putSingle("a5", new Array[Byte](4000), StorageLevel.MEMORY_AND_DISK) + // Memory store should contain a2, a4, a5; disk store should contain a1, a3 + assert(!store.memoryStore.contains("a1"), "a1 was in memory store") + assert(store.memoryStore.contains("a2"), "a2 was not in memory store") + assert(!store.memoryStore.contains("a3"), "a3 was in memory store") + assert(store.memoryStore.contains("a4"), "a4 was not in memory store") + assert(store.memoryStore.contains("a5"), "a5 was not in memory store") + assert(store.diskStore.contains("a1"), "a1 was not in disk store") + assert(!store.diskStore.contains("a2"), "a2 was in disk store") + assert(store.diskStore.contains("a3"), "a3 was not in disk store") + assert(!store.diskStore.contains("a4"), "a4 was in disk store") + assert(!store.diskStore.contains("a5"), "a5 was in disk store") + } } From d9d02a846ef6bb67ba08e946dde280692d0ee2e9 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 8 Jul 2014 18:04:05 -0700 Subject: [PATCH 18/55] Remove unused param in unfoldSafely --- core/src/main/scala/org/apache/spark/CacheManager.scala | 2 +- core/src/main/scala/org/apache/spark/storage/MemoryStore.scala | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 6ea0f1404e951..c3a079d64db7a 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -155,7 +155,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { * single partition. Instead, we unfold the values cautiously, potentially aborting and * dropping the partition to disk if applicable. */ - blockManager.memoryStore.unfoldSafely(key, values, storageLevel, updatedBlocks) match { + blockManager.memoryStore.unfoldSafely(key, values, updatedBlocks) match { case Left(arrayValues) => // We have successfully unfolded the entire partition, so cache it in memory updatedBlocks ++= blockManager.put(key, arrayValues, storageLevel, tellMaster = true) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 78d16a4863567..c102f5b36c2ad 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -98,7 +98,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) level: StorageLevel, returnValues: Boolean): PutResult = { val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] - val unfoldedValues = unfoldSafely(blockId, values, level, droppedBlocks) + val unfoldedValues = unfoldSafely(blockId, values, droppedBlocks) unfoldedValues match { case Left(arrayValues) => // Values are fully unfolded in memory, so store them as an array @@ -187,7 +187,6 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) def unfoldSafely( blockId: BlockId, values: Iterator[Any], - storageLevel: StorageLevel, droppedBlocks: ArrayBuffer[(BlockId, BlockStatus)]) : Either[Array[Any], Iterator[Any]] = { From 198e3743eb5f132611613979924503faa2b5390a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 17 Jul 2014 16:33:54 -0700 Subject: [PATCH 19/55] Unfold -> unroll --- .../scala/org/apache/spark/CacheManager.scala | 10 +++---- .../apache/spark/storage/MemoryStore.scala | 29 ++++++++++--------- .../spark/storage/BlockManagerSuite.scala | 4 +-- 3 files changed, 22 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index c3a079d64db7a..07b32680a6c8a 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -132,7 +132,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { if (!storageLevel.useMemory) { /* * This RDD is not to be cached in memory, so we can just pass the computed values - * as an iterator directly to the BlockManager, rather than first fully unfolding + * as an iterator directly to the BlockManager, rather than first fully unrolling * it in memory. The latter option potentially uses much more memory and risks OOM * exceptions that can be avoided. */ @@ -150,14 +150,14 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { * we may end up dropping a partition from memory store before getting it back, e.g. * when the entirety of the RDD does not fit in memory. * - * In addition, we must be careful to not unfold the entire partition in memory at once. + * In addition, we must be careful to not unroll the entire partition in memory at once. * Otherwise, we may cause an OOM exception if the JVM does not have enough space for this - * single partition. Instead, we unfold the values cautiously, potentially aborting and + * single partition. Instead, we unroll the values cautiously, potentially aborting and * dropping the partition to disk if applicable. */ - blockManager.memoryStore.unfoldSafely(key, values, updatedBlocks) match { + blockManager.memoryStore.unrollSafely(key, values, updatedBlocks) match { case Left(arrayValues) => - // We have successfully unfolded the entire partition, so cache it in memory + // We have successfully unrolled the entire partition, so cache it in memory updatedBlocks ++= blockManager.put(key, arrayValues, storageLevel, tellMaster = true) arrayValues.iterator.asInstanceOf[Iterator[T]] case Right(iteratorValues) => diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index c102f5b36c2ad..2b8fb9c7d3137 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -45,7 +45,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) private val putLock = new Object() /** - * The amount of space ensured for unfolding values in memory, shared across all cores. + * The amount of space ensured for unrolling values in memory, shared across all cores. * This space is not reserved in advance, but allocated dynamically by dropping existing blocks. */ private val globalBufferMemory = { @@ -98,15 +98,15 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) level: StorageLevel, returnValues: Boolean): PutResult = { val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] - val unfoldedValues = unfoldSafely(blockId, values, droppedBlocks) - unfoldedValues match { + val unrolledValues = unrollSafely(blockId, values, droppedBlocks) + unrolledValues match { case Left(arrayValues) => - // Values are fully unfolded in memory, so store them as an array + // Values are fully unrolled in memory, so store them as an array val result = putValues(blockId, arrayValues, level, returnValues) droppedBlocks ++= result.droppedBlocks PutResult(result.size, result.data, droppedBlocks) case Right(iteratorValues) => - // Not enough space to unfold this block; drop to disk if applicable + // Not enough space to unroll this block; drop to disk if applicable logWarning(s"Not enough space to store $blockId in memory! Free memory is ${freeMemory}B.") if (level.useDisk) { logWarning(s"Persisting $blockId to disk instead.") @@ -174,24 +174,25 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) } /** - * Unfold the given block in memory safely. + * Unroll the given block in memory safely. * * The safety of this operation refers to avoiding potential OOM exceptions caused by - * unfolding the entirety of the block in memory at once. This is achieved by periodically - * checking whether the memory restrictions for unfolding blocks are still satisfied, + * unrolling the entirety of the block in memory at once. This is achieved by periodically + * checking whether the memory restrictions for unrolling blocks are still satisfied, * stopping immediately if not. This check is a safeguard against the scenario in which * there is not enough free memory to accommodate the entirety of a single block. * - * This method returns either a fully unfolded array or a partially unfolded iterator. + * This method returns either an array with the contents of the entire block or an iterator + * containing the values of the block (if the array would have exceeded available memory). */ - def unfoldSafely( + def unrollSafely( blockId: BlockId, values: Iterator[Any], droppedBlocks: ArrayBuffer[(BlockId, BlockStatus)]) : Either[Array[Any], Iterator[Any]] = { - var count = 0 // The number of elements unfolded so far - var enoughMemory = true // Whether there is enough memory to unfold this block + var count = 0 // The number of elements unrolled so far + var enoughMemory = true // Whether there is enough memory to unroll this block var previousSize = 0L // Previous estimate of the size of our buffer val memoryRequestPeriod = 1000 // How frequently we request for more memory for our buffer @@ -235,10 +236,10 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) } if (enoughMemory) { - // We successfully unfolded the entirety of this block + // We successfully unrolled the entirety of this block Left(buffer.array) } else { - // We ran out of space while unfolding the values for this block + // We ran out of space while unrolling the values for this block Right(buffer.iterator ++ values) } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 10793439abf4a..45324d1d4be3f 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -1017,7 +1017,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(!store.memoryStore.contains(rdd(1, 0)), "rdd_1_0 was in store") } - test("safely unfold blocks") { + test("safely unroll blocks") { conf.set("spark.storage.bufferFraction", "0.2") store = new BlockManager( "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) @@ -1049,7 +1049,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(store.memoryStore.contains("a5"), "a5 was not in store") } - test("safely unfold blocks (disk)") { + test("safely unroll blocks (disk)") { conf.set("spark.storage.bufferFraction", "0.2") store = new BlockManager( "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) From e9c3cb03f54f4760f2f8635feef4852a92008842 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 17 Jul 2014 17:09:34 -0700 Subject: [PATCH 20/55] cacheMemoryMap -> unrollMemoryMap --- .../main/scala/org/apache/spark/CacheManager.scala | 2 -- .../src/main/scala/org/apache/spark/SparkEnv.scala | 6 +++--- .../scala/org/apache/spark/executor/Executor.scala | 4 ++-- .../org/apache/spark/storage/MemoryStore.scala | 14 +++++++------- 4 files changed, 12 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 07b32680a6c8a..4e9999ad118a3 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -20,10 +20,8 @@ package org.apache.spark import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.apache.spark.executor.InputMetrics import org.apache.spark.rdd.RDD import org.apache.spark.storage._ -import org.apache.spark.util.collection.SizeTrackingAppendOnlyBuffer /** * Spark class responsible for passing RDDs partition contents to the BlockManager and making diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 8923d44e3b439..f4f26cadcc738 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -67,13 +67,13 @@ class SparkEnv ( val metricsSystem: MetricsSystem, val conf: SparkConf) extends Logging { - // A mapping of thread ID to amount of memory, in bytes, used for shuffle + // A mapping of thread ID to amount of memory, in bytes, used for shuffle aggregations // All accesses should be manually synchronized val shuffleMemoryMap = mutable.HashMap[Long, Long]() - // A mapping of thread ID to amount of memory, in bytes, used for unrolling an RDD partition + // A mapping of thread ID to amount of memory, in bytes, used for unrolling a block // All accesses should be manually synchronized - val cacheMemoryMap = mutable.HashMap[Long, Long]() + val unrollMemoryMap = mutable.HashMap[Long, Long]() private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 7bf8084b8b0e3..5e748c100d680 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -268,9 +268,9 @@ private[spark] class Executor( } finally { val threadId = Thread.currentThread().getId val shuffleMemoryMap = env.shuffleMemoryMap - val cacheMemoryMap = env.cacheMemoryMap + val unrollMemoryMap = env.unrollMemoryMap shuffleMemoryMap.synchronized { shuffleMemoryMap.remove(threadId) } - cacheMemoryMap.synchronized { cacheMemoryMap.remove(threadId) } + unrollMemoryMap.synchronized { unrollMemoryMap.remove(threadId) } runningTasks.remove(taskId) } } diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 2b8fb9c7d3137..08782ca0835e0 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -197,7 +197,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) val memoryRequestPeriod = 1000 // How frequently we request for more memory for our buffer val threadId = Thread.currentThread().getId - val cacheMemoryMap = SparkEnv.get.cacheMemoryMap + val unrollMemoryMap = SparkEnv.get.unrollMemoryMap var buffer = new SizeTrackingAppendOnlyBuffer[Any] try { @@ -212,15 +212,15 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) previousSize = currentSize // Atomically check whether there is sufficient memory in the global pool to continue - cacheMemoryMap.synchronized { - val previouslyOccupiedMemory = cacheMemoryMap.get(threadId).getOrElse(0L) - val otherThreadsMemory = cacheMemoryMap.values.sum - previouslyOccupiedMemory + unrollMemoryMap.synchronized { + val previouslyOccupiedMemory = unrollMemoryMap.get(threadId).getOrElse(0L) + val otherThreadsMemory = unrollMemoryMap.values.sum - previouslyOccupiedMemory // Request for memory for the local buffer, and return whether request is granted def requestForMemory(): Boolean = { val availableMemory = freeMemory - otherThreadsMemory val granted = availableMemory > memoryToRequest - if (granted) { cacheMemoryMap(threadId) = memoryToRequest } + if (granted) { unrollMemoryMap(threadId) = memoryToRequest } granted } @@ -247,8 +247,8 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) // Unless we return an iterator that depends on the buffer, free up space for other threads if (enoughMemory) { buffer = null - cacheMemoryMap.synchronized { - cacheMemoryMap(threadId) = 0 + unrollMemoryMap.synchronized { + unrollMemoryMap(threadId) = 0 } } } From a10b0e72a4941ae8ca19d8f528b40ba25a129c6d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 17 Jul 2014 17:21:32 -0700 Subject: [PATCH 21/55] Add initial memory request threshold + rename a few things --- .../apache/spark/storage/MemoryStore.scala | 25 ++++++++++--------- 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 08782ca0835e0..6791369795954 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -191,20 +191,21 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) droppedBlocks: ArrayBuffer[(BlockId, BlockStatus)]) : Either[Array[Any], Iterator[Any]] = { - var count = 0 // The number of elements unrolled so far - var enoughMemory = true // Whether there is enough memory to unroll this block - var previousSize = 0L // Previous estimate of the size of our buffer - val memoryRequestPeriod = 1000 // How frequently we request for more memory for our buffer + var count = 0 // The number of elements unrolled so far + var atMemoryLimit = false // Whether we are at the memory limit for unrolling blocks + var previousSize = 0L // Previous estimate of the size of our buffer + val memoryRequestPeriod = 1000 // How frequently we request more memory for our buffer + val memoryRequestThreshold = 100 // Before this is exceeded, request memory every round val threadId = Thread.currentThread().getId val unrollMemoryMap = SparkEnv.get.unrollMemoryMap var buffer = new SizeTrackingAppendOnlyBuffer[Any] try { - while (values.hasNext && enoughMemory) { + while (values.hasNext && !atMemoryLimit) { buffer += values.next() count += 1 - if (count % memoryRequestPeriod == 1) { + if (count % memoryRequestPeriod == 1 || count < memoryRequestThreshold) { // Calculate the amount of memory to request from the global memory pool val currentSize = buffer.estimateSize() val delta = if (previousSize > 0) math.max(currentSize - previousSize, 0) else 0 @@ -216,8 +217,8 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) val previouslyOccupiedMemory = unrollMemoryMap.get(threadId).getOrElse(0L) val otherThreadsMemory = unrollMemoryMap.values.sum - previouslyOccupiedMemory - // Request for memory for the local buffer, and return whether request is granted - def requestForMemory(): Boolean = { + // Request memory for the local buffer and return whether request is granted + def requestMemory(): Boolean = { val availableMemory = freeMemory - otherThreadsMemory val granted = availableMemory > memoryToRequest if (granted) { unrollMemoryMap(threadId) = memoryToRequest } @@ -226,16 +227,16 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) // If the first request is not granted, try again after ensuring free space // If there is still not enough space, give up and drop the partition - if (!requestForMemory()) { + if (!requestMemory()) { val result = ensureFreeSpace(blockId, globalBufferMemory) droppedBlocks ++= result.droppedBlocks - enoughMemory = requestForMemory() + atMemoryLimit = !requestMemory() } } } } - if (enoughMemory) { + if (!atMemoryLimit) { // We successfully unrolled the entirety of this block Left(buffer.array) } else { @@ -245,7 +246,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) } finally { // Unless we return an iterator that depends on the buffer, free up space for other threads - if (enoughMemory) { + if (!atMemoryLimit) { buffer = null unrollMemoryMap.synchronized { unrollMemoryMap(threadId) = 0 From 649bdb36434fea04c080b2fe652d4e82c13d876b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 17 Jul 2014 17:27:21 -0700 Subject: [PATCH 22/55] Document spark.storage.bufferFraction --- docs/configuration.md | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/docs/configuration.md b/docs/configuration.md index a70007c165442..8b2d9f44b8c6b 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -458,6 +458,15 @@ Apart from these, the following properties are also available, and may be useful increase it if you configure your own old generation size. + + spark.storage.bufferFraction + 0.2 + + Fraction of spark.storage.memoryFraction to use for unrolling blocks in memory. + This is dynamically allocated by dropping existing blocks when there is not enough free + storage space to unroll the new block in its entirety. + + spark.tachyonStore.baseDir System.getProperty("java.io.tmpdir") From 9b9a2737de20c27ee5cc8fbd2a87478df3255965 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 18 Jul 2014 11:41:01 -0700 Subject: [PATCH 23/55] Fix tests --- .../apache/spark/storage/BlockManagerSuite.scala | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 45324d1d4be3f..a0664f3336472 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -423,10 +423,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("correct BlockResult returned from get() calls") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr, + store = new BlockManager("", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) - val list1 = List(new Array[Byte](200), new Array[Byte](200)) - val list2 = List(new Array[Byte](50), new Array[Byte](100), new Array[Byte](150)) + val list1 = List(new Array[Byte](2000), new Array[Byte](2000)) + val list2 = List(new Array[Byte](500), new Array[Byte](1000), new Array[Byte](1500)) val list1SizeEstimate = SizeEstimator.estimate(list1.iterator.toArray) val list2SizeEstimate = SizeEstimator.estimate(list2.iterator.toArray) store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) @@ -919,11 +919,14 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } assert(store.get("list4").isDefined, "list4 was not in store") - // No updated blocks - nothing is kicked out of memory because list5 is too big to be added + // 1 updated block - list5 is too big to fit in store, but list3 is kicked out in the process val updatedBlocks5 = store.put("list5", bigList.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - assert(updatedBlocks5.size === 0) + assert(updatedBlocks5.size === 1) + assert(updatedBlocks5.head._1 === TestBlockId("list3")) + assert(updatedBlocks5.head._2.storageLevel === StorageLevel.NONE) assert(store.get("list2").isDefined, "list2 was not in store") + assert(!store.get("list3").isDefined, "list3 was in store") assert(store.get("list4").isDefined, "list4 was not in store") assert(!store.get("list5").isDefined, "list5 was in store") } From 2b7ee66c8a2713faff5a86a0d03a5ccb7466c4b8 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 18 Jul 2014 11:44:44 -0700 Subject: [PATCH 24/55] Fix bug in SizeTracking* We were not incorporating the delta in sizes, because Seq does not match the case a :: b :: c, while List does. This was originally reported by @mateiz. --- .../scala/org/apache/spark/util/collection/SizeTracker.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTracker.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTracker.scala index 70b2e63bf1df3..fd71c51886c45 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/SizeTracker.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTracker.scala @@ -80,7 +80,7 @@ private[spark] trait SizeTracker { if (samples.size > 2) { samples.remove(0) } - val bytesDelta = samples.toSeq.reverse match { + val bytesDelta = samples.toList.reverse match { case latest :: previous :: tail => (latest.size - previous.size).toDouble / (latest.numUpdates - previous.numUpdates) // If fewer than 2 samples, assume no change From 24185ea689933546dc24f37eeeda99b74d310ec8 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 18 Jul 2014 12:17:30 -0700 Subject: [PATCH 25/55] Avoid dropping a block back to disk if reading from disk When the BlockManager reads a block from disk, it attempts to cache it in memory. However, when we put the values in memory, it writes them back out to disk, which is unnecessary and has confusing semantics. Instead, we create a flag "allowPersistToDisk" for memory store's putValues to identify cases we don't want to drop the block to disk even if it doesn't fit in memory. --- .../apache/spark/storage/BlockManager.scala | 10 ++++---- .../apache/spark/storage/MemoryStore.scala | 23 ++++++++++++++++++- 2 files changed, 26 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index f32500f0146cb..a2b9c9485cf70 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -463,15 +463,13 @@ private[spark] class BlockManager( val values = dataDeserialize(blockId, bytes) if (level.deserialized) { // Cache the values before returning them - val putResult = memoryStore.putValues(blockId, values, level, returnValues = true) + val putResult = memoryStore.putValues( + blockId, values, level, returnValues = true, allowPersistToDisk = false) putResult.data match { case Left(it) => return Some(new BlockResult(it, DataReadMethod.Disk, info.size)) - case Right(b) => - return Some(new BlockResult( - dataDeserialize(blockId, b), - DataReadMethod.Disk, - info.size)) + case _ => + throw new SparkException("Memory store did not return an iterator!") } } else { return Some(new BlockResult(values, DataReadMethod.Disk, info.size)) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 6791369795954..3a093e41d576f 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -97,6 +97,27 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) values: Iterator[Any], level: StorageLevel, returnValues: Boolean): PutResult = { + putValues(blockId, values, level, returnValues, allowPersistToDisk = true) + } + + /** + * Attempt to put the given block in memory store. + * + * There may not be enough space to fully unroll the iterator in memory, in which case we + * optionally drop the values to disk if + * (1) the block's storage level specifies useDisk, and + * (2) `allowPersistToDisk` is true. + * + * One scenario in which `allowPersistToDisk` is false is when the BlockManager reads a block + * back from disk and attempts to cache it in memory. In this case, we should not persist the + * block back on disk again, as it is already in disk store. + */ + private[storage] def putValues( + blockId: BlockId, + values: Iterator[Any], + level: StorageLevel, + returnValues: Boolean, + allowPersistToDisk: Boolean): PutResult = { val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] val unrolledValues = unrollSafely(blockId, values, droppedBlocks) unrolledValues match { @@ -108,7 +129,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) case Right(iteratorValues) => // Not enough space to unroll this block; drop to disk if applicable logWarning(s"Not enough space to store $blockId in memory! Free memory is ${freeMemory}B.") - if (level.useDisk) { + if (level.useDisk && allowPersistToDisk) { logWarning(s"Persisting $blockId to disk instead.") val newLevel = StorageLevel( useDisk = true, From ecc8c2d0ef5a9be0660778bcb63c92dd05641b85 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 18 Jul 2014 14:02:20 -0700 Subject: [PATCH 26/55] Fix binary incompatibility --- project/MimaExcludes.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index a64f34d56d5cc..6600d52b6df68 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -61,8 +61,13 @@ object MimaExcludes { ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.storage.MemoryStore.Entry"), ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.rdd.PairRDDFunctions.org$apache$spark$rdd$PairRDDFunctions$$" - + "createZero$1") + "org.apache.spark.rdd.PairRDDFunctions.org$apache$spark$rdd$PairRDDFunctions$$createZero$1"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.storage.MemoryStore.putValues"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.storage.DiskStore.putValues"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.storage.TachyonStore.putValues") ) ++ Seq( ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.FlumeReceiver.this") From 4f4834ed03019c8fab3fb3e4ccde2d53c6c4eb61 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 18 Jul 2014 15:01:37 -0700 Subject: [PATCH 27/55] Use original storage level for blocks dropped to disk ... instead of replacing its original storage level with DISK_ONLY. Otherwise, when there is more free memory later, we will not cache the block in memory when reading back from disk, because the level does not specify cache in memory. This is the case even if the original storage level is MEMORY_AND_DISK. This commit makes sure we always put a block in BlockManager with its original storage level intact. --- .../scala/org/apache/spark/CacheManager.scala | 10 ++++------ .../org/apache/spark/storage/BlockInfo.scala | 16 +++++++++++++++- .../org/apache/spark/storage/BlockManager.scala | 12 ++++++++++++ .../org/apache/spark/storage/MemoryStore.scala | 17 +++++------------ 4 files changed, 36 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 4e9999ad118a3..585074fe08e77 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -165,13 +165,11 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { var returnValues = iteratorValues.asInstanceOf[Iterator[T]] if (storageLevel.useDisk) { logWarning(s"Persisting $key to disk instead.") - val newLevel = StorageLevel( - useDisk = true, - useMemory = false, - useOffHeap = false, - deserialized = false, - storageLevel.replication) + val newLevel = StorageLevel(useDisk = true, useMemory = false, + useOffHeap = false, deserialized = false, storageLevel.replication) // DISK_ONLY returnValues = putInBlockManager[T](key, returnValues, newLevel, updatedBlocks) + // Restore original storage level + blockManager.updateStorageLevel(key, storageLevel) } returnValues } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala index 22fdf73e9d1f4..846ce459cae16 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala @@ -19,13 +19,15 @@ package org.apache.spark.storage import java.util.concurrent.ConcurrentHashMap -private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) { +private[storage] class BlockInfo(private var _level: StorageLevel, val tellMaster: Boolean) { // To save space, 'pending' and 'failed' are encoded as special sizes: @volatile var size: Long = BlockInfo.BLOCK_PENDING private def pending: Boolean = size == BlockInfo.BLOCK_PENDING private def failed: Boolean = size == BlockInfo.BLOCK_FAILED private def initThread: Thread = BlockInfo.blockInfoInitThreads.get(this) + def level: StorageLevel = _level + setInitThread() private def setInitThread() { @@ -70,6 +72,18 @@ private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolea this.notifyAll() } } + + /** + * Update the storage level. + * + * This is currently only used for forcing blocks with storage level MEMORY_AND_DISK to disk. + * BlockManager does not provide an interface to do this, and so we workaround this by passing + * in a mock DISK_ONLY level and manually restoring the original level afterwards. + */ + def updateStorageLevel(newLevel: StorageLevel) = { + _level = newLevel + } + } private object BlockInfo { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index a2b9c9485cf70..db55e864836ff 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -241,6 +241,18 @@ private[spark] class BlockManager( } } + /** + * Update the storage level of an existing block. This is currently only used for forcing + * MEMORY_AND_DISK blocks to disk, and restoring the original storage level afterwards. + */ + def updateStorageLevel(blockId: BlockId, level: StorageLevel) = { + if (blockInfo.contains(blockId)) { + blockInfo(blockId).updateStorageLevel(level) + } else { + logWarning(s"Block $blockId not found when attempting to update its storage level!") + } + } + /** * Get the ids of existing blocks that match the given filter. Note that this will * query the blocks stored in the disk block manager (that the block manager diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 3a093e41d576f..5744481237f71 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -123,23 +123,16 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) unrolledValues match { case Left(arrayValues) => // Values are fully unrolled in memory, so store them as an array - val result = putValues(blockId, arrayValues, level, returnValues) - droppedBlocks ++= result.droppedBlocks - PutResult(result.size, result.data, droppedBlocks) + val res = putValues(blockId, arrayValues, level, returnValues) + droppedBlocks ++= res.droppedBlocks + PutResult(res.size, res.data, droppedBlocks) case Right(iteratorValues) => // Not enough space to unroll this block; drop to disk if applicable logWarning(s"Not enough space to store $blockId in memory! Free memory is ${freeMemory}B.") if (level.useDisk && allowPersistToDisk) { logWarning(s"Persisting $blockId to disk instead.") - val newLevel = StorageLevel( - useDisk = true, - useMemory = false, - useOffHeap = false, - deserialized = false, - level.replication) - val result = blockManager.diskStore.putValues( - blockId, iteratorValues, newLevel, returnValues) - PutResult(result.size, result.data, droppedBlocks) + val res = blockManager.diskStore.putValues(blockId, iteratorValues, level, returnValues) + PutResult(res.size, res.data, droppedBlocks) } else { PutResult(0, Left(iteratorValues), droppedBlocks) } From 64e7d4c5c62d04ae3fdc325ea2d6031ac600afb5 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 18 Jul 2014 17:44:46 -0700 Subject: [PATCH 28/55] Add/modify a few comments (minor) --- core/src/main/scala/org/apache/spark/CacheManager.scala | 3 +-- .../main/scala/org/apache/spark/storage/BlockManager.scala | 3 ++- .../scala/org/apache/spark/storage/BlockManagerSuite.scala | 6 +++--- project/MimaExcludes.scala | 1 + 4 files changed, 7 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 585074fe08e77..8f72c128e76e2 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -145,8 +145,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { /* * This RDD is to be cached in memory. In this case we cannot pass the computed values * to the BlockManager as an iterator and expect to read it back later. This is because - * we may end up dropping a partition from memory store before getting it back, e.g. - * when the entirety of the RDD does not fit in memory. + * we may end up dropping a partition from memory store before getting it back. * * In addition, we must be careful to not unroll the entire partition in memory at once. * Otherwise, we may cause an OOM exception if the JVM does not have enough space for this diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index db55e864836ff..a8d33b54e2a6f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -243,7 +243,7 @@ private[spark] class BlockManager( /** * Update the storage level of an existing block. This is currently only used for forcing - * MEMORY_AND_DISK blocks to disk, and restoring the original storage level afterwards. + * MEMORY_AND_DISK blocks to disk and restoring the original storage level afterwards. */ def updateStorageLevel(blockId: BlockId, level: StorageLevel) = { if (blockInfo.contains(blockId)) { @@ -481,6 +481,7 @@ private[spark] class BlockManager( case Left(it) => return Some(new BlockResult(it, DataReadMethod.Disk, info.size)) case _ => + // This only happens if we dropped the values back to disk (which is never) throw new SparkException("Memory store did not return an iterator!") } } else { diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index a0664f3336472..ba35add7dc97c 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -1038,9 +1038,9 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(store.memoryStore.contains("a2"), "a2 was not in store") assert(!store.memoryStore.contains("a3"), "a3 was in store") /* - * After a4 is cached, and there is less than 2000 bytes of free space left. Now we try - * to cache a5, which is the same size as a3. Ensuring free space of 2400 bytes now drops - * the LRU block (i.e. a1) from memory to accommodate a5. + * After a4 is cached, there is less than 2000 bytes of free space left. Now we try to cache + * a5, which is the same size as a3. Ensuring free space of 2400 bytes now drops the LRU block + * (i.e. a1) from memory to accommodate a5 (because 2000 < 2400). */ store.putSingle("a4", new Array[Byte](2000), StorageLevel.MEMORY_ONLY) store.putSingle("a5", new Array[Byte](4000), StorageLevel.MEMORY_ONLY) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 77e39e761c3b8..12e5012ed2f75 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -62,6 +62,7 @@ object MimaExcludes { "org.apache.spark.storage.MemoryStore.Entry"), ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.rdd.PairRDDFunctions.org$apache$spark$rdd$PairRDDFunctions$$createZero$1"), + // BlockManager now uses Arrays instead of ArrayBuffers ProblemFilters.exclude[IncompatibleMethTypeProblem]( "org.apache.spark.storage.MemoryStore.putValues"), ProblemFilters.exclude[IncompatibleMethTypeProblem]( From 087183536a60c202009bb71eebea4ba8e754a9e2 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sat, 19 Jul 2014 11:15:39 -0700 Subject: [PATCH 29/55] Add an effective storage level interface to BlockManager There is currently no mechanism to specify a different put behavior in BlockManager other than using a mock storage level and restoring the original one specified by the user later. The refactor in this commit allows BlockManager to take in both the original level and an optional effective level. The latter is the level BlockManager will actually use to handle the block. --- .../scala/org/apache/spark/CacheManager.scala | 35 ++++---- .../org/apache/spark/storage/BlockInfo.scala | 16 +--- .../apache/spark/storage/BlockManager.scala | 81 ++++++++++--------- .../apache/spark/storage/ThreadingTest.scala | 2 +- .../spark/storage/BlockManagerSuite.scala | 60 +++++++------- .../receiver/ReceiverSupervisorImpl.scala | 4 +- 6 files changed, 97 insertions(+), 101 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 8f72c128e76e2..e4c027c8532eb 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -124,17 +124,20 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { private def putInBlockManager[T]( key: BlockId, values: Iterator[T], - storageLevel: StorageLevel, - updatedBlocks: ArrayBuffer[(BlockId, BlockStatus)]): Iterator[T] = { + level: StorageLevel, + updatedBlocks: ArrayBuffer[(BlockId, BlockStatus)], + effectiveStorageLevel: Option[StorageLevel] = None): Iterator[T] = { - if (!storageLevel.useMemory) { + val putLevel = effectiveStorageLevel.getOrElse(level) + if (!putLevel.useMemory) { /* * This RDD is not to be cached in memory, so we can just pass the computed values * as an iterator directly to the BlockManager, rather than first fully unrolling * it in memory. The latter option potentially uses much more memory and risks OOM * exceptions that can be avoided. */ - updatedBlocks ++= blockManager.put(key, values, storageLevel, tellMaster = true) + updatedBlocks ++= + blockManager.putIterator(key, values, level, tellMaster = true, effectiveStorageLevel) blockManager.get(key) match { case Some(v) => v.data.asInstanceOf[Iterator[T]] case None => @@ -153,22 +156,22 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { * dropping the partition to disk if applicable. */ blockManager.memoryStore.unrollSafely(key, values, updatedBlocks) match { - case Left(arrayValues) => + case Left(arr) => // We have successfully unrolled the entire partition, so cache it in memory - updatedBlocks ++= blockManager.put(key, arrayValues, storageLevel, tellMaster = true) - arrayValues.iterator.asInstanceOf[Iterator[T]] - case Right(iteratorValues) => + updatedBlocks ++= + blockManager.putArray(key, arr, level, tellMaster = true, effectiveStorageLevel) + arr.iterator.asInstanceOf[Iterator[T]] + case Right(it) => // There is not enough space to cache this partition in memory logWarning(s"Not enough space to cache $key in memory! " + - s"Free memory is ${blockManager.memoryStore.freeMemory}B.") - var returnValues = iteratorValues.asInstanceOf[Iterator[T]] - if (storageLevel.useDisk) { + s"Free memory is ${blockManager.memoryStore.freeMemory} bytes.") + var returnValues = it.asInstanceOf[Iterator[T]] + if (putLevel.useDisk) { logWarning(s"Persisting $key to disk instead.") - val newLevel = StorageLevel(useDisk = true, useMemory = false, - useOffHeap = false, deserialized = false, storageLevel.replication) // DISK_ONLY - returnValues = putInBlockManager[T](key, returnValues, newLevel, updatedBlocks) - // Restore original storage level - blockManager.updateStorageLevel(key, storageLevel) + val diskOnlyLevel = StorageLevel(useDisk = true, useMemory = false, + useOffHeap = false, deserialized = false, putLevel.replication) + returnValues = + putInBlockManager[T](key, returnValues, level, updatedBlocks, Some(diskOnlyLevel)) } returnValues } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala index 846ce459cae16..22fdf73e9d1f4 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala @@ -19,15 +19,13 @@ package org.apache.spark.storage import java.util.concurrent.ConcurrentHashMap -private[storage] class BlockInfo(private var _level: StorageLevel, val tellMaster: Boolean) { +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) { // To save space, 'pending' and 'failed' are encoded as special sizes: @volatile var size: Long = BlockInfo.BLOCK_PENDING private def pending: Boolean = size == BlockInfo.BLOCK_PENDING private def failed: Boolean = size == BlockInfo.BLOCK_FAILED private def initThread: Thread = BlockInfo.blockInfoInitThreads.get(this) - def level: StorageLevel = _level - setInitThread() private def setInitThread() { @@ -72,18 +70,6 @@ private[storage] class BlockInfo(private var _level: StorageLevel, val tellMaste this.notifyAll() } } - - /** - * Update the storage level. - * - * This is currently only used for forcing blocks with storage level MEMORY_AND_DISK to disk. - * BlockManager does not provide an interface to do this, and so we workaround this by passing - * in a mock DISK_ONLY level and manually restoring the original level afterwards. - */ - def updateStorageLevel(newLevel: StorageLevel) = { - _level = newLevel - } - } private object BlockInfo { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index a8d33b54e2a6f..c2007c206dacf 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -241,18 +241,6 @@ private[spark] class BlockManager( } } - /** - * Update the storage level of an existing block. This is currently only used for forcing - * MEMORY_AND_DISK blocks to disk and restoring the original storage level afterwards. - */ - def updateStorageLevel(blockId: BlockId, level: StorageLevel) = { - if (blockInfo.contains(blockId)) { - blockInfo(blockId).updateStorageLevel(level) - } else { - logWarning(s"Block $blockId not found when attempting to update its storage level!") - } - } - /** * Get the ids of existing blocks that match the given filter. Note that this will * query the blocks stored in the disk block manager (that the block manager @@ -572,13 +560,14 @@ private[spark] class BlockManager( iter } - def put( + def putIterator( blockId: BlockId, values: Iterator[Any], level: StorageLevel, - tellMaster: Boolean): Seq[(BlockId, BlockStatus)] = { + tellMaster: Boolean = true, + effectiveStorageLevel: Option[StorageLevel] = None): Seq[(BlockId, BlockStatus)] = { require(values != null, "Values is null") - doPut(blockId, IteratorValues(values), level, tellMaster) + doPut(blockId, IteratorValues(values), level, tellMaster, effectiveStorageLevel) } /** @@ -600,13 +589,14 @@ private[spark] class BlockManager( * Put a new block of values to the block manager. * Return a list of blocks updated as a result of this put. */ - def put( + def putArray( blockId: BlockId, values: Array[Any], level: StorageLevel, - tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { + tellMaster: Boolean = true, + effectiveStorageLevel: Option[StorageLevel] = None): Seq[(BlockId, BlockStatus)] = { require(values != null, "Values is null") - doPut(blockId, ArrayValues(values), level, tellMaster) + doPut(blockId, ArrayValues(values), level, tellMaster, effectiveStorageLevel) } /** @@ -617,19 +607,33 @@ private[spark] class BlockManager( blockId: BlockId, bytes: ByteBuffer, level: StorageLevel, - tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { + tellMaster: Boolean = true, + effectiveStorageLevel: Option[StorageLevel] = None): Seq[(BlockId, BlockStatus)] = { require(bytes != null, "Bytes is null") - doPut(blockId, ByteBufferValues(bytes), level, tellMaster) + doPut(blockId, ByteBufferValues(bytes), level, tellMaster, effectiveStorageLevel) } + /** + * Put the given block according to the given level in one of the block stores, replicating + * the values if necessary. + * + * The effective storage level refers to the level according to which the block will actually be + * handled. This allows the caller to specify an alternate behavior of doPut while preserving + * the original level specified by the user. + */ private def doPut( blockId: BlockId, data: BlockValues, level: StorageLevel, - tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { + tellMaster: Boolean = true, + effectiveStorageLevel: Option[StorageLevel] = None) + : Seq[(BlockId, BlockStatus)] = { require(blockId != null, "BlockId is null") require(level != null && level.isValid, "StorageLevel is null or invalid") + effectiveStorageLevel.foreach { level => + require(level != null && level.isValid, "Effective StorageLevel is null or invalid") + } // Return value val updatedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] @@ -668,13 +672,16 @@ private[spark] class BlockManager( // Size of the block in bytes var size = 0L + // The level we actually use to put the block + val putLevel = effectiveStorageLevel.getOrElse(level) + // If we're storing bytes, then initiate the replication before storing them locally. // This is faster as data is already serialized and ready to send. val replicationFuture = data match { - case b: ByteBufferValues if level.replication > 1 => + case b: ByteBufferValues if putLevel.replication > 1 => // Duplicate doesn't copy the bytes, but just creates a wrapper val bufferView = b.buffer.duplicate() - Future { replicate(blockId, bufferView, level) } + Future { replicate(blockId, bufferView, putLevel) } case _ => null } @@ -687,18 +694,18 @@ private[spark] class BlockManager( // returnValues - Whether to return the values put // blockStore - The type of storage to put these values into val (returnValues, blockStore: BlockStore) = { - if (level.useMemory) { + if (putLevel.useMemory) { // Put it in memory first, even if it also has useDisk set to true; // We will drop it to disk later if the memory store can't hold it. (true, memoryStore) - } else if (level.useOffHeap) { + } else if (putLevel.useOffHeap) { // Use tachyon for off-heap storage (false, tachyonStore) - } else if (level.useDisk) { + } else if (putLevel.useDisk) { // Don't get back the bytes from put unless we replicate them - (level.replication > 1, diskStore) + (putLevel.replication > 1, diskStore) } else { - assert(level == StorageLevel.NONE) + assert(putLevel == StorageLevel.NONE) throw new BlockException( blockId, s"Attempted to put block $blockId without specifying storage level!") } @@ -707,22 +714,22 @@ private[spark] class BlockManager( // Actually put the values val result = data match { case IteratorValues(iterator) => - blockStore.putValues(blockId, iterator, level, returnValues) + blockStore.putValues(blockId, iterator, putLevel, returnValues) case ArrayValues(array) => - blockStore.putValues(blockId, array, level, returnValues) + blockStore.putValues(blockId, array, putLevel, returnValues) case ByteBufferValues(bytes) => bytes.rewind() - blockStore.putBytes(blockId, bytes, level) + blockStore.putBytes(blockId, bytes, putLevel) } size = result.size result.data match { - case Left (newIterator) if level.useMemory => valuesAfterPut = newIterator + case Left (newIterator) if putLevel.useMemory => valuesAfterPut = newIterator case Right (newBytes) => bytesAfterPut = newBytes case _ => } // Keep track of which blocks are dropped from memory - if (level.useMemory) { + if (putLevel.useMemory) { result.droppedBlocks.foreach { updatedBlocks += _ } } @@ -753,7 +760,7 @@ private[spark] class BlockManager( // Either we're storing bytes and we asynchronously started replication, or we're storing // values and need to serialize and replicate them now: - if (level.replication > 1) { + if (putLevel.replication > 1) { data match { case ByteBufferValues(bytes) => if (replicationFuture != null) { @@ -769,7 +776,7 @@ private[spark] class BlockManager( } bytesAfterPut = dataSerialize(blockId, valuesAfterPut) } - replicate(blockId, bytesAfterPut, level) + replicate(blockId, bytesAfterPut, putLevel) logDebug("Put block %s remotely took %s" .format(blockId, Utils.getUsedTimeMs(remoteStartTime))) } @@ -777,7 +784,7 @@ private[spark] class BlockManager( BlockManager.dispose(bytesAfterPut) - if (level.replication > 1) { + if (putLevel.replication > 1) { logDebug("Putting block %s with replication took %s" .format(blockId, Utils.getUsedTimeMs(startTimeMs))) } else { @@ -829,7 +836,7 @@ private[spark] class BlockManager( value: Any, level: StorageLevel, tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { - put(blockId, Iterator(value), level, tellMaster) + putIterator(blockId, Iterator(value), level, tellMaster) } /** diff --git a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala index 328be158db680..75c2e09a6bbb8 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -48,7 +48,7 @@ private[spark] object ThreadingTest { val block = (1 to blockSize).map(_ => Random.nextInt()) val level = randomLevel() val startTime = System.currentTimeMillis() - manager.put(blockId, block.iterator, level, tellMaster = true) + manager.putIterator(blockId, block.iterator, level, tellMaster = true) println("Pushed block " + blockId + " in " + (System.currentTimeMillis - startTime) + " ms") queue.add((blockId, block)) } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index ba35add7dc97c..96c9b370cb576 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -395,7 +395,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter master.removeExecutor(store.blockManagerId.executorId) val t1 = new Thread { override def run() { - store.put("a2", a2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("a2", a2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) } } val t2 = new Thread { @@ -429,9 +429,9 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter val list2 = List(new Array[Byte](500), new Array[Byte](1000), new Array[Byte](1500)) val list1SizeEstimate = SizeEstimator.estimate(list1.iterator.toArray) val list2SizeEstimate = SizeEstimator.estimate(list2.iterator.toArray) - store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store.put("list2memory", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store.put("list2disk", list2.iterator, StorageLevel.DISK_ONLY, tellMaster = true) + store.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list2memory", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list2disk", list2.iterator, StorageLevel.DISK_ONLY, tellMaster = true) val list1Get = store.get("list1") assert(list1Get.isDefined, "list1 expected to be in store") assert(list1Get.get.data.size === 2) @@ -662,9 +662,9 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter val list1 = List(new Array[Byte](2000), new Array[Byte](2000)) val list2 = List(new Array[Byte](2000), new Array[Byte](2000)) val list3 = List(new Array[Byte](2000), new Array[Byte](2000)) - store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store.put("list2", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store.put("list3", list3.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list2", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list3", list3.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(store.get("list2").isDefined, "list2 was not in store") assert(store.get("list2").get.data.size === 2) assert(store.get("list3").isDefined, "list3 was not in store") @@ -673,7 +673,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(store.get("list2").isDefined, "list2 was not in store") assert(store.get("list2").get.data.size === 2) // At this point list2 was gotten last, so LRU will getSingle rid of list3 - store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(store.get("list1").isDefined, "list1 was not in store") assert(store.get("list1").get.data.size === 2) assert(store.get("list2").isDefined, "list2 was not in store") @@ -689,9 +689,9 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter val list3 = List(new Array[Byte](2000), new Array[Byte](2000)) val list4 = List(new Array[Byte](2000), new Array[Byte](2000)) // First store list1 and list2, both in memory, and list3, on disk only - store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) - store.put("list2", list2.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) - store.put("list3", list3.iterator, StorageLevel.DISK_ONLY, tellMaster = true) + store.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) + store.putIterator("list2", list2.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) + store.putIterator("list3", list3.iterator, StorageLevel.DISK_ONLY, tellMaster = true) val listForSizeEstimate = new ArrayBuffer[Any] listForSizeEstimate ++= list1.iterator val listSize = SizeEstimator.estimate(listForSizeEstimate) @@ -709,7 +709,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(store.get("list3").isDefined, "list3 was not in store") assert(store.get("list3").get.data.size === 2) // Now let's add in list4, which uses both disk and memory; list1 should drop out - store.put("list4", list4.iterator, StorageLevel.MEMORY_AND_DISK_SER, tellMaster = true) + store.putIterator("list4", list4.iterator, StorageLevel.MEMORY_AND_DISK_SER, tellMaster = true) assert(store.get("list1") === None, "list1 was in store") assert(store.get("list2").isDefined, "list2 was not in store") assert(store.get("list2").get.data.size === 2) @@ -881,21 +881,21 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter // 1 updated block (i.e. list1) val updatedBlocks1 = - store.put("list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(updatedBlocks1.size === 1) assert(updatedBlocks1.head._1 === TestBlockId("list1")) assert(updatedBlocks1.head._2.storageLevel === StorageLevel.MEMORY_ONLY) // 1 updated block (i.e. list2) val updatedBlocks2 = - store.put("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) assert(updatedBlocks2.size === 1) assert(updatedBlocks2.head._1 === TestBlockId("list2")) assert(updatedBlocks2.head._2.storageLevel === StorageLevel.MEMORY_ONLY) // 2 updated blocks - list1 is kicked out of memory while list3 is added val updatedBlocks3 = - store.put("list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(updatedBlocks3.size === 2) updatedBlocks3.foreach { case (id, status) => id match { @@ -908,7 +908,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter // 2 updated blocks - list2 is kicked out of memory (but put on disk) while list4 is added val updatedBlocks4 = - store.put("list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(updatedBlocks4.size === 2) updatedBlocks4.foreach { case (id, status) => id match { @@ -921,7 +921,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter // 1 updated block - list5 is too big to fit in store, but list3 is kicked out in the process val updatedBlocks5 = - store.put("list5", bigList.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list5", bigList.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(updatedBlocks5.size === 1) assert(updatedBlocks5.head._1 === TestBlockId("list3")) assert(updatedBlocks5.head._2.storageLevel === StorageLevel.NONE) @@ -937,9 +937,9 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter val list = List.fill(2)(new Array[Byte](2000)) // Tell master. By LRU, only list2 and list3 remains. - store.put("list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store.put("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) - store.put("list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator("list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) // getLocations and getBlockStatus should yield the same locations assert(store.master.getLocations("list1").size === 0) @@ -953,9 +953,9 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(store.master.getBlockStatus("list3", askSlaves = true).size === 1) // This time don't tell master and see what happens. By LRU, only list5 and list6 remains. - store.put("list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) - store.put("list5", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) - store.put("list6", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) + store.putIterator("list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) + store.putIterator("list5", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + store.putIterator("list6", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) // getLocations should return nothing because the master is not informed // getBlockStatus without asking slaves should have the same result @@ -977,18 +977,18 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter val list = List.fill(2)(new Array[Byte](100)) // insert some blocks - store.put("list1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) - store.put("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) - store.put("list3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator("list1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator("list3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) // getLocations and getBlockStatus should yield the same locations assert(store.master.getMatchingBlockIds(_.toString.contains("list"), askSlaves = false).size === 3) assert(store.master.getMatchingBlockIds(_.toString.contains("list1"), askSlaves = false).size === 1) // insert some more blocks - store.put("newlist1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) - store.put("newlist2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) - store.put("newlist3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + store.putIterator("newlist1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator("newlist2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + store.putIterator("newlist3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) // getLocations and getBlockStatus should yield the same locations assert(store.master.getMatchingBlockIds(_.toString.contains("newlist"), askSlaves = false).size === 1) @@ -996,7 +996,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter val blockIds = Seq(RDDBlockId(1, 0), RDDBlockId(1, 1), RDDBlockId(2, 0)) blockIds.foreach { blockId => - store.put(blockId, list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator(blockId, list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) } val matchedBlockIds = store.master.getMatchingBlockIds(_ match { case RDDBlockId(1, _) => true diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index 4579ffea678bd..d934b9cbfc3e8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -110,7 +110,7 @@ private[streaming] class ReceiverSupervisorImpl( ) { val blockId = optionalBlockId.getOrElse(nextBlockId) val time = System.currentTimeMillis - blockManager.put(blockId, arrayBuffer.toArray[Any], storageLevel, tellMaster = true) + blockManager.putArray(blockId, arrayBuffer.toArray[Any], storageLevel, tellMaster = true) logDebug("Pushed block " + blockId + " in " + (System.currentTimeMillis - time) + " ms") reportPushedBlock(blockId, arrayBuffer.size, optionalMetadata) } @@ -123,7 +123,7 @@ private[streaming] class ReceiverSupervisorImpl( ) { val blockId = optionalBlockId.getOrElse(nextBlockId) val time = System.currentTimeMillis - blockManager.put(blockId, iterator, storageLevel, tellMaster = true) + blockManager.putIterator(blockId, iterator, storageLevel, tellMaster = true) logDebug("Pushed block " + blockId + " in " + (System.currentTimeMillis - time) + " ms") reportPushedBlock(blockId, -1, optionalMetadata) } From 3dd96aac9c72fa630b532c3a8072cfec25f41de3 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 21 Jul 2014 13:42:37 -0700 Subject: [PATCH 30/55] AppendOnlyBuffer -> Vector (+ a few small changes) --- .../apache/spark/storage/MemoryStore.scala | 4 +- .../spark/util/collection/SizeTracker.scala | 8 ++-- ...yBuffer.scala => SizeTrackingVector.scala} | 7 ++- .../util/collection/SizeTrackerSuite.scala | 48 +++++++++---------- 4 files changed, 35 insertions(+), 32 deletions(-) rename core/src/main/scala/org/apache/spark/util/collection/{SizeTrackingAppendOnlyBuffer.scala => SizeTrackingVector.scala} (89%) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 5744481237f71..3bff514476d0a 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.SparkEnv import org.apache.spark.util.{SizeEstimator, Utils} -import org.apache.spark.util.collection.SizeTrackingAppendOnlyBuffer +import org.apache.spark.util.collection.SizeTrackingVector private case class MemoryEntry(value: Any, size: Long, deserialized: Boolean) @@ -213,7 +213,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) val threadId = Thread.currentThread().getId val unrollMemoryMap = SparkEnv.get.unrollMemoryMap - var buffer = new SizeTrackingAppendOnlyBuffer[Any] + var buffer = new SizeTrackingVector[Any] try { while (values.hasNext && !atMemoryLimit) { diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTracker.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTracker.scala index fd71c51886c45..3eb1010dc1e8d 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/SizeTracker.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTracker.scala @@ -17,7 +17,7 @@ package org.apache.spark.util.collection -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable import org.apache.spark.util.SizeEstimator @@ -37,7 +37,7 @@ private[spark] trait SizeTracker { private val SAMPLE_GROWTH_RATE = 1.1 /** Samples taken since last resetSamples(). Only the last two are kept for extrapolation. */ - private val samples = new ArrayBuffer[Sample] + private val samples = new mutable.Queue[Sample] /** The average number of bytes per update between our last two samples. */ private var bytesPerUpdate: Double = _ @@ -75,10 +75,10 @@ private[spark] trait SizeTracker { * Take a new sample of the current collection's size. */ private def takeSample(): Unit = { - samples += Sample(SizeEstimator.estimate(this), numUpdates) + samples.enqueue(Sample(SizeEstimator.estimate(this), numUpdates)) // Only use the last two samples to extrapolate if (samples.size > 2) { - samples.remove(0) + samples.dequeue() } val bytesDelta = samples.toList.reverse match { case latest :: previous :: tail => diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyBuffer.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingVector.scala similarity index 89% rename from core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyBuffer.scala rename to core/src/main/scala/org/apache/spark/util/collection/SizeTrackingVector.scala index 2ea4b1d996fe4..65a7b4e0d497b 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingVector.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag /** * An append-only buffer that keeps track of its estimated size in bytes. */ -private[spark] class SizeTrackingAppendOnlyBuffer[T: ClassTag] +private[spark] class SizeTrackingVector[T: ClassTag] extends PrimitiveVector[T] with SizeTracker { @@ -37,7 +37,10 @@ private[spark] class SizeTrackingAppendOnlyBuffer[T: ClassTag] this } - override def array: Array[T] = { + /** + * Return a trimmed version of the underlying array. + */ + def toArray: Array[T] = { super.iterator.toArray } } diff --git a/core/src/test/scala/org/apache/spark/util/collection/SizeTrackerSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/SizeTrackerSuite.scala index 83ad339506d97..1f33967249654 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/SizeTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/SizeTrackerSuite.scala @@ -30,20 +30,20 @@ class SizeTrackerSuite extends FunSuite { import SizeTrackerSuite._ - test("buffer fixed size insertions") { - testBuffer[Long](10000, i => i.toLong) - testBuffer[(Long, Long)](10000, i => (i.toLong, i.toLong)) - testBuffer[LargeDummyClass](10000, i => new LargeDummyClass) + test("vector fixed size insertions") { + testVector[Long](10000, i => i.toLong) + testVector[(Long, Long)](10000, i => (i.toLong, i.toLong)) + testVector[LargeDummyClass](10000, i => new LargeDummyClass) } - test("buffer variable size insertions") { + test("vector variable size insertions") { val rand = new Random(123456789) def randString(minLen: Int, maxLen: Int): String = { "a" * (rand.nextInt(maxLen - minLen) + minLen) } - testBuffer[String](10000, i => randString(0, 10)) - testBuffer[String](10000, i => randString(0, 100)) - testBuffer[String](10000, i => randString(90, 100)) + testVector[String](10000, i => randString(0, 10)) + testVector[String](10000, i => randString(0, 100)) + testVector[String](10000, i => randString(90, 100)) } test("map fixed size insertions") { @@ -70,12 +70,12 @@ class SizeTrackerSuite extends FunSuite { testMap[String, Int](10000, i => (randString(0, 10000), i)) } - def testBuffer[T: ClassTag](numElements: Int, makeElement: Int => T) { - val buffer = new SizeTrackingAppendOnlyBuffer[T] + def testVector[T: ClassTag](numElements: Int, makeElement: Int => T) { + val vector = new SizeTrackingVector[T] for (i <- 0 until numElements) { val item = makeElement(i) - buffer += item - expectWithinError(buffer, buffer.estimateSize(), if (i < 32) HIGH_ERROR else NORMAL_ERROR) + vector += item + expectWithinError(vector, vector.estimateSize(), if (i < 32) HIGH_ERROR else NORMAL_ERROR) } } @@ -108,40 +108,40 @@ private object SizeTrackerSuite { System.exit(1) } val numElements = args(0).toInt - bufferSpeedTest(numElements) + vectorSpeedTest(numElements) mapSpeedTest(numElements) } /** - * Speed test for SizeTrackingAppendOnlyBuffer. + * Speed test for SizeTrackingVector. * * Results for 100000 elements (possibly non-deterministic): * PrimitiveVector 15 ms * SizeTracker 51 ms * SizeEstimator 2000 ms */ - def bufferSpeedTest(numElements: Int): Unit = { + def vectorSpeedTest(numElements: Int): Unit = { val baseTimes = for (i <- 0 until 10) yield time { - val buffer = new PrimitiveVector[LargeDummyClass] + val vector = new PrimitiveVector[LargeDummyClass] for (i <- 0 until numElements) { - buffer += new LargeDummyClass + vector += new LargeDummyClass } } val sampledTimes = for (i <- 0 until 10) yield time { - val buffer = new SizeTrackingAppendOnlyBuffer[LargeDummyClass] + val vector = new SizeTrackingVector[LargeDummyClass] for (i <- 0 until numElements) { - buffer += new LargeDummyClass - buffer.estimateSize() + vector += new LargeDummyClass + vector.estimateSize() } } val unsampledTimes = for (i <- 0 until 3) yield time { - val buffer = new PrimitiveVector[LargeDummyClass] + val vector = new PrimitiveVector[LargeDummyClass] for (i <- 0 until numElements) { - buffer += new LargeDummyClass - SizeEstimator.estimate(buffer) + vector += new LargeDummyClass + SizeEstimator.estimate(vector) } } - printSpeedTestResult("SizeTrackingAppendOnlyBuffer", baseTimes, sampledTimes, unsampledTimes) + printSpeedTestResult("SizeTrackingVector", baseTimes, sampledTimes, unsampledTimes) } /** From abeae4f9c7083f3a89d7e36781fce077f2af9f53 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 21 Jul 2014 13:46:16 -0700 Subject: [PATCH 31/55] Add comment clarifying the MEMORY_AND_DISK case --- core/src/main/scala/org/apache/spark/storage/BlockManager.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index c2007c206dacf..690c8f32a705e 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -465,6 +465,8 @@ private[spark] class BlockManager( // Cache the values before returning them val putResult = memoryStore.putValues( blockId, values, level, returnValues = true, allowPersistToDisk = false) + // The put may or may not have succeeded, depending on whether there was enough + // space to unroll the block. Either way, putValues should return an iterator. putResult.data match { case Left(it) => return Some(new BlockResult(it, DataReadMethod.Disk, info.size)) From 9a65245f4333f83f7191163b35b0a388544e160a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 22 Jul 2014 11:32:38 -0700 Subject: [PATCH 32/55] Update a few comments + minor control flow changes --- .../scala/org/apache/spark/CacheManager.scala | 23 +++++++++++-------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index e4c027c8532eb..ffcb9a6a9ad4d 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -118,8 +118,13 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { } /** - * Cache the values of a partition, keeping track of any updates in the storage statuses - * of other blocks along the way. + * Cache the values of a partition, keeping track of any updates in the storage statuses of + * other blocks along the way. + * + * The effective storage level refers to the level that actually specifies BlockManager put + * behavior, not the level originally specified by the user. This is mainly for forcing a + * MEMORY_AND_DISK partition to disk if there is not enough room to unroll the partition, + * while preserving the the original semantics of the RDD as specified by the application. */ private def putInBlockManager[T]( key: BlockId, @@ -131,10 +136,8 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { val putLevel = effectiveStorageLevel.getOrElse(level) if (!putLevel.useMemory) { /* - * This RDD is not to be cached in memory, so we can just pass the computed values - * as an iterator directly to the BlockManager, rather than first fully unrolling - * it in memory. The latter option potentially uses much more memory and risks OOM - * exceptions that can be avoided. + * This RDD is not to be cached in memory, so we can just pass the computed values as an + * iterator directly to the BlockManager rather than first fully unrolling it in memory. */ updatedBlocks ++= blockManager.putIterator(key, values, level, tellMaster = true, effectiveStorageLevel) @@ -165,15 +168,15 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { // There is not enough space to cache this partition in memory logWarning(s"Not enough space to cache $key in memory! " + s"Free memory is ${blockManager.memoryStore.freeMemory} bytes.") - var returnValues = it.asInstanceOf[Iterator[T]] + val returnValues = it.asInstanceOf[Iterator[T]] if (putLevel.useDisk) { logWarning(s"Persisting $key to disk instead.") val diskOnlyLevel = StorageLevel(useDisk = true, useMemory = false, useOffHeap = false, deserialized = false, putLevel.replication) - returnValues = - putInBlockManager[T](key, returnValues, level, updatedBlocks, Some(diskOnlyLevel)) + putInBlockManager[T](key, returnValues, level, updatedBlocks, Some(diskOnlyLevel)) + } else { + returnValues } - returnValues } } } From 129c441bd5da71892a31fabe99068c6859cc6623 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 22 Jul 2014 11:38:09 -0700 Subject: [PATCH 33/55] Fix bug: Use toArray rather than array --- core/src/main/scala/org/apache/spark/storage/MemoryStore.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 3bff514476d0a..ef9b35ff20804 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -252,7 +252,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) if (!atMemoryLimit) { // We successfully unrolled the entirety of this block - Left(buffer.array) + Left(buffer.toArray) } else { // We ran out of space while unrolling the values for this block Right(buffer.iterator ++ values) From 728323bed98f187b614f1f76c3b363a48b5dd90d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 23 Jul 2014 19:10:26 -0700 Subject: [PATCH 34/55] Do not synchronize every 1000 elements The new mechanism is as follows: Each thread maintains a local threshold, and checks every N elements whether its vector's size has exceeded the threshold. If so, request more memory, equal to the current vector size * F, then set the new threshold to this value. In my implementation, N = 16 and F = 1.5. We also start off with an initial threshold of 1MB (arbitrary). Note that estimating the size is cheap because of our underlying implementation of SizeTrackingVector. This avoids synchronizing across all threads every 1000 elements as before, which could be expensive. --- .../apache/spark/storage/MemoryStore.scala | 88 +++++++++++-------- .../spark/storage/BlockManagerSuite.scala | 45 +++++----- 2 files changed, 73 insertions(+), 60 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index ef9b35ff20804..822ceccb634fb 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -205,63 +205,73 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) droppedBlocks: ArrayBuffer[(BlockId, BlockStatus)]) : Either[Array[Any], Iterator[Any]] = { - var count = 0 // The number of elements unrolled so far - var atMemoryLimit = false // Whether we are at the memory limit for unrolling blocks - var previousSize = 0L // Previous estimate of the size of our buffer - val memoryRequestPeriod = 1000 // How frequently we request more memory for our buffer - val memoryRequestThreshold = 100 // Before this is exceeded, request memory every round + // Number of elements unrolled so far + var elementsUnrolled = 0 + // Whether there is still enough memory for us to continue unrolling this block + var keepUnrolling = true + // Initial per-thread memory to request for unrolling blocks (bytes). Exposed for testing. + val initialMemoryThreshold = conf.getLong("spark.storage.unrollMemoryThreshold", 1024 * 1024) + // How often to check whether we need to request more memory. Exposed for testing. + val memoryCheckPeriod = conf.getLong("spark.storage.unrollCheckPeriod", 16) + // Memory currently reserved by this thread (bytes) + var memoryThreshold = initialMemoryThreshold + // Memory to request as a multiple of current vector size + val memoryGrowthFactor = 1.5 val threadId = Thread.currentThread().getId val unrollMemoryMap = SparkEnv.get.unrollMemoryMap - var buffer = new SizeTrackingVector[Any] + var vector = new SizeTrackingVector[Any] + + // Request memory for our vector and return whether request is granted. + // This involves synchronizing across all threads and can be expensive if called frequently. + def requestMemory(memoryToRequest: Long): Boolean = { + unrollMemoryMap.synchronized { + val previouslyOccupiedMemory = unrollMemoryMap.get(threadId).getOrElse(0L) + val otherThreadsMemory = unrollMemoryMap.values.sum - previouslyOccupiedMemory + val availableMemory = freeMemory - otherThreadsMemory + val granted = availableMemory > memoryToRequest + if (granted) { unrollMemoryMap(threadId) = memoryToRequest } + granted + } + } - try { - while (values.hasNext && !atMemoryLimit) { - buffer += values.next() - count += 1 - if (count % memoryRequestPeriod == 1 || count < memoryRequestThreshold) { - // Calculate the amount of memory to request from the global memory pool - val currentSize = buffer.estimateSize() - val delta = if (previousSize > 0) math.max(currentSize - previousSize, 0) else 0 - val memoryToRequest = currentSize + delta - previousSize = currentSize - - // Atomically check whether there is sufficient memory in the global pool to continue - unrollMemoryMap.synchronized { - val previouslyOccupiedMemory = unrollMemoryMap.get(threadId).getOrElse(0L) - val otherThreadsMemory = unrollMemoryMap.values.sum - previouslyOccupiedMemory - - // Request memory for the local buffer and return whether request is granted - def requestMemory(): Boolean = { - val availableMemory = freeMemory - otherThreadsMemory - val granted = availableMemory > memoryToRequest - if (granted) { unrollMemoryMap(threadId) = memoryToRequest } - granted - } + // Request enough memory to begin unrolling + keepUnrolling = requestMemory(memoryThreshold) - // If the first request is not granted, try again after ensuring free space - // If there is still not enough space, give up and drop the partition - if (!requestMemory()) { + // Unroll this block safely, checking whether we have exceeded our threshold periodically + try { + while (values.hasNext && keepUnrolling) { + vector += values.next() + if (elementsUnrolled % memoryCheckPeriod == 0) { + // If our vector's size has exceeded the threshold, request more memory + val currentSize = vector.estimateSize() + if (currentSize >= memoryThreshold) { + val amountToRequest = (currentSize * memoryGrowthFactor).toLong + if (!requestMemory(amountToRequest)) { + // If the first request is not granted, try again after ensuring free space + // If there is still not enough space, give up and drop the partition val result = ensureFreeSpace(blockId, globalBufferMemory) droppedBlocks ++= result.droppedBlocks - atMemoryLimit = !requestMemory() + keepUnrolling = requestMemory(amountToRequest) } + memoryThreshold = amountToRequest } } + elementsUnrolled += 1 } - if (!atMemoryLimit) { + if (keepUnrolling) { // We successfully unrolled the entirety of this block - Left(buffer.toArray) + Left(vector.toArray) } else { // We ran out of space while unrolling the values for this block - Right(buffer.iterator ++ values) + Right(vector.iterator ++ values) } } finally { - // Unless we return an iterator that depends on the buffer, free up space for other threads - if (!atMemoryLimit) { - buffer = null + // Unless we return an iterator that depends on the vector, free up space for other threads + if (keepUnrolling) { + vector = null unrollMemoryMap.synchronized { unrollMemoryMap(threadId) = 0 } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 96c9b370cb576..d02f6a18c0465 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -75,6 +75,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter conf.set("spark.storage.disableBlockManagerHeartBeat", "true") conf.set("spark.driver.port", boundPort.toString) conf.set("spark.storage.bufferFraction", "0.4") + conf.set("spark.storage.unrollMemoryThreshold", "512") + conf.set("spark.storage.unrollCheckPeriod", "1") SparkEnv.set(env) master = new BlockManagerMaster( @@ -143,11 +145,11 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("master + 1 manager interaction") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, + store = new BlockManager("", actorSystem, master, serializer, 20000, conf, securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) // Putting a1, a2 and a3 in memory and telling master only about a1 and a2 store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) @@ -192,11 +194,11 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("removing block") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, + store = new BlockManager("", actorSystem, master, serializer, 20000, conf, securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) // Putting a1, a2 and a3 in memory and telling master only about a1 and a2 store.putSingle("a1-to-remove", a1, StorageLevel.MEMORY_ONLY) @@ -205,8 +207,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter // Checking whether blocks are in memory and memory size val memStatus = master.getMemoryStatus.head._2 - assert(memStatus._1 == 2000L, "total memory " + memStatus._1 + " should equal 2000") - assert(memStatus._2 <= 1200L, "remaining memory " + memStatus._2 + " should <= 1200") + assert(memStatus._1 == 20000L, "total memory " + memStatus._1 + " should equal 20000") + assert(memStatus._2 <= 12000L, "remaining memory " + memStatus._2 + " should <= 12000") assert(store.getSingle("a1-to-remove").isDefined, "a1 was not in store") assert(store.getSingle("a2-to-remove").isDefined, "a2 was not in store") assert(store.getSingle("a3-to-remove").isDefined, "a3 was not in store") @@ -235,17 +237,17 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { val memStatus = master.getMemoryStatus.head._2 - memStatus._1 should equal (2000L) - memStatus._2 should equal (2000L) + memStatus._1 should equal (20000L) + memStatus._2 should equal (20000L) } } test("removing rdd") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, + store = new BlockManager("", actorSystem, master, serializer, 20000, conf, securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) // Putting a1, a2 and a3 in memory. store.putSingle(rdd(0, 0), a1, StorageLevel.MEMORY_ONLY) store.putSingle(rdd(0, 1), a2, StorageLevel.MEMORY_ONLY) @@ -925,6 +927,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(updatedBlocks5.size === 1) assert(updatedBlocks5.head._1 === TestBlockId("list3")) assert(updatedBlocks5.head._2.storageLevel === StorageLevel.NONE) + assert(!store.get("list1").isDefined, "list1 was in store") assert(store.get("list2").isDefined, "list2 was not in store") assert(!store.get("list3").isDefined, "list3 was in store") assert(store.get("list4").isDefined, "list4 was not in store") @@ -1025,14 +1028,14 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter store = new BlockManager( "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) /* - * After a1 and a2 are cached, there is less than 4000 bytes of free space left. When we try + * After a1 and a2 are cached, we do not have enough room to unroll a3 safely. When we try * to cache a3, we attempt to ensure free space of only 12000 * 0.2 = 2400 bytes, which is * not enough to drop a1 or a2, as there is already >= 2400 bytes of free space. Therefore, * we give up and drop a3 on the spot. */ store.putSingle("a1", new Array[Byte](4000), StorageLevel.MEMORY_ONLY) store.putSingle("a2", new Array[Byte](4000), StorageLevel.MEMORY_ONLY) - store.putSingle("a3", new Array[Byte](4000), StorageLevel.MEMORY_ONLY) + store.putSingle("a3", new Array[Byte](3000), StorageLevel.MEMORY_ONLY) // Memory store should contain a1, a2 assert(store.memoryStore.contains("a1"), "a1 was not in store") assert(store.memoryStore.contains("a2"), "a2 was not in store") @@ -1043,7 +1046,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter * (i.e. a1) from memory to accommodate a5 (because 2000 < 2400). */ store.putSingle("a4", new Array[Byte](2000), StorageLevel.MEMORY_ONLY) - store.putSingle("a5", new Array[Byte](4000), StorageLevel.MEMORY_ONLY) + store.putSingle("a5", new Array[Byte](3000), StorageLevel.MEMORY_ONLY) // Memory store should contain a2, a4, a5 assert(!store.memoryStore.contains("a1"), "a1 was in store") assert(store.memoryStore.contains("a2"), "a2 was not in store") @@ -1062,7 +1065,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter */ store.putSingle("a1", new Array[Byte](4000), StorageLevel.MEMORY_AND_DISK) store.putSingle("a2", new Array[Byte](4000), StorageLevel.MEMORY_AND_DISK) - store.putSingle("a3", new Array[Byte](4000), StorageLevel.MEMORY_AND_DISK) + store.putSingle("a3", new Array[Byte](3000), StorageLevel.MEMORY_AND_DISK) // Memory store should contain a1, a2; disk store should contain a3 assert(store.memoryStore.contains("a1"), "a1 was not in memory store") assert(store.memoryStore.contains("a2"), "a2 was not in memory store") @@ -1071,7 +1074,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(!store.diskStore.contains("a2"), "a2 was in disk store") assert(store.diskStore.contains("a3"), "a3 was not in disk store") store.putSingle("a4", new Array[Byte](2000), StorageLevel.MEMORY_AND_DISK) - store.putSingle("a5", new Array[Byte](4000), StorageLevel.MEMORY_AND_DISK) + store.putSingle("a5", new Array[Byte](3000), StorageLevel.MEMORY_AND_DISK) // Memory store should contain a2, a4, a5; disk store should contain a1, a3 assert(!store.memoryStore.contains("a1"), "a1 was in memory store") assert(store.memoryStore.contains("a2"), "a2 was not in memory store") From 28edfa387d92ae4c6e02351e8b2d177f3475ff77 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 23 Jul 2014 19:21:28 -0700 Subject: [PATCH 35/55] Update a few comments / log messages --- core/src/main/scala/org/apache/spark/CacheManager.scala | 4 ++-- .../scala/org/apache/spark/storage/MemoryStore.scala | 9 ++++++--- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index ffcb9a6a9ad4d..5ddda4d6953fa 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -166,11 +166,11 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { arr.iterator.asInstanceOf[Iterator[T]] case Right(it) => // There is not enough space to cache this partition in memory - logWarning(s"Not enough space to cache $key in memory! " + + logWarning(s"Not enough space to cache partition $key in memory! " + s"Free memory is ${blockManager.memoryStore.freeMemory} bytes.") val returnValues = it.asInstanceOf[Iterator[T]] if (putLevel.useDisk) { - logWarning(s"Persisting $key to disk instead.") + logWarning(s"Persisting partition $key to disk instead.") val diskOnlyLevel = StorageLevel(useDisk = true, useMemory = false, useOffHeap = false, deserialized = false, putLevel.replication) putInBlockManager[T](key, returnValues, level, updatedBlocks, Some(diskOnlyLevel)) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 822ceccb634fb..cdcb3f19111bd 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -128,9 +128,10 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) PutResult(res.size, res.data, droppedBlocks) case Right(iteratorValues) => // Not enough space to unroll this block; drop to disk if applicable - logWarning(s"Not enough space to store $blockId in memory! Free memory is ${freeMemory}B.") + logWarning(s"Not enough space to store block $blockId in memory! " + + s"Free memory is $freeMemory bytes.") if (level.useDisk && allowPersistToDisk) { - logWarning(s"Persisting $blockId to disk instead.") + logWarning(s"Persisting block $blockId to disk instead.") val res = blockManager.diskStore.putValues(blockId, iteratorValues, level, returnValues) PutResult(res.size, res.data, droppedBlocks) } else { @@ -269,7 +270,9 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) } } finally { - // Unless we return an iterator that depends on the vector, free up space for other threads + // If we return an array, the values returned do not depend on the underlying vector and + // we can immediately free up space for other threads. Otherwise, if we return an iterator, + // we release the memory claimed by this thread later on when the task finishes. if (keepUnrolling) { vector = null unrollMemoryMap.synchronized { From 4f18a3d5d3722b3d9965c95408007152cd77d37a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 23 Jul 2014 19:27:05 -0700 Subject: [PATCH 36/55] bufferFraction -> unrollFraction --- .../main/scala/org/apache/spark/storage/MemoryStore.scala | 8 ++++---- .../org/apache/spark/storage/BlockManagerSuite.scala | 6 +++--- docs/configuration.md | 2 +- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index cdcb3f19111bd..662999dd439c4 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -48,9 +48,9 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) * The amount of space ensured for unrolling values in memory, shared across all cores. * This space is not reserved in advance, but allocated dynamically by dropping existing blocks. */ - private val globalBufferMemory = { - val bufferFraction = conf.getDouble("spark.storage.bufferFraction", 0.2) - (maxMemory * bufferFraction).toLong + private val globalUnrollMemory = { + val unrollFraction = conf.getDouble("spark.storage.unrollFraction", 0.2) + (maxMemory * unrollFraction).toLong } logInfo("MemoryStore started with capacity %s".format(Utils.bytesToString(maxMemory))) @@ -251,7 +251,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) if (!requestMemory(amountToRequest)) { // If the first request is not granted, try again after ensuring free space // If there is still not enough space, give up and drop the partition - val result = ensureFreeSpace(blockId, globalBufferMemory) + val result = ensureFreeSpace(blockId, globalUnrollMemory) droppedBlocks ++= result.droppedBlocks keepUnrolling = requestMemory(amountToRequest) } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index d02f6a18c0465..7f9878fbf669c 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -74,7 +74,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter conf.set("spark.test.useCompressedOops", "true") conf.set("spark.storage.disableBlockManagerHeartBeat", "true") conf.set("spark.driver.port", boundPort.toString) - conf.set("spark.storage.bufferFraction", "0.4") + conf.set("spark.storage.unrollFraction", "0.4") conf.set("spark.storage.unrollMemoryThreshold", "512") conf.set("spark.storage.unrollCheckPeriod", "1") SparkEnv.set(env) @@ -1024,7 +1024,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("safely unroll blocks") { - conf.set("spark.storage.bufferFraction", "0.2") + conf.set("spark.storage.unrollFraction", "0.2") store = new BlockManager( "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) /* @@ -1056,7 +1056,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("safely unroll blocks (disk)") { - conf.set("spark.storage.bufferFraction", "0.2") + conf.set("spark.storage.unrollFraction", "0.2") store = new BlockManager( "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) /* diff --git a/docs/configuration.md b/docs/configuration.md index 8b2d9f44b8c6b..990a044d7b0d5 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -459,7 +459,7 @@ Apart from these, the following properties are also available, and may be useful - spark.storage.bufferFraction + spark.storage.unrollFraction 0.2 Fraction of spark.storage.memoryFraction to use for unrolling blocks in memory. From 8288228fa11923ae2ef8b9647c492e1ff253a5fc Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 23 Jul 2014 20:14:25 -0700 Subject: [PATCH 37/55] Synchronize put and unroll properly When unrolling, it is possible for another thread to concurrently use up the free space we just ensured. Therefore, we need to hold the putLock. When putting, we must take into account the space we currently use for unrolling, so we don't end up putting a block into the space currently occupied by unrolling. --- .../apache/spark/storage/MemoryStore.scala | 34 +++++++++++++------ 1 file changed, 24 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 662999dd439c4..6a38881066487 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -248,14 +248,18 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) val currentSize = vector.estimateSize() if (currentSize >= memoryThreshold) { val amountToRequest = (currentSize * memoryGrowthFactor).toLong - if (!requestMemory(amountToRequest)) { - // If the first request is not granted, try again after ensuring free space - // If there is still not enough space, give up and drop the partition - val result = ensureFreeSpace(blockId, globalUnrollMemory) - droppedBlocks ++= result.droppedBlocks - keepUnrolling = requestMemory(amountToRequest) + // Hold the put lock, in case another thread concurrently puts a block that + // takes up the free space we just ensured for unrolling here + putLock.synchronized { + if (!requestMemory(amountToRequest)) { + // If the first request is not granted, try again after ensuring free space + // If there is still not enough space, give up and drop the partition + val result = ensureFreeSpace(blockId, globalUnrollMemory, unrolling = true) + droppedBlocks ++= result.droppedBlocks + keepUnrolling = requestMemory(amountToRequest) + } + memoryThreshold = amountToRequest } - memoryThreshold = amountToRequest } } elementsUnrolled += 1 @@ -357,9 +361,10 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) * * Return whether there is enough free space, along with the blocks dropped in the process. */ - private[spark] def ensureFreeSpace( + private def ensureFreeSpace( blockIdToAdd: BlockId, - space: Long): ResultWithDroppedBlocks = { + space: Long, + unrolling: Boolean = false): ResultWithDroppedBlocks = { logInfo(s"ensureFreeSpace($space) called with curMem=$currentMemory, maxMem=$maxMemory") val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] @@ -369,7 +374,16 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) return ResultWithDroppedBlocks(success = false, droppedBlocks) } - if (maxMemory - currentMemory < space) { + // Take into account the amount of memory currently occupied by unrolling blocks + val freeSpace = + if (!unrolling) { + val unrollMemoryMap = SparkEnv.get.unrollMemoryMap + unrollMemoryMap.synchronized { freeMemory - unrollMemoryMap.values.sum } + } else { + freeMemory + } + + if (freeSpace < space) { val rddToAdd = getRddId(blockIdToAdd) val selectedBlocks = new ArrayBuffer[BlockId] var selectedMemory = 0L From 3f5a0833976f6653f9be0606eb21f5bb5450196a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 24 Jul 2014 11:09:51 -0700 Subject: [PATCH 38/55] Simplify signature of ensureFreeSpace We do this by requesting a different amount of free space to ensure rather than passing in an extra boolean parameter. --- .../org/apache/spark/storage/MemoryStore.scala | 15 +++++---------- 1 file changed, 5 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 6a38881066487..05e6e30484dda 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -254,7 +254,8 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) if (!requestMemory(amountToRequest)) { // If the first request is not granted, try again after ensuring free space // If there is still not enough space, give up and drop the partition - val result = ensureFreeSpace(blockId, globalUnrollMemory, unrolling = true) + val extraSpaceNeeded = globalUnrollMemory - unrollMemoryMap.values.sum + val result = ensureFreeSpace(blockId, extraSpaceNeeded) droppedBlocks ++= result.droppedBlocks keepUnrolling = requestMemory(amountToRequest) } @@ -363,8 +364,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) */ private def ensureFreeSpace( blockIdToAdd: BlockId, - space: Long, - unrolling: Boolean = false): ResultWithDroppedBlocks = { + space: Long): ResultWithDroppedBlocks = { logInfo(s"ensureFreeSpace($space) called with curMem=$currentMemory, maxMem=$maxMemory") val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] @@ -375,13 +375,8 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) } // Take into account the amount of memory currently occupied by unrolling blocks - val freeSpace = - if (!unrolling) { - val unrollMemoryMap = SparkEnv.get.unrollMemoryMap - unrollMemoryMap.synchronized { freeMemory - unrollMemoryMap.values.sum } - } else { - freeMemory - } + val unrollMemoryMap = SparkEnv.get.unrollMemoryMap + val freeSpace = unrollMemoryMap.synchronized { freeMemory - unrollMemoryMap.values.sum } if (freeSpace < space) { val rddToAdd = getRddId(blockIdToAdd) From 69bc0a5321c7274abeff13a2996f18ce92d97805 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 24 Jul 2014 12:14:48 -0700 Subject: [PATCH 39/55] Always synchronize on putLock before unrollMemoryMap This avoids potential deadlock. --- .../apache/spark/storage/MemoryStore.scala | 55 +++++++++++-------- 1 file changed, 33 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 05e6e30484dda..deab798ee08d2 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -44,6 +44,15 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) // dropping blocks from the memory store. private val putLock = new Object() + /** + * Mapping from thread ID to memory used for unrolling blocks. + * + * To avoid potential deadlocks, all accesses of this map in MemoryStore are assumed to + * first synchronize on `putLock` and then on `unrollMemoryMap`, in that particular order. + * This is lazy because SparkEnv does not exist when we mock this class in tests. + */ + private lazy val unrollMemoryMap = SparkEnv.get.unrollMemoryMap + /** * The amount of space ensured for unrolling values in memory, shared across all cores. * This space is not reserved in advance, but allocated dynamically by dropping existing blocks. @@ -220,19 +229,20 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) val memoryGrowthFactor = 1.5 val threadId = Thread.currentThread().getId - val unrollMemoryMap = SparkEnv.get.unrollMemoryMap var vector = new SizeTrackingVector[Any] - // Request memory for our vector and return whether request is granted. - // This involves synchronizing across all threads and can be expensive if called frequently. + // Request memory for our vector and return whether the request is granted. This involves + // synchronizing on putLock and unrollMemoryMap (in that order), which could be expensive. def requestMemory(memoryToRequest: Long): Boolean = { - unrollMemoryMap.synchronized { - val previouslyOccupiedMemory = unrollMemoryMap.get(threadId).getOrElse(0L) - val otherThreadsMemory = unrollMemoryMap.values.sum - previouslyOccupiedMemory - val availableMemory = freeMemory - otherThreadsMemory - val granted = availableMemory > memoryToRequest - if (granted) { unrollMemoryMap(threadId) = memoryToRequest } - granted + putLock.synchronized { + unrollMemoryMap.synchronized { + val previouslyOccupiedMemory = unrollMemoryMap.get(threadId).getOrElse(0L) + val otherThreadsMemory = unrollMemoryMap.values.sum - previouslyOccupiedMemory + val availableMemory = freeMemory - otherThreadsMemory + val granted = availableMemory > memoryToRequest + if (granted) { unrollMemoryMap(threadId) = memoryToRequest } + granted + } } } @@ -248,18 +258,20 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) val currentSize = vector.estimateSize() if (currentSize >= memoryThreshold) { val amountToRequest = (currentSize * memoryGrowthFactor).toLong - // Hold the put lock, in case another thread concurrently puts a block that - // takes up the free space we just ensured for unrolling here + // Hold the put lock, in case another thread concurrently puts a block that takes + // up the unrolling space we just ensured here putLock.synchronized { - if (!requestMemory(amountToRequest)) { - // If the first request is not granted, try again after ensuring free space - // If there is still not enough space, give up and drop the partition - val extraSpaceNeeded = globalUnrollMemory - unrollMemoryMap.values.sum - val result = ensureFreeSpace(blockId, extraSpaceNeeded) - droppedBlocks ++= result.droppedBlocks - keepUnrolling = requestMemory(amountToRequest) + unrollMemoryMap.synchronized { + if (!requestMemory(amountToRequest)) { + // If the first request is not granted, try again after ensuring free space + // If there is still not enough space, give up and drop the partition + val extraSpaceNeeded = globalUnrollMemory - unrollMemoryMap.values.sum + val result = ensureFreeSpace(blockId, extraSpaceNeeded) + droppedBlocks ++= result.droppedBlocks + keepUnrolling = requestMemory(amountToRequest) + } + memoryThreshold = amountToRequest } - memoryThreshold = amountToRequest } } } @@ -357,7 +369,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) * from the same RDD (which leads to a wasteful cyclic replacement pattern for RDDs that * don't fit into memory that we want to avoid). * - * Assume that a lock is held by the caller to ensure only one thread is dropping blocks. + * Assume that `putLock` is held by the caller to ensure only one thread is dropping blocks. * Otherwise, the freed space may fill up before the caller puts in their new value. * * Return whether there is enough free space, along with the blocks dropped in the process. @@ -375,7 +387,6 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) } // Take into account the amount of memory currently occupied by unrolling blocks - val unrollMemoryMap = SparkEnv.get.unrollMemoryMap val freeSpace = unrollMemoryMap.synchronized { freeMemory - unrollMemoryMap.values.sum } if (freeSpace < space) { From a49ba4d9e01f9313aa36920b6f8a0ba28a5d96e8 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 24 Jul 2014 12:17:24 -0700 Subject: [PATCH 40/55] Do not expose unroll memory check period --- .../src/main/scala/org/apache/spark/storage/MemoryStore.scala | 4 ++-- .../scala/org/apache/spark/storage/BlockManagerSuite.scala | 1 - 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index deab798ee08d2..cac7dcfbde0bc 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -221,8 +221,8 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) var keepUnrolling = true // Initial per-thread memory to request for unrolling blocks (bytes). Exposed for testing. val initialMemoryThreshold = conf.getLong("spark.storage.unrollMemoryThreshold", 1024 * 1024) - // How often to check whether we need to request more memory. Exposed for testing. - val memoryCheckPeriod = conf.getLong("spark.storage.unrollCheckPeriod", 16) + // How often to check whether we need to request more memory + val memoryCheckPeriod = 16 // Memory currently reserved by this thread (bytes) var memoryThreshold = initialMemoryThreshold // Memory to request as a multiple of current vector size diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 7f9878fbf669c..1e8dd83e6d803 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -76,7 +76,6 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter conf.set("spark.driver.port", boundPort.toString) conf.set("spark.storage.unrollFraction", "0.4") conf.set("spark.storage.unrollMemoryThreshold", "512") - conf.set("spark.storage.unrollCheckPeriod", "1") SparkEnv.set(env) master = new BlockManagerMaster( From 8448c9bb6823f228adf05c75b20364b4c47b9770 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 24 Jul 2014 12:56:07 -0700 Subject: [PATCH 41/55] Fix tests --- .../spark/storage/BlockManagerSuite.scala | 34 +++++++++++++------ 1 file changed, 23 insertions(+), 11 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 1e8dd83e6d803..9524c4d69d2cd 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -107,6 +107,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } System.clearProperty("spark.test.useCompressedOops") + + // This is normally cleared when a task ends, but since we do not + // mock the entire job here, we have to manually clear this ourselves + SparkEnv.get.unrollMemoryMap.clear() } test("StorageLevel object caching") { @@ -905,7 +909,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter case _ => fail("Updated block is neither list1 nor list3") } } - assert(store.get("list3").isDefined, "list3 was not in store") + assert(store.memoryStore.contains("list3"), "list3 was not in memory store") // 2 updated blocks - list2 is kicked out of memory (but put on disk) while list4 is added val updatedBlocks4 = @@ -918,19 +922,27 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter case _ => fail("Updated block is neither list2 nor list4") } } - assert(store.get("list4").isDefined, "list4 was not in store") + assert(store.diskStore.contains("list2"), "list2 was not in disk store") + assert(store.memoryStore.contains("list4"), "list4 was not in memory store") - // 1 updated block - list5 is too big to fit in store, but list3 is kicked out in the process + // 0 updated blocks - list5 is too big to fit in store val updatedBlocks5 = store.putIterator("list5", bigList.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - assert(updatedBlocks5.size === 1) - assert(updatedBlocks5.head._1 === TestBlockId("list3")) - assert(updatedBlocks5.head._2.storageLevel === StorageLevel.NONE) - assert(!store.get("list1").isDefined, "list1 was in store") - assert(store.get("list2").isDefined, "list2 was not in store") - assert(!store.get("list3").isDefined, "list3 was in store") - assert(store.get("list4").isDefined, "list4 was not in store") - assert(!store.get("list5").isDefined, "list5 was in store") + assert(updatedBlocks5.size === 0) + + // memory store contains only list3 and list4 + assert(!store.memoryStore.contains("list1"), "list1 was in memory store") + assert(!store.memoryStore.contains("list2"), "list2 was in memory store") + assert(store.memoryStore.contains("list3"), "list3 was not in memory store") + assert(store.memoryStore.contains("list4"), "list4 was not in memory store") + assert(!store.memoryStore.contains("list5"), "list5 was in memory store") + + // disk store contains only list2 + assert(!store.diskStore.contains("list1"), "list1 was in disk store") + assert(store.diskStore.contains("list2"), "list2 was not in disk store") + assert(!store.diskStore.contains("list3"), "list3 was in disk store") + assert(!store.diskStore.contains("list4"), "list4 was in disk store") + assert(!store.diskStore.contains("list5"), "list5 was in disk store") } test("query block statuses") { From f9ff82ed8d96e761e2f49e64037bae844b25a845 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 24 Jul 2014 13:00:29 -0700 Subject: [PATCH 42/55] putValues -> putIterator + putArray --- .../org/apache/spark/storage/BlockManager.scala | 10 +++++----- .../org/apache/spark/storage/BlockStore.scala | 4 ++-- .../scala/org/apache/spark/storage/DiskStore.scala | 6 +++--- .../org/apache/spark/storage/MemoryStore.scala | 14 +++++++------- .../org/apache/spark/storage/TachyonStore.scala | 6 +++--- 5 files changed, 20 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 690c8f32a705e..d746526639e58 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -463,10 +463,10 @@ private[spark] class BlockManager( val values = dataDeserialize(blockId, bytes) if (level.deserialized) { // Cache the values before returning them - val putResult = memoryStore.putValues( + val putResult = memoryStore.putIterator( blockId, values, level, returnValues = true, allowPersistToDisk = false) // The put may or may not have succeeded, depending on whether there was enough - // space to unroll the block. Either way, putValues should return an iterator. + // space to unroll the block. Either way, the put here should return an iterator. putResult.data match { case Left(it) => return Some(new BlockResult(it, DataReadMethod.Disk, info.size)) @@ -716,9 +716,9 @@ private[spark] class BlockManager( // Actually put the values val result = data match { case IteratorValues(iterator) => - blockStore.putValues(blockId, iterator, putLevel, returnValues) + blockStore.putIterator(blockId, iterator, putLevel, returnValues) case ArrayValues(array) => - blockStore.putValues(blockId, array, putLevel, returnValues) + blockStore.putArray(blockId, array, putLevel, returnValues) case ByteBufferValues(bytes) => bytes.rewind() blockStore.putBytes(blockId, bytes, putLevel) @@ -873,7 +873,7 @@ private[spark] class BlockManager( logInfo(s"Writing block $blockId to disk") data match { case Left(elements) => - diskStore.putValues(blockId, elements, level, returnValues = false) + diskStore.putArray(blockId, elements, level, returnValues = false) case Right(bytes) => diskStore.putBytes(blockId, bytes, level) } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala index 68fd2b2156ee7..69985c9759e2d 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala @@ -37,13 +37,13 @@ private[spark] abstract class BlockStore(val blockManager: BlockManager) extends * @return a PutResult that contains the size of the data, as well as the values put if * returnValues is true (if not, the result's data field can be null) */ - def putValues( + def putIterator( blockId: BlockId, values: Iterator[Any], level: StorageLevel, returnValues: Boolean): PutResult - def putValues( + def putArray( blockId: BlockId, values: Array[Any], level: StorageLevel, diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index 422179b44b189..c83261dd91b36 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -55,15 +55,15 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc PutResult(bytes.limit(), Right(bytes.duplicate())) } - override def putValues( + override def putArray( blockId: BlockId, values: Array[Any], level: StorageLevel, returnValues: Boolean): PutResult = { - putValues(blockId, values.toIterator, level, returnValues) + putIterator(blockId, values.toIterator, level, returnValues) } - override def putValues( + override def putIterator( blockId: BlockId, values: Iterator[Any], level: StorageLevel, diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index cac7dcfbde0bc..e11504fb4018e 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -78,14 +78,14 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) bytes.rewind() if (level.deserialized) { val values = blockManager.dataDeserialize(blockId, bytes) - putValues(blockId, values, level, returnValues = true) + putIterator(blockId, values, level, returnValues = true) } else { val putAttempt = tryToPut(blockId, bytes, bytes.limit, deserialized = false) PutResult(bytes.limit(), Right(bytes.duplicate()), putAttempt.droppedBlocks) } } - override def putValues( + override def putArray( blockId: BlockId, values: Array[Any], level: StorageLevel, @@ -101,12 +101,12 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) } } - override def putValues( + override def putIterator( blockId: BlockId, values: Iterator[Any], level: StorageLevel, returnValues: Boolean): PutResult = { - putValues(blockId, values, level, returnValues, allowPersistToDisk = true) + putIterator(blockId, values, level, returnValues, allowPersistToDisk = true) } /** @@ -121,7 +121,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) * back from disk and attempts to cache it in memory. In this case, we should not persist the * block back on disk again, as it is already in disk store. */ - private[storage] def putValues( + private[storage] def putIterator( blockId: BlockId, values: Iterator[Any], level: StorageLevel, @@ -132,7 +132,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) unrolledValues match { case Left(arrayValues) => // Values are fully unrolled in memory, so store them as an array - val res = putValues(blockId, arrayValues, level, returnValues) + val res = putArray(blockId, arrayValues, level, returnValues) droppedBlocks ++= res.droppedBlocks PutResult(res.size, res.data, droppedBlocks) case Right(iteratorValues) => @@ -141,7 +141,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) s"Free memory is $freeMemory bytes.") if (level.useDisk && allowPersistToDisk) { logWarning(s"Persisting block $blockId to disk instead.") - val res = blockManager.diskStore.putValues(blockId, iteratorValues, level, returnValues) + val res = blockManager.diskStore.putIterator(blockId, iteratorValues, level, returnValues) PutResult(res.size, res.data, droppedBlocks) } else { PutResult(0, Left(iteratorValues), droppedBlocks) diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala index c843a50b70304..932b5616043b4 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala @@ -43,15 +43,15 @@ private[spark] class TachyonStore( putIntoTachyonStore(blockId, bytes, returnValues = true) } - override def putValues( + override def putArray( blockId: BlockId, values: Array[Any], level: StorageLevel, returnValues: Boolean): PutResult = { - putValues(blockId, values.toIterator, level, returnValues) + putIterator(blockId, values.toIterator, level, returnValues) } - override def putValues( + override def putIterator( blockId: BlockId, values: Iterator[Any], level: StorageLevel, From ed6cda421fe05685297720a404c85c94d4f8811b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 24 Jul 2014 13:02:21 -0700 Subject: [PATCH 43/55] Formatting fix (super minor) --- .../src/main/scala/org/apache/spark/storage/MemoryStore.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index e11504fb4018e..f115b8e106c32 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -40,8 +40,8 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) @volatile private var currentMemory = 0L - // Object used to ensure that only one thread is putting blocks and if necessary, - // dropping blocks from the memory store. + // Object used to ensure that only one thread is putting blocks and if necessary, dropping + // blocks from the memory store. private val putLock = new Object() /** From b9a6eee0a7fd5acdf3af647797416b337ef4fa11 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 24 Jul 2014 15:37:41 -0700 Subject: [PATCH 44/55] Simplify locking behavior on unrollMemoryMap This gets rid of the need to synchronize on unrollMemoryMap in addition to putLock. Now we require all accesses on unrollMemoryMap to synchronize on putLock, including when the executor releases the unroll memory after a task ends. --- .../scala/org/apache/spark/SparkEnv.scala | 4 - .../org/apache/spark/executor/Executor.scala | 10 ++- .../apache/spark/storage/MemoryStore.scala | 87 +++++++++++-------- 3 files changed, 57 insertions(+), 44 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index f4f26cadcc738..ae9147e499710 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -71,10 +71,6 @@ class SparkEnv ( // All accesses should be manually synchronized val shuffleMemoryMap = mutable.HashMap[Long, Long]() - // A mapping of thread ID to amount of memory, in bytes, used for unrolling a block - // All accesses should be manually synchronized - val unrollMemoryMap = mutable.HashMap[Long, Long]() - private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() // A general, soft-reference map for metadata needed during HadoopRDD split computation diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 5e748c100d680..bd9d2cfb034be 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -266,11 +266,13 @@ private[spark] class Executor( } } } finally { - val threadId = Thread.currentThread().getId + // Release memory used by this thread for shuffles val shuffleMemoryMap = env.shuffleMemoryMap - val unrollMemoryMap = env.unrollMemoryMap - shuffleMemoryMap.synchronized { shuffleMemoryMap.remove(threadId) } - unrollMemoryMap.synchronized { unrollMemoryMap.remove(threadId) } + shuffleMemoryMap.synchronized { + shuffleMemoryMap.remove(Thread.currentThread().getId) + } + // Release memory used by this thread for unrolling blocks + env.blockManager.memoryStore.releaseUnrollMemory() runningTasks.remove(taskId) } } diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index f115b8e106c32..d9da1478be71c 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -20,9 +20,9 @@ package org.apache.spark.storage import java.nio.ByteBuffer import java.util.LinkedHashMap +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.apache.spark.SparkEnv import org.apache.spark.util.{SizeEstimator, Utils} import org.apache.spark.util.collection.SizeTrackingVector @@ -44,20 +44,15 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) // blocks from the memory store. private val putLock = new Object() - /** - * Mapping from thread ID to memory used for unrolling blocks. - * - * To avoid potential deadlocks, all accesses of this map in MemoryStore are assumed to - * first synchronize on `putLock` and then on `unrollMemoryMap`, in that particular order. - * This is lazy because SparkEnv does not exist when we mock this class in tests. - */ - private lazy val unrollMemoryMap = SparkEnv.get.unrollMemoryMap + // A mapping from thread ID to amount of memory used for unrolling a block (in bytes) + // All accesses of this map are assumed to have manually synchronized on `putLock` + private val unrollMemoryMap = mutable.HashMap[Long, Long]() /** * The amount of space ensured for unrolling values in memory, shared across all cores. * This space is not reserved in advance, but allocated dynamically by dropping existing blocks. */ - private val globalUnrollMemory = { + private val maxUnrollMemory: Long = { val unrollFraction = conf.getDouble("spark.storage.unrollFraction", 0.2) (maxMemory * unrollFraction).toLong } @@ -227,22 +222,18 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) var memoryThreshold = initialMemoryThreshold // Memory to request as a multiple of current vector size val memoryGrowthFactor = 1.5 - - val threadId = Thread.currentThread().getId + // Underlying vector for unrolling the block var vector = new SizeTrackingVector[Any] - // Request memory for our vector and return whether the request is granted. This involves - // synchronizing on putLock and unrollMemoryMap (in that order), which could be expensive. + // Request memory for our vector and return whether the request is granted + // This involves synchronizing across all threads, which is expensive if called frequently def requestMemory(memoryToRequest: Long): Boolean = { putLock.synchronized { - unrollMemoryMap.synchronized { - val previouslyOccupiedMemory = unrollMemoryMap.get(threadId).getOrElse(0L) - val otherThreadsMemory = unrollMemoryMap.values.sum - previouslyOccupiedMemory - val availableMemory = freeMemory - otherThreadsMemory - val granted = availableMemory > memoryToRequest - if (granted) { unrollMemoryMap(threadId) = memoryToRequest } - granted - } + val otherThreadsMemory = currentUnrollMemory - threadCurrentUnrollMemory + val availableMemory = freeMemory - otherThreadsMemory + val granted = availableMemory > memoryToRequest + if (granted) { reserveUnrollMemory(memoryToRequest) } + granted } } @@ -261,17 +252,15 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) // Hold the put lock, in case another thread concurrently puts a block that takes // up the unrolling space we just ensured here putLock.synchronized { - unrollMemoryMap.synchronized { - if (!requestMemory(amountToRequest)) { - // If the first request is not granted, try again after ensuring free space - // If there is still not enough space, give up and drop the partition - val extraSpaceNeeded = globalUnrollMemory - unrollMemoryMap.values.sum - val result = ensureFreeSpace(blockId, extraSpaceNeeded) - droppedBlocks ++= result.droppedBlocks - keepUnrolling = requestMemory(amountToRequest) - } - memoryThreshold = amountToRequest + if (!requestMemory(amountToRequest)) { + // If the first request is not granted, try again after ensuring free space + // If there is still not enough space, give up and drop the partition + val extraSpaceNeeded = maxUnrollMemory - currentUnrollMemory + val result = ensureFreeSpace(blockId, extraSpaceNeeded) + droppedBlocks ++= result.droppedBlocks + keepUnrolling = requestMemory(amountToRequest) } + memoryThreshold = amountToRequest } } } @@ -292,9 +281,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) // we release the memory claimed by this thread later on when the task finishes. if (keepUnrolling) { vector = null - unrollMemoryMap.synchronized { - unrollMemoryMap(threadId) = 0 - } + releaseUnrollMemory() } } } @@ -387,7 +374,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) } // Take into account the amount of memory currently occupied by unrolling blocks - val freeSpace = unrollMemoryMap.synchronized { freeMemory - unrollMemoryMap.values.sum } + val freeSpace = freeMemory - currentUnrollMemory if (freeSpace < space) { val rddToAdd = getRddId(blockIdToAdd) @@ -439,6 +426,34 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) override def contains(blockId: BlockId): Boolean = { entries.synchronized { entries.containsKey(blockId) } } + + /** + * Reserve memory for unrolling blocks used by this thread. + */ + private def reserveUnrollMemory(memory: Long): Unit = putLock.synchronized { + unrollMemoryMap(Thread.currentThread().getId) = memory + } + + /** + * Release memory used by this thread for unrolling blocks. + */ + private[spark] def releaseUnrollMemory(): Unit = putLock.synchronized { + unrollMemoryMap.remove(Thread.currentThread().getId) + } + + /** + * Return the amount of memory currently occupied for unrolling blocks across all threads. + */ + private def currentUnrollMemory: Long = putLock.synchronized { + unrollMemoryMap.values.sum + } + + /** + * Return the amount of memory currently occupied for unrolling blocks by this thread. + */ + private def threadCurrentUnrollMemory: Long = putLock.synchronized { + unrollMemoryMap.getOrElse(Thread.currentThread().getId, 0L) + } } private[spark] case class ResultWithDroppedBlocks( From ff77aa1307edc584f68845a2a5c8bd2fc300e70e Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 24 Jul 2014 18:40:43 -0700 Subject: [PATCH 45/55] Fix tests --- core/src/main/scala/org/apache/spark/SparkEnv.scala | 8 -------- .../org/apache/spark/storage/BlockManagerSuite.scala | 6 ------ 2 files changed, 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index ae9147e499710..6ee731b22c03c 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -134,14 +134,6 @@ object SparkEnv extends Logging { env.get() } - /** - * Create an empty SparkEnv. For testing only. - */ - private[spark] def empty(conf: SparkConf): SparkEnv = { - new SparkEnv( - "test", null, null, null, null, null, null, null, null, null, null, null, null, null, conf) - } - private[spark] def create( conf: SparkConf, executorId: String, diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 9524c4d69d2cd..de29ae8f970a9 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -45,7 +45,6 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter with PrivateMethodTester { private val conf = new SparkConf(false) - val env: SparkEnv = SparkEnv.empty(conf) var store: BlockManager = null var store2: BlockManager = null var actorSystem: ActorSystem = null @@ -76,7 +75,6 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter conf.set("spark.driver.port", boundPort.toString) conf.set("spark.storage.unrollFraction", "0.4") conf.set("spark.storage.unrollMemoryThreshold", "512") - SparkEnv.set(env) master = new BlockManagerMaster( actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), @@ -107,10 +105,6 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } System.clearProperty("spark.test.useCompressedOops") - - // This is normally cleared when a task ends, but since we do not - // mock the entire job here, we have to manually clear this ourselves - SparkEnv.get.unrollMemoryMap.clear() } test("StorageLevel object caching") { From e40c60d4082157443afa19da97bdc9a301e555af Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 24 Jul 2014 21:51:18 -0700 Subject: [PATCH 46/55] Fix MIMA excludes --- project/MimaExcludes.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 9598e453f5798..5ff88f0dd1cac 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -62,12 +62,12 @@ object MimaExcludes { "org.apache.spark.storage.MemoryStore.Entry") ) ++ Seq( - // BlockManager now uses Arrays instead of ArrayBuffers - ProblemFilters.exclude[IncompatibleMethTypeProblem]( + // Renamed putValues -> putArray + putIterator + ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.storage.MemoryStore.putValues"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]( + ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.storage.DiskStore.putValues"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]( + ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.storage.TachyonStore.putValues") ) ++ Seq( From 68730b31e9fe26bc59e6698c518b8d4373d4e542 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 25 Jul 2014 10:29:36 -0700 Subject: [PATCH 47/55] Fix weird scalatest behavior --- core/src/test/scala/org/apache/spark/CacheManagerSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index 3aaa10c459bb8..9c5f394d3899d 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -59,7 +59,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar val getValue = blockManager.get(RDDBlockId(rdd.id, split.index)) assert(computeValue.toList === List(1, 2, 3, 4)) assert(getValue.isDefined, "Block cached from getOrCompute is not found!") - assert(getValue.get.data.toArray === List(1, 2, 3, 4)) + assert(getValue.get.data.toList === List(1, 2, 3, 4)) } test("get cached rdd") { From a66fbd2a491d002613ef40248dc290702d3dda02 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 25 Jul 2014 10:36:54 -0700 Subject: [PATCH 48/55] Rename a few things + update comments --- .../org/apache/spark/executor/Executor.scala | 2 +- .../apache/spark/storage/MemoryStore.scala | 31 +++++++++---------- 2 files changed, 16 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index bd9d2cfb034be..3b69bc4ca4142 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -272,7 +272,7 @@ private[spark] class Executor( shuffleMemoryMap.remove(Thread.currentThread().getId) } // Release memory used by this thread for unrolling blocks - env.blockManager.memoryStore.releaseUnrollMemory() + env.blockManager.memoryStore.releaseUnrollMemoryForThisThread() runningTasks.remove(taskId) } } diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index d9da1478be71c..129f1d5ca989e 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -40,12 +40,11 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) @volatile private var currentMemory = 0L - // Object used to ensure that only one thread is putting blocks and if necessary, dropping - // blocks from the memory store. - private val putLock = new Object() + // Ensure only one thread is putting, and if necessary, dropping blocks at any given time + private val accountingLock = new Object // A mapping from thread ID to amount of memory used for unrolling a block (in bytes) - // All accesses of this map are assumed to have manually synchronized on `putLock` + // All accesses of this map are assumed to have manually synchronized on `accountingLock` private val unrollMemoryMap = mutable.HashMap[Long, Long]() /** @@ -228,11 +227,11 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) // Request memory for our vector and return whether the request is granted // This involves synchronizing across all threads, which is expensive if called frequently def requestMemory(memoryToRequest: Long): Boolean = { - putLock.synchronized { - val otherThreadsMemory = currentUnrollMemory - threadCurrentUnrollMemory + accountingLock.synchronized { + val otherThreadsMemory = currentUnrollMemory - currentUnrollMemoryForThisThread val availableMemory = freeMemory - otherThreadsMemory val granted = availableMemory > memoryToRequest - if (granted) { reserveUnrollMemory(memoryToRequest) } + if (granted) { reserveUnrollMemoryForThisThread(memoryToRequest) } granted } } @@ -251,7 +250,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) val amountToRequest = (currentSize * memoryGrowthFactor).toLong // Hold the put lock, in case another thread concurrently puts a block that takes // up the unrolling space we just ensured here - putLock.synchronized { + accountingLock.synchronized { if (!requestMemory(amountToRequest)) { // If the first request is not granted, try again after ensuring free space // If there is still not enough space, give up and drop the partition @@ -281,7 +280,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) // we release the memory claimed by this thread later on when the task finishes. if (keepUnrolling) { vector = null - releaseUnrollMemory() + releaseUnrollMemoryForThisThread() } } } @@ -298,7 +297,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) * an Array if deserialized is true or a ByteBuffer otherwise. Its (possibly estimated) size * must also be passed by the caller. * - * Lock on the object putLock to ensure that all the put requests and its associated block + * Synchronize on `accountingLock` to ensure that all the put requests and its associated block * dropping is done by only on thread at a time. Otherwise while one thread is dropping * blocks to free memory for one block, another thread may use up the freed space for * another block. @@ -320,7 +319,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) var putSuccess = false val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] - putLock.synchronized { + accountingLock.synchronized { val freeSpaceResult = ensureFreeSpace(blockId, size) val enoughFreeSpace = freeSpaceResult.success droppedBlocks ++= freeSpaceResult.droppedBlocks @@ -356,7 +355,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) * from the same RDD (which leads to a wasteful cyclic replacement pattern for RDDs that * don't fit into memory that we want to avoid). * - * Assume that `putLock` is held by the caller to ensure only one thread is dropping blocks. + * Assume that `accountingLock` is held by the caller to ensure only one thread is dropping blocks. * Otherwise, the freed space may fill up before the caller puts in their new value. * * Return whether there is enough free space, along with the blocks dropped in the process. @@ -430,28 +429,28 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) /** * Reserve memory for unrolling blocks used by this thread. */ - private def reserveUnrollMemory(memory: Long): Unit = putLock.synchronized { + private def reserveUnrollMemoryForThisThread(memory: Long): Unit = accountingLock.synchronized { unrollMemoryMap(Thread.currentThread().getId) = memory } /** * Release memory used by this thread for unrolling blocks. */ - private[spark] def releaseUnrollMemory(): Unit = putLock.synchronized { + private[spark] def releaseUnrollMemoryForThisThread(): Unit = accountingLock.synchronized { unrollMemoryMap.remove(Thread.currentThread().getId) } /** * Return the amount of memory currently occupied for unrolling blocks across all threads. */ - private def currentUnrollMemory: Long = putLock.synchronized { + private def currentUnrollMemory: Long = accountingLock.synchronized { unrollMemoryMap.values.sum } /** * Return the amount of memory currently occupied for unrolling blocks by this thread. */ - private def threadCurrentUnrollMemory: Long = putLock.synchronized { + private def currentUnrollMemoryForThisThread: Long = accountingLock.synchronized { unrollMemoryMap.getOrElse(Thread.currentThread().getId, 0L) } } From f4d035c85b2996e0fa1b17d35cb7da75e85bb8a6 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 25 Jul 2014 11:42:42 -0700 Subject: [PATCH 49/55] Allow one thread to unroll multiple blocks This situation could arise if the RDD has multiple dependencies (e.g. zip or join). This commit accounts for this by expressing reserve and release as incremental operations rather than one-time operations. Then, after each unroll, we release only the amount we reserved in this operation, rather than all of this thread's unroll memory. --- .../apache/spark/storage/MemoryStore.scala | 45 +++++++++++++------ .../spark/storage/BlockManagerSuite.scala | 4 +- 2 files changed, 34 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 129f1d5ca989e..acf2258cc8d20 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -217,14 +217,16 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) val initialMemoryThreshold = conf.getLong("spark.storage.unrollMemoryThreshold", 1024 * 1024) // How often to check whether we need to request more memory val memoryCheckPeriod = 16 - // Memory currently reserved by this thread (bytes) + // Memory currently reserved by this thread for this particular unrolling operation var memoryThreshold = initialMemoryThreshold // Memory to request as a multiple of current vector size val memoryGrowthFactor = 1.5 + // Previous unroll memory held by this thread, for releasing later + val previousMemoryReserved = currentUnrollMemoryForThisThread // Underlying vector for unrolling the block var vector = new SizeTrackingVector[Any] - // Request memory for our vector and return whether the request is granted + // Request additional memory for our vector and return whether the request is granted // This involves synchronizing across all threads, which is expensive if called frequently def requestMemory(memoryToRequest: Long): Boolean = { accountingLock.synchronized { @@ -237,7 +239,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) } // Request enough memory to begin unrolling - keepUnrolling = requestMemory(memoryThreshold) + keepUnrolling = requestMemory(initialMemoryThreshold) // Unroll this block safely, checking whether we have exceeded our threshold periodically try { @@ -247,7 +249,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) // If our vector's size has exceeded the threshold, request more memory val currentSize = vector.estimateSize() if (currentSize >= memoryThreshold) { - val amountToRequest = (currentSize * memoryGrowthFactor).toLong + val amountToRequest = (currentSize * (memoryGrowthFactor - 1)).toLong // Hold the put lock, in case another thread concurrently puts a block that takes // up the unrolling space we just ensured here accountingLock.synchronized { @@ -259,8 +261,9 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) droppedBlocks ++= result.droppedBlocks keepUnrolling = requestMemory(amountToRequest) } - memoryThreshold = amountToRequest } + // New threshold is currentSize * memoryGrowthFactor + memoryThreshold = currentSize + amountToRequest } } elementsUnrolled += 1 @@ -280,7 +283,8 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) // we release the memory claimed by this thread later on when the task finishes. if (keepUnrolling) { vector = null - releaseUnrollMemoryForThisThread() + val amountToRelease = currentUnrollMemoryForThisThread - previousMemoryReserved + releaseUnrollMemoryForThisThread(amountToRelease) } } } @@ -355,8 +359,8 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) * from the same RDD (which leads to a wasteful cyclic replacement pattern for RDDs that * don't fit into memory that we want to avoid). * - * Assume that `accountingLock` is held by the caller to ensure only one thread is dropping blocks. - * Otherwise, the freed space may fill up before the caller puts in their new value. + * Assume that `accountingLock` is held by the caller to ensure only one thread is dropping + * blocks. Otherwise, the freed space may fill up before the caller puts in their new value. * * Return whether there is enough free space, along with the blocks dropped in the process. */ @@ -427,17 +431,32 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) } /** - * Reserve memory for unrolling blocks used by this thread. + * Reserve additional memory for unrolling blocks used by this thread. */ - private def reserveUnrollMemoryForThisThread(memory: Long): Unit = accountingLock.synchronized { - unrollMemoryMap(Thread.currentThread().getId) = memory + private def reserveUnrollMemoryForThisThread(memory: Long): Unit = { + val threadId = Thread.currentThread().getId + accountingLock.synchronized { + unrollMemoryMap(threadId) = unrollMemoryMap.getOrElse(threadId, 0L) + memory + } } /** * Release memory used by this thread for unrolling blocks. + * If the amount is not specified, remove the current thread's allocation altogether. */ - private[spark] def releaseUnrollMemoryForThisThread(): Unit = accountingLock.synchronized { - unrollMemoryMap.remove(Thread.currentThread().getId) + private[spark] def releaseUnrollMemoryForThisThread(memory: Long = -1L): Unit = { + val threadId = Thread.currentThread().getId + accountingLock.synchronized { + if (memory < 0) { + unrollMemoryMap.remove(threadId) + } else { + unrollMemoryMap(threadId) = unrollMemoryMap.getOrElse(threadId, memory) - memory + // If this thread claims no more unroll memory, release it completely + if (unrollMemoryMap(threadId) <= 0) { + unrollMemoryMap.remove(threadId) + } + } + } } /** diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index de29ae8f970a9..2154d540a59cf 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -1040,7 +1040,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter */ store.putSingle("a1", new Array[Byte](4000), StorageLevel.MEMORY_ONLY) store.putSingle("a2", new Array[Byte](4000), StorageLevel.MEMORY_ONLY) - store.putSingle("a3", new Array[Byte](3000), StorageLevel.MEMORY_ONLY) + store.putSingle("a3", new Array[Byte](4000), StorageLevel.MEMORY_ONLY) // Memory store should contain a1, a2 assert(store.memoryStore.contains("a1"), "a1 was not in store") assert(store.memoryStore.contains("a2"), "a2 was not in store") @@ -1051,7 +1051,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter * (i.e. a1) from memory to accommodate a5 (because 2000 < 2400). */ store.putSingle("a4", new Array[Byte](2000), StorageLevel.MEMORY_ONLY) - store.putSingle("a5", new Array[Byte](3000), StorageLevel.MEMORY_ONLY) + store.putSingle("a5", new Array[Byte](4000), StorageLevel.MEMORY_ONLY) // Memory store should contain a2, a4, a5 assert(!store.memoryStore.contains("a1"), "a1 was in store") assert(store.memoryStore.contains("a2"), "a2 was not in store") From 369ad07a48752b92d7c4471a46cec5c83413ac60 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 25 Jul 2014 15:12:17 -0700 Subject: [PATCH 50/55] Correct ensureFreeSpace and requestMemory behavior This fixes three bugs from the previous commit: 1. requestMemory compared the total available memory with the amount we are requesting. However, the latter is the delta (i.e. how much _extra_ we want), not the new allocation (how much we want). We need to compare apples to apples. 2. We occasionally ensured a negative amount of free space. This happens whenever we use more than what spark.storage.unrollBuffer allows us to use, and we need to ensure free space. We need to make sure we only call ensureFreeSpace with a positive amount. 3. ensureFreeSpace never actually took into account the space used by unrolling. This is fixed in this commit. --- .../apache/spark/storage/MemoryStore.scala | 21 ++++++++++--------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index acf2258cc8d20..5bf64256e86a6 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -58,6 +58,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) logInfo("MemoryStore started with capacity %s".format(Utils.bytesToString(maxMemory))) + /** Free memory not occupied by existing blocks. Note that this does not include unroll memory. */ def freeMemory: Long = maxMemory - currentMemory override def getSize(blockId: BlockId): Long = { @@ -230,9 +231,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) // This involves synchronizing across all threads, which is expensive if called frequently def requestMemory(memoryToRequest: Long): Boolean = { accountingLock.synchronized { - val otherThreadsMemory = currentUnrollMemory - currentUnrollMemoryForThisThread - val availableMemory = freeMemory - otherThreadsMemory - val granted = availableMemory > memoryToRequest + val granted = freeMemory > currentUnrollMemory + memoryToRequest if (granted) { reserveUnrollMemoryForThisThread(memoryToRequest) } granted } @@ -256,9 +255,11 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) if (!requestMemory(amountToRequest)) { // If the first request is not granted, try again after ensuring free space // If there is still not enough space, give up and drop the partition - val extraSpaceNeeded = maxUnrollMemory - currentUnrollMemory - val result = ensureFreeSpace(blockId, extraSpaceNeeded) - droppedBlocks ++= result.droppedBlocks + val spaceToEnsure = maxUnrollMemory - currentUnrollMemory + if (spaceToEnsure > 0) { + val result = ensureFreeSpace(blockId, spaceToEnsure) + droppedBlocks ++= result.droppedBlocks + } keepUnrolling = requestMemory(amountToRequest) } } @@ -377,9 +378,9 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) } // Take into account the amount of memory currently occupied by unrolling blocks - val freeSpace = freeMemory - currentUnrollMemory + val actualFreeMemory = freeMemory - currentUnrollMemory - if (freeSpace < space) { + if (actualFreeMemory < space) { val rddToAdd = getRddId(blockIdToAdd) val selectedBlocks = new ArrayBuffer[BlockId] var selectedMemory = 0L @@ -389,7 +390,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) // can lead to exceptions. entries.synchronized { val iterator = entries.entrySet().iterator() - while (maxMemory - (currentMemory - selectedMemory) < space && iterator.hasNext) { + while (actualFreeMemory + selectedMemory < space && iterator.hasNext) { val pair = iterator.next() val blockId = pair.getKey if (rddToAdd.isEmpty || rddToAdd != getRddId(blockId)) { @@ -399,7 +400,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) } } - if (maxMemory - (currentMemory - selectedMemory) >= space) { + if (actualFreeMemory + selectedMemory >= space) { logInfo(s"${selectedBlocks.size} blocks selected for dropping") for (blockId <- selectedBlocks) { val entry = entries.synchronized { entries.get(blockId) } From 71672a744f02560802a600a3f785d60024851b7d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 25 Jul 2014 15:14:14 -0700 Subject: [PATCH 51/55] Update unrollSafely tests --- .../spark/storage/BlockManagerSuite.scala | 194 +++++++++++++----- 1 file changed, 138 insertions(+), 56 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 2154d540a59cf..ead1efe53648b 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -1028,68 +1028,150 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(!store.memoryStore.contains(rdd(1, 0)), "rdd_1_0 was in store") } + /** + * Verify the result of MemoryStore#unrollSafely is as expected. + */ + private def verifyUnroll( + expected: Iterator[Any], + result: Either[Array[Any], Iterator[Any]], + shouldBeArray: Boolean): Unit = { + val actual: Iterator[Any] = result match { + case Left(arr: Array[Any]) => + assert(shouldBeArray, "expected iterator from unroll!") + arr.iterator + case Right(it: Iterator[Any]) => + assert(!shouldBeArray, "expected array from unroll!") + it + case _ => + fail("unroll returned neither an iterator nor an array...") + } + expected.zip(actual).foreach { case (e, a) => + assert(e === a, "unroll did not return original values!") + } + } + test("safely unroll blocks") { - conf.set("spark.storage.unrollFraction", "0.2") store = new BlockManager( "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) - /* - * After a1 and a2 are cached, we do not have enough room to unroll a3 safely. When we try - * to cache a3, we attempt to ensure free space of only 12000 * 0.2 = 2400 bytes, which is - * not enough to drop a1 or a2, as there is already >= 2400 bytes of free space. Therefore, - * we give up and drop a3 on the spot. - */ - store.putSingle("a1", new Array[Byte](4000), StorageLevel.MEMORY_ONLY) - store.putSingle("a2", new Array[Byte](4000), StorageLevel.MEMORY_ONLY) - store.putSingle("a3", new Array[Byte](4000), StorageLevel.MEMORY_ONLY) - // Memory store should contain a1, a2 - assert(store.memoryStore.contains("a1"), "a1 was not in store") - assert(store.memoryStore.contains("a2"), "a2 was not in store") - assert(!store.memoryStore.contains("a3"), "a3 was in store") - /* - * After a4 is cached, there is less than 2000 bytes of free space left. Now we try to cache - * a5, which is the same size as a3. Ensuring free space of 2400 bytes now drops the LRU block - * (i.e. a1) from memory to accommodate a5 (because 2000 < 2400). - */ - store.putSingle("a4", new Array[Byte](2000), StorageLevel.MEMORY_ONLY) - store.putSingle("a5", new Array[Byte](4000), StorageLevel.MEMORY_ONLY) - // Memory store should contain a2, a4, a5 - assert(!store.memoryStore.contains("a1"), "a1 was in store") - assert(store.memoryStore.contains("a2"), "a2 was not in store") - assert(!store.memoryStore.contains("a3"), "a3 was in store") - assert(store.memoryStore.contains("a4"), "a4 was not in store") - assert(store.memoryStore.contains("a5"), "a5 was not in store") + val smallList = List.fill(40)(new Array[Byte](100)) + val bigList = List.fill(40)(new Array[Byte](1000)) + val memoryStore = store.memoryStore + val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] + + // Unroll with all the space in the world. This should succeed and return an array. + var unrollResult = memoryStore.unrollSafely("unroll", smallList.iterator, droppedBlocks) + verifyUnroll(smallList.iterator, unrollResult, shouldBeArray = true) + + // Unroll with not enough space. This should succeed after kicking out someBlock1. + store.putIterator("someBlock1", smallList.iterator, StorageLevel.MEMORY_ONLY) + store.putIterator("someBlock2", smallList.iterator, StorageLevel.MEMORY_ONLY) + unrollResult = memoryStore.unrollSafely("unroll", smallList.iterator, droppedBlocks) + verifyUnroll(smallList.iterator, unrollResult, shouldBeArray = true) + assert(droppedBlocks.size === 1) + assert(droppedBlocks.head._1 === TestBlockId("someBlock1")) + droppedBlocks.clear() + + // Unroll huge block with not enough space. Even after ensuring free space of 12000 * 0.4 = + // 4800 bytes, there is still not enough room to unroll this block. This returns an iterator. + // In the mean time, however, we kicked out someBlock2 before giving up. + store.putIterator("someBlock3", smallList.iterator, StorageLevel.MEMORY_ONLY) + unrollResult = memoryStore.unrollSafely("unroll", bigList.iterator, droppedBlocks) + verifyUnroll(bigList.iterator, unrollResult, shouldBeArray = false) + assert(droppedBlocks.size === 1) + assert(droppedBlocks.head._1 === TestBlockId("someBlock2")) + droppedBlocks.clear() + } + + test("safely unroll blocks through putIterator") { + store = new BlockManager( + "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + val memOnly = StorageLevel.MEMORY_ONLY + val memoryStore = store.memoryStore + val smallList = List.fill(40)(new Array[Byte](100)) + val bigList = List.fill(40)(new Array[Byte](1000)) + def smallIterator = smallList.iterator.asInstanceOf[Iterator[Any]] + def bigIterator = bigList.iterator.asInstanceOf[Iterator[Any]] + + // Unroll with plenty of space. This should succeed and cache both blocks. + val result1 = memoryStore.putIterator("b1", smallIterator, memOnly, returnValues = true) + val result2 = memoryStore.putIterator("b2", smallIterator, memOnly, returnValues = true) + assert(memoryStore.contains("b1")) + assert(memoryStore.contains("b2")) + assert(result1.size > 0) // unroll was successful + assert(result2.size > 0) + assert(result1.data.isLeft) // unroll did not drop this block to disk + assert(result2.data.isLeft) // unroll did not drop this block to disk + + // Re-put these two blocks so block manager knows about them too. Otherwise, block manager + // would not know how to drop them from memory later. + memoryStore.remove("b1") + memoryStore.remove("b2") + store.putIterator("b1", smallIterator, memOnly) + store.putIterator("b2", smallIterator, memOnly) + + // Unroll with not enough space. This should succeed but kick out b1 in the process. + val result3 = memoryStore.putIterator("b3", smallIterator, memOnly, returnValues = true) + assert(result3.size > 0) + assert(result3.data.isLeft) + assert(!memoryStore.contains("b1")) + assert(memoryStore.contains("b2")) + assert(memoryStore.contains("b3")) + memoryStore.remove("b3") + store.putIterator("b3", smallIterator, memOnly) + + // Unroll huge block with not enough space. This should fail and kick out b2 in the process. + val result4 = memoryStore.putIterator("b4", bigIterator, memOnly, returnValues = true) + assert(result4.size === 0) // unroll was unsuccessful + assert(result4.data.isLeft) + assert(!memoryStore.contains("b1")) + assert(!memoryStore.contains("b2")) + assert(memoryStore.contains("b3")) + assert(!memoryStore.contains("b4")) } - test("safely unroll blocks (disk)") { - conf.set("spark.storage.unrollFraction", "0.2") + /** + * This test is essentially identical to the preceding one, except that it uses MEMORY_AND_DISK. + */ + test("safely unroll blocks through putIterator (disk)") { store = new BlockManager( "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) - /* - * This test is the same as the previous, except it caches each block using MEMORY_AND_DISK. - * The effect is that all dropped blocks now go to disk instead of simply disappear. - */ - store.putSingle("a1", new Array[Byte](4000), StorageLevel.MEMORY_AND_DISK) - store.putSingle("a2", new Array[Byte](4000), StorageLevel.MEMORY_AND_DISK) - store.putSingle("a3", new Array[Byte](3000), StorageLevel.MEMORY_AND_DISK) - // Memory store should contain a1, a2; disk store should contain a3 - assert(store.memoryStore.contains("a1"), "a1 was not in memory store") - assert(store.memoryStore.contains("a2"), "a2 was not in memory store") - assert(!store.memoryStore.contains("a3"), "a3 was in memory store") - assert(!store.diskStore.contains("a1"), "a1 was in disk store") - assert(!store.diskStore.contains("a2"), "a2 was in disk store") - assert(store.diskStore.contains("a3"), "a3 was not in disk store") - store.putSingle("a4", new Array[Byte](2000), StorageLevel.MEMORY_AND_DISK) - store.putSingle("a5", new Array[Byte](3000), StorageLevel.MEMORY_AND_DISK) - // Memory store should contain a2, a4, a5; disk store should contain a1, a3 - assert(!store.memoryStore.contains("a1"), "a1 was in memory store") - assert(store.memoryStore.contains("a2"), "a2 was not in memory store") - assert(!store.memoryStore.contains("a3"), "a3 was in memory store") - assert(store.memoryStore.contains("a4"), "a4 was not in memory store") - assert(store.memoryStore.contains("a5"), "a5 was not in memory store") - assert(store.diskStore.contains("a1"), "a1 was not in disk store") - assert(!store.diskStore.contains("a2"), "a2 was in disk store") - assert(store.diskStore.contains("a3"), "a3 was not in disk store") - assert(!store.diskStore.contains("a4"), "a4 was in disk store") - assert(!store.diskStore.contains("a5"), "a5 was in disk store") + val memAndDisk = StorageLevel.MEMORY_AND_DISK + val memoryStore = store.memoryStore + val diskStore = store.diskStore + val smallList = List.fill(40)(new Array[Byte](100)) + val bigList = List.fill(40)(new Array[Byte](1000)) + def smallIterator = smallList.iterator.asInstanceOf[Iterator[Any]] + def bigIterator = bigList.iterator.asInstanceOf[Iterator[Any]] + + store.putIterator("b1", smallIterator, memAndDisk) + store.putIterator("b2", smallIterator, memAndDisk) + + // Unroll with not enough space. This should succeed but kick out b1 in the process. + // Memory store should contain b2 and b3, while disk store should contain only b1 + val result3 = memoryStore.putIterator("b3", smallIterator, memAndDisk, returnValues = true) + assert(result3.size > 0) + assert(!memoryStore.contains("b1")) + assert(memoryStore.contains("b2")) + assert(memoryStore.contains("b3")) + assert(diskStore.contains("b1")) + assert(!diskStore.contains("b2")) + assert(!diskStore.contains("b3")) + memoryStore.remove("b3") + store.putIterator("b3", smallIterator, StorageLevel.MEMORY_ONLY) + + // Unroll huge block with not enough space. This should fail and drop the new block to disk + // directly in addition to kicking out b2 in the process. Memory store should contain only + // b3, while disk store should contain b1, b2 and b4. + val result4 = memoryStore.putIterator("b4", bigIterator, memAndDisk, returnValues = true) + assert(result4.size > 0) + assert(result4.data.isRight) // unroll returned bytes from disk + assert(!memoryStore.contains("b1")) + assert(!memoryStore.contains("b2")) + assert(memoryStore.contains("b3")) + assert(!memoryStore.contains("b4")) + assert(diskStore.contains("b1")) + assert(diskStore.contains("b2")) + assert(!diskStore.contains("b3")) + assert(diskStore.contains("b4")) } } From f12916d0933924d426347951a59b031eaccf9381 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 25 Jul 2014 15:24:14 -0700 Subject: [PATCH 52/55] Slightly clean up tests --- .../spark/storage/BlockManagerSuite.scala | 119 +++++++----------- 1 file changed, 43 insertions(+), 76 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index ead1efe53648b..8dfbddfd0a65f 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -62,6 +62,11 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value) def rdd(rddId: Int, splitId: Int) = RDDBlockId(rddId, splitId) + private def makeBlockManager(maxMem: Long, name: String = ""): BlockManager = { + new BlockManager( + name, actorSystem, master, serializer, maxMem, conf, securityMgr, mapOutputTracker) + } + before { val (actorSystem, boundPort) = AkkaUtils.createActorSystem( "test", "localhost", 0, conf = conf, securityManager = securityMgr) @@ -142,8 +147,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("master + 1 manager interaction") { - store = new BlockManager("", actorSystem, master, serializer, 20000, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(20000) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -173,10 +177,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("master + 2 managers interaction") { - store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer(conf), 2000, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(2000, "exec1") + store2 = makeBlockManager(2000, "exec2") val peers = master.getPeers(store.blockManagerId, 1) assert(peers.size === 1, "master did not return the other manager as a peer") @@ -191,8 +193,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("removing block") { - store = new BlockManager("", actorSystem, master, serializer, 20000, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(20000) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -240,8 +241,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("removing rdd") { - store = new BlockManager("", actorSystem, master, serializer, 20000, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(20000) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -274,11 +274,9 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("removing broadcast") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(2000) val driverStore = store - val executorStore = new BlockManager("executor", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) + val executorStore = makeBlockManager(2000, "executor") val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -347,8 +345,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter test("reregistration on heart beat") { val heartBeat = PrivateMethod[Unit]('heartBeat) - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(2000) val a1 = new Array[Byte](400) store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) @@ -384,8 +381,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter test("reregistration doesn't dead lock") { val heartBeat = PrivateMethod[Unit]('heartBeat) - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(2000) val a1 = new Array[Byte](400) val a2 = List(new Array[Byte](400)) @@ -422,8 +418,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("correct BlockResult returned from get() calls") { - store = new BlockManager("", actorSystem, master, serializer, 12000, conf, securityMgr, - mapOutputTracker) + store = makeBlockManager(12000) val list1 = List(new Array[Byte](2000), new Array[Byte](2000)) val list2 = List(new Array[Byte](500), new Array[Byte](1000), new Array[Byte](1500)) val list1SizeEstimate = SizeEstimator.estimate(list1.iterator.toArray) @@ -451,8 +446,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("in-memory LRU storage") { - store = new BlockManager( - "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + store = makeBlockManager(12000) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -471,8 +465,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("in-memory LRU storage with serialization") { - store = new BlockManager( - "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + store = makeBlockManager(12000) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -491,8 +484,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("in-memory LRU for partitions of same RDD") { - store = new BlockManager( - "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + store = makeBlockManager(12000) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -511,8 +503,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("in-memory LRU for partitions of multiple RDDs") { - store = new BlockManager( - "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + store = makeBlockManager(12000) store.putSingle(rdd(0, 1), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(0, 2), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(1, 1), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) @@ -538,8 +529,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter // TODO Make the spark.test.tachyon.enable true after using tachyon 0.5.0 testing jar. val tachyonUnitTestEnabled = conf.getBoolean("spark.test.tachyon.enable", false) if (tachyonUnitTestEnabled) { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(1200) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -555,8 +545,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("on-disk storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(1200) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -569,8 +558,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("disk and memory storage") { - store = new BlockManager("", actorSystem, master, serializer, 12000, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(12000) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -585,8 +573,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("disk and memory storage with getLocalBytes") { - store = new BlockManager( - "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + store = makeBlockManager(12000) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -601,8 +588,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("disk and memory storage with serialization") { - store = new BlockManager( - "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + store = makeBlockManager(12000) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -617,8 +603,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("disk and memory storage with serialization and getLocalBytes") { - store = new BlockManager( - "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + store = makeBlockManager(12000) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -633,8 +618,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("LRU with mixed storage levels") { - store = new BlockManager( - "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + store = makeBlockManager(12000) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -656,8 +640,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("in-memory LRU with streams") { - store = new BlockManager( - "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + store = makeBlockManager(12000) val list1 = List(new Array[Byte](2000), new Array[Byte](2000)) val list2 = List(new Array[Byte](2000), new Array[Byte](2000)) val list3 = List(new Array[Byte](2000), new Array[Byte](2000)) @@ -681,8 +664,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("LRU with mixed storage levels and streams") { - store = new BlockManager( - "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + store = makeBlockManager(12000) val list1 = List(new Array[Byte](2000), new Array[Byte](2000)) val list2 = List(new Array[Byte](2000), new Array[Byte](2000)) val list3 = List(new Array[Byte](2000), new Array[Byte](2000)) @@ -731,8 +713,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("overly large block") { - store = new BlockManager( - "", actorSystem, master, serializer, 5000, conf, securityMgr, mapOutputTracker) + store = makeBlockManager(5000) store.putSingle("a1", new Array[Byte](10000), StorageLevel.MEMORY_ONLY) assert(store.getSingle("a1") === None, "a1 was in store") store.putSingle("a2", new Array[Byte](10000), StorageLevel.MEMORY_AND_DISK) @@ -743,8 +724,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter test("block compression") { try { conf.set("spark.shuffle.compress", "true") - store = new BlockManager( - "exec1", actorSystem, master, serializer, 20000, conf, securityMgr, mapOutputTracker) + store = makeBlockManager(20000, "exec1") store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) <= 100, "shuffle_0_0_0 was not compressed") @@ -752,8 +732,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter store = null conf.set("spark.shuffle.compress", "false") - store = new BlockManager( - "exec2", actorSystem, master, serializer, 20000, conf, securityMgr, mapOutputTracker) + store = makeBlockManager(20000, "exec2") store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) >= 10000, "shuffle_0_0_0 was compressed") @@ -761,8 +740,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter store = null conf.set("spark.broadcast.compress", "true") - store = new BlockManager( - "exec3", actorSystem, master, serializer, 20000, conf, securityMgr, mapOutputTracker) + store = makeBlockManager(20000, "exec3") store.putSingle(BroadcastBlockId(0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(BroadcastBlockId(0)) <= 1000, "broadcast_0 was not compressed") @@ -770,32 +748,28 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter store = null conf.set("spark.broadcast.compress", "false") - store = new BlockManager( - "exec4", actorSystem, master, serializer, 20000, conf, securityMgr, mapOutputTracker) + store = makeBlockManager(20000, "exec4") store.putSingle(BroadcastBlockId(0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(BroadcastBlockId(0)) >= 10000, "broadcast_0 was compressed") store.stop() store = null conf.set("spark.rdd.compress", "true") - store = new BlockManager( - "exec5", actorSystem, master, serializer, 20000, conf, securityMgr, mapOutputTracker) + store = makeBlockManager(20000, "exec5") store.putSingle(rdd(0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(rdd(0, 0)) <= 1000, "rdd_0_0 was not compressed") store.stop() store = null conf.set("spark.rdd.compress", "false") - store = new BlockManager( - "exec6", actorSystem, master, serializer, 20000, conf, securityMgr, mapOutputTracker) + store = makeBlockManager(20000, "exec6") store.putSingle(rdd(0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(rdd(0, 0)) >= 10000, "rdd_0_0 was compressed") store.stop() store = null // Check that any other block types are also kept uncompressed - store = new BlockManager( - "exec7", actorSystem, master, serializer, 20000, conf, securityMgr, mapOutputTracker) + store = makeBlockManager(20000, "exec7") store.putSingle("other_block", new Array[Byte](10000), StorageLevel.MEMORY_ONLY) assert(store.memoryStore.getSize("other_block") >= 10000, "other_block was compressed") store.stop() @@ -873,8 +847,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("updated block statuses") { - store = new BlockManager( - "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + store = makeBlockManager(12000) val list = List.fill(2)(new Array[Byte](2000)) val bigList = List.fill(8)(new Array[Byte](2000)) @@ -940,8 +913,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("query block statuses") { - store = new BlockManager( - "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + store = makeBlockManager(12000) val list = List.fill(2)(new Array[Byte](2000)) // Tell master. By LRU, only list2 and list3 remains. @@ -980,8 +952,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("get matching blocks") { - store = new BlockManager( - "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + store = makeBlockManager(12000) val list = List.fill(2)(new Array[Byte](100)) // insert some blocks @@ -1014,8 +985,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("SPARK-1194 regression: fix the same-RDD rule for cache replacement") { - store = new BlockManager( - "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + store = makeBlockManager(12000) store.putSingle(rdd(0, 0), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(1, 0), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) // Access rdd_1_0 to ensure it's not least recently used. @@ -1051,8 +1021,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("safely unroll blocks") { - store = new BlockManager( - "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + store = makeBlockManager(12000) val smallList = List.fill(40)(new Array[Byte](100)) val bigList = List.fill(40)(new Array[Byte](1000)) val memoryStore = store.memoryStore @@ -1083,8 +1052,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("safely unroll blocks through putIterator") { - store = new BlockManager( - "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + store = makeBlockManager(12000) val memOnly = StorageLevel.MEMORY_ONLY val memoryStore = store.memoryStore val smallList = List.fill(40)(new Array[Byte](100)) @@ -1133,8 +1101,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter * This test is essentially identical to the preceding one, except that it uses MEMORY_AND_DISK. */ test("safely unroll blocks through putIterator (disk)") { - store = new BlockManager( - "", actorSystem, master, serializer, 12000, conf, securityMgr, mapOutputTracker) + store = makeBlockManager(12000) val memAndDisk = StorageLevel.MEMORY_AND_DISK val memoryStore = store.memoryStore val diskStore = store.diskStore From b7e165c3ec3a545661b2dfa6c939b01ab8c1b234 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 25 Jul 2014 17:27:29 -0700 Subject: [PATCH 53/55] Add new tests for unrolling blocks --- .../apache/spark/storage/MemoryStore.scala | 6 +- .../spark/storage/BlockManagerSuite.scala | 77 +++++++++++++++++++ 2 files changed, 80 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 5bf64256e86a6..9ad2b543b4375 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -434,7 +434,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) /** * Reserve additional memory for unrolling blocks used by this thread. */ - private def reserveUnrollMemoryForThisThread(memory: Long): Unit = { + private[spark] def reserveUnrollMemoryForThisThread(memory: Long): Unit = { val threadId = Thread.currentThread().getId accountingLock.synchronized { unrollMemoryMap(threadId) = unrollMemoryMap.getOrElse(threadId, 0L) + memory @@ -463,14 +463,14 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) /** * Return the amount of memory currently occupied for unrolling blocks across all threads. */ - private def currentUnrollMemory: Long = accountingLock.synchronized { + private[spark] def currentUnrollMemory: Long = accountingLock.synchronized { unrollMemoryMap.values.sum } /** * Return the amount of memory currently occupied for unrolling blocks by this thread. */ - private def currentUnrollMemoryForThisThread: Long = accountingLock.synchronized { + private[spark] def currentUnrollMemoryForThisThread: Long = accountingLock.synchronized { unrollMemoryMap.getOrElse(Thread.currentThread().getId, 0L) } } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 8dfbddfd0a65f..6fc28790cf7d1 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -998,6 +998,35 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(!store.memoryStore.contains(rdd(1, 0)), "rdd_1_0 was in store") } + test("reserve/release unroll memory") { + store = makeBlockManager(12000) + val memoryStore = store.memoryStore + assert(memoryStore.currentUnrollMemory === 0) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + // Reserve + memoryStore.reserveUnrollMemoryForThisThread(1480) + assert(memoryStore.currentUnrollMemoryForThisThread === 1480) + memoryStore.reserveUnrollMemoryForThisThread(1000000) + assert(memoryStore.currentUnrollMemoryForThisThread === 1001480) + + // Release + memoryStore.releaseUnrollMemoryForThisThread(1000000) + assert(memoryStore.currentUnrollMemoryForThisThread === 1480) + memoryStore.releaseUnrollMemoryForThisThread(100) + assert(memoryStore.currentUnrollMemoryForThisThread === 1380) + + // Reserve again + memoryStore.reserveUnrollMemoryForThisThread(3620) + assert(memoryStore.currentUnrollMemoryForThisThread === 5000) + + // Release again + memoryStore.releaseUnrollMemoryForThisThread(1000) + assert(memoryStore.currentUnrollMemoryForThisThread === 4000) + memoryStore.releaseUnrollMemoryForThisThread() // release all + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + } + /** * Verify the result of MemoryStore#unrollSafely is as expected. */ @@ -1026,16 +1055,19 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter val bigList = List.fill(40)(new Array[Byte](1000)) val memoryStore = store.memoryStore val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] + assert(memoryStore.currentUnrollMemoryForThisThread === 0) // Unroll with all the space in the world. This should succeed and return an array. var unrollResult = memoryStore.unrollSafely("unroll", smallList.iterator, droppedBlocks) verifyUnroll(smallList.iterator, unrollResult, shouldBeArray = true) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) // Unroll with not enough space. This should succeed after kicking out someBlock1. store.putIterator("someBlock1", smallList.iterator, StorageLevel.MEMORY_ONLY) store.putIterator("someBlock2", smallList.iterator, StorageLevel.MEMORY_ONLY) unrollResult = memoryStore.unrollSafely("unroll", smallList.iterator, droppedBlocks) verifyUnroll(smallList.iterator, unrollResult, shouldBeArray = true) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) assert(droppedBlocks.size === 1) assert(droppedBlocks.head._1 === TestBlockId("someBlock1")) droppedBlocks.clear() @@ -1046,6 +1078,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter store.putIterator("someBlock3", smallList.iterator, StorageLevel.MEMORY_ONLY) unrollResult = memoryStore.unrollSafely("unroll", bigList.iterator, droppedBlocks) verifyUnroll(bigList.iterator, unrollResult, shouldBeArray = false) + assert(memoryStore.currentUnrollMemoryForThisThread > 0) // we returned an iterator assert(droppedBlocks.size === 1) assert(droppedBlocks.head._1 === TestBlockId("someBlock2")) droppedBlocks.clear() @@ -1059,6 +1092,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter val bigList = List.fill(40)(new Array[Byte](1000)) def smallIterator = smallList.iterator.asInstanceOf[Iterator[Any]] def bigIterator = bigList.iterator.asInstanceOf[Iterator[Any]] + assert(memoryStore.currentUnrollMemoryForThisThread === 0) // Unroll with plenty of space. This should succeed and cache both blocks. val result1 = memoryStore.putIterator("b1", smallIterator, memOnly, returnValues = true) @@ -1069,6 +1103,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(result2.size > 0) assert(result1.data.isLeft) // unroll did not drop this block to disk assert(result2.data.isLeft) // unroll did not drop this block to disk + assert(memoryStore.currentUnrollMemoryForThisThread === 0) // Re-put these two blocks so block manager knows about them too. Otherwise, block manager // would not know how to drop them from memory later. @@ -1084,6 +1119,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(!memoryStore.contains("b1")) assert(memoryStore.contains("b2")) assert(memoryStore.contains("b3")) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) memoryStore.remove("b3") store.putIterator("b3", smallIterator, memOnly) @@ -1095,6 +1131,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(!memoryStore.contains("b2")) assert(memoryStore.contains("b3")) assert(!memoryStore.contains("b4")) + assert(memoryStore.currentUnrollMemoryForThisThread > 0) // we returned an iterator } /** @@ -1109,6 +1146,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter val bigList = List.fill(40)(new Array[Byte](1000)) def smallIterator = smallList.iterator.asInstanceOf[Iterator[Any]] def bigIterator = bigList.iterator.asInstanceOf[Iterator[Any]] + assert(memoryStore.currentUnrollMemoryForThisThread === 0) store.putIterator("b1", smallIterator, memAndDisk) store.putIterator("b2", smallIterator, memAndDisk) @@ -1125,6 +1163,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(!diskStore.contains("b3")) memoryStore.remove("b3") store.putIterator("b3", smallIterator, StorageLevel.MEMORY_ONLY) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) // Unroll huge block with not enough space. This should fail and drop the new block to disk // directly in addition to kicking out b2 in the process. Memory store should contain only @@ -1140,5 +1179,43 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(diskStore.contains("b2")) assert(!diskStore.contains("b3")) assert(diskStore.contains("b4")) + assert(memoryStore.currentUnrollMemoryForThisThread > 0) // we returned an iterator + } + + test("multiple unrolls by the same thread") { + store = makeBlockManager(12000) + val memOnly = StorageLevel.MEMORY_ONLY + val memoryStore = store.memoryStore + val smallList = List.fill(40)(new Array[Byte](100)) + def smallIterator = smallList.iterator.asInstanceOf[Iterator[Any]] + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + // All unroll memory used is released because unrollSafely returned an array + memoryStore.putIterator("b1", smallIterator, memOnly, returnValues = true) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + memoryStore.putIterator("b2", smallIterator, memOnly, returnValues = true) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + // Unroll memory is not released because unrollSafely returned an iterator + // that still depends on the underlying vector used in the process + memoryStore.putIterator("b3", smallIterator, memOnly, returnValues = true) + val unrollMemoryAfterB3 = memoryStore.currentUnrollMemoryForThisThread + assert(unrollMemoryAfterB3 > 0) + + // The unroll memory owned by this thread builds on top of its value after the previous unrolls + memoryStore.putIterator("b4", smallIterator, memOnly, returnValues = true) + val unrollMemoryAfterB4 = memoryStore.currentUnrollMemoryForThisThread + assert(unrollMemoryAfterB4 > unrollMemoryAfterB3) + + // ... but only to a certain extent (until we run out of free space to grant new unroll memory) + memoryStore.putIterator("b5", smallIterator, memOnly, returnValues = true) + val unrollMemoryAfterB5 = memoryStore.currentUnrollMemoryForThisThread + memoryStore.putIterator("b6", smallIterator, memOnly, returnValues = true) + val unrollMemoryAfterB6 = memoryStore.currentUnrollMemoryForThisThread + memoryStore.putIterator("b7", smallIterator, memOnly, returnValues = true) + val unrollMemoryAfterB7 = memoryStore.currentUnrollMemoryForThisThread + assert(unrollMemoryAfterB5 === unrollMemoryAfterB4) + assert(unrollMemoryAfterB6 === unrollMemoryAfterB4) + assert(unrollMemoryAfterB7 === unrollMemoryAfterB4) } } From 269d07b11721db9d75d7c25ca7b6e9a2f489e5d5 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sun, 27 Jul 2014 00:40:49 -0700 Subject: [PATCH 54/55] Very minor changes to tests --- .../scala/org/apache/spark/storage/BlockManagerSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 6fc28790cf7d1..ffecbb60b7153 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -31,7 +31,7 @@ import org.scalatest.concurrent.Timeouts._ import org.scalatest.Matchers import org.scalatest.time.SpanSugar._ -import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf, SparkEnv} +import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} import org.apache.spark.executor.DataReadMethod import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} @@ -892,7 +892,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(store.diskStore.contains("list2"), "list2 was not in disk store") assert(store.memoryStore.contains("list4"), "list4 was not in memory store") - // 0 updated blocks - list5 is too big to fit in store + // No updated blocks - list5 is too big to fit in store and nothing is kicked out val updatedBlocks5 = store.putIterator("list5", bigList.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(updatedBlocks5.size === 0) @@ -1102,7 +1102,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(result1.size > 0) // unroll was successful assert(result2.size > 0) assert(result1.data.isLeft) // unroll did not drop this block to disk - assert(result2.data.isLeft) // unroll did not drop this block to disk + assert(result2.data.isLeft) assert(memoryStore.currentUnrollMemoryForThisThread === 0) // Re-put these two blocks so block manager knows about them too. Otherwise, block manager From c7c8832e1015a4028fb61b38c3779a309153244c Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sun, 27 Jul 2014 10:49:28 -0700 Subject: [PATCH 55/55] Simplify logic + update a few comments --- .../apache/spark/storage/MemoryStore.scala | 34 ++++++++----------- .../spark/storage/BlockManagerSuite.scala | 23 +++++++------ 2 files changed, 27 insertions(+), 30 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 9ad2b543b4375..28f675c2bbb1e 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -222,23 +222,13 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) var memoryThreshold = initialMemoryThreshold // Memory to request as a multiple of current vector size val memoryGrowthFactor = 1.5 - // Previous unroll memory held by this thread, for releasing later + // Previous unroll memory held by this thread, for releasing later (only at the very end) val previousMemoryReserved = currentUnrollMemoryForThisThread // Underlying vector for unrolling the block var vector = new SizeTrackingVector[Any] - // Request additional memory for our vector and return whether the request is granted - // This involves synchronizing across all threads, which is expensive if called frequently - def requestMemory(memoryToRequest: Long): Boolean = { - accountingLock.synchronized { - val granted = freeMemory > currentUnrollMemory + memoryToRequest - if (granted) { reserveUnrollMemoryForThisThread(memoryToRequest) } - granted - } - } - // Request enough memory to begin unrolling - keepUnrolling = requestMemory(initialMemoryThreshold) + keepUnrolling = reserveUnrollMemoryForThisThread(initialMemoryThreshold) // Unroll this block safely, checking whether we have exceeded our threshold periodically try { @@ -249,10 +239,10 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) val currentSize = vector.estimateSize() if (currentSize >= memoryThreshold) { val amountToRequest = (currentSize * (memoryGrowthFactor - 1)).toLong - // Hold the put lock, in case another thread concurrently puts a block that takes - // up the unrolling space we just ensured here + // Hold the accounting lock, in case another thread concurrently puts a block that + // takes up the unrolling space we just ensured here accountingLock.synchronized { - if (!requestMemory(amountToRequest)) { + if (!reserveUnrollMemoryForThisThread(amountToRequest)) { // If the first request is not granted, try again after ensuring free space // If there is still not enough space, give up and drop the partition val spaceToEnsure = maxUnrollMemory - currentUnrollMemory @@ -260,7 +250,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) val result = ensureFreeSpace(blockId, spaceToEnsure) droppedBlocks ++= result.droppedBlocks } - keepUnrolling = requestMemory(amountToRequest) + keepUnrolling = reserveUnrollMemoryForThisThread(amountToRequest) } } // New threshold is currentSize * memoryGrowthFactor @@ -283,7 +273,6 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) // we can immediately free up space for other threads. Otherwise, if we return an iterator, // we release the memory claimed by this thread later on when the task finishes. if (keepUnrolling) { - vector = null val amountToRelease = currentUnrollMemoryForThisThread - previousMemoryReserved releaseUnrollMemoryForThisThread(amountToRelease) } @@ -433,11 +422,16 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) /** * Reserve additional memory for unrolling blocks used by this thread. + * Return whether the request is granted. */ - private[spark] def reserveUnrollMemoryForThisThread(memory: Long): Unit = { - val threadId = Thread.currentThread().getId + private[spark] def reserveUnrollMemoryForThisThread(memory: Long): Boolean = { accountingLock.synchronized { - unrollMemoryMap(threadId) = unrollMemoryMap.getOrElse(threadId, 0L) + memory + val granted = freeMemory > currentUnrollMemory + memory + if (granted) { + val threadId = Thread.currentThread().getId + unrollMemoryMap(threadId) = unrollMemoryMap.getOrElse(threadId, 0L) + memory + } + granted } } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index ffecbb60b7153..dd4fd535d3577 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -1005,21 +1005,24 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(memoryStore.currentUnrollMemoryForThisThread === 0) // Reserve - memoryStore.reserveUnrollMemoryForThisThread(1480) - assert(memoryStore.currentUnrollMemoryForThisThread === 1480) + memoryStore.reserveUnrollMemoryForThisThread(100) + assert(memoryStore.currentUnrollMemoryForThisThread === 100) + memoryStore.reserveUnrollMemoryForThisThread(200) + assert(memoryStore.currentUnrollMemoryForThisThread === 300) + memoryStore.reserveUnrollMemoryForThisThread(500) + assert(memoryStore.currentUnrollMemoryForThisThread === 800) memoryStore.reserveUnrollMemoryForThisThread(1000000) - assert(memoryStore.currentUnrollMemoryForThisThread === 1001480) - + assert(memoryStore.currentUnrollMemoryForThisThread === 800) // not granted // Release - memoryStore.releaseUnrollMemoryForThisThread(1000000) - assert(memoryStore.currentUnrollMemoryForThisThread === 1480) memoryStore.releaseUnrollMemoryForThisThread(100) - assert(memoryStore.currentUnrollMemoryForThisThread === 1380) - + assert(memoryStore.currentUnrollMemoryForThisThread === 700) + memoryStore.releaseUnrollMemoryForThisThread(100) + assert(memoryStore.currentUnrollMemoryForThisThread === 600) // Reserve again - memoryStore.reserveUnrollMemoryForThisThread(3620) + memoryStore.reserveUnrollMemoryForThisThread(4400) assert(memoryStore.currentUnrollMemoryForThisThread === 5000) - + memoryStore.reserveUnrollMemoryForThisThread(20000) + assert(memoryStore.currentUnrollMemoryForThisThread === 5000) // not granted // Release again memoryStore.releaseUnrollMemoryForThisThread(1000) assert(memoryStore.currentUnrollMemoryForThisThread === 4000)