From 5d130e44dbb8259588ac1b9006dc41c597c8a4a0 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 8 Jan 2016 13:11:51 -0800 Subject: [PATCH 01/81] Add block reference counting class. --- .../storage/MemoryStoreReferenceCounter.scala | 81 +++++++++++++ .../MemoryStoreReferenceCounterSuite.scala | 113 ++++++++++++++++++ 2 files changed, 194 insertions(+) create mode 100644 core/src/main/scala/org/apache/spark/storage/MemoryStoreReferenceCounter.scala create mode 100644 core/src/test/scala/org/apache/spark/storage/MemoryStoreReferenceCounterSuite.scala diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStoreReferenceCounter.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStoreReferenceCounter.scala new file mode 100644 index 0000000000000..14ea381c82aea --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStoreReferenceCounter.scala @@ -0,0 +1,81 @@ +/* + * 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.storage + +import scala.collection.JavaConverters._ + +import com.google.common.cache.{CacheBuilder, CacheLoader} +import com.google.common.collect.ConcurrentHashMultiset + +private[storage] class MemoryStoreReferenceCounter { + + type TaskAttemptId = java.lang.Long + + private[this] val allReferencedBlocks = ConcurrentHashMultiset.create[BlockId]() + private[this] val blocksReferencedByTask = { + val loader = new CacheLoader[TaskAttemptId, ConcurrentHashMultiset[BlockId]] { + override def load(t: TaskAttemptId) = ConcurrentHashMultiset.create[BlockId]() + } + CacheBuilder.newBuilder().build(loader) + } + + /** + * Return the number of map entries in this reference counter's internal data structures. + * This is used in unit tests in order to detect memory leaks. + */ + private[storage] def getNumberOfMapEntries: Long = { + allReferencedBlocks.size() + + blocksReferencedByTask.size() + + blocksReferencedByTask.asMap().asScala.map(_._2.size()).sum + } + + def getReferenceCountForBlock(blockId: BlockId): Int = allReferencedBlocks.count(blockId) + + def retainBlockForTask(taskAttemptId: TaskAttemptId, blockId: BlockId): Unit = { + blocksReferencedByTask.get(taskAttemptId).add(blockId) + allReferencedBlocks.add(blockId) + } + + def releaseBlockForTask(taskAttemptId: TaskAttemptId, blockId: BlockId): Unit = { + val countsForTask = blocksReferencedByTask.get(taskAttemptId) + val newReferenceCountForTask: Int = countsForTask.remove(blockId, 1) - 1 + val newTotalReferenceCount: Int = allReferencedBlocks.remove(blockId, 1) - 1 + if (newReferenceCountForTask < 0) { + throw new IllegalStateException( + s"Task $taskAttemptId block $blockId more times than it was retained") + } + if (newTotalReferenceCount < 0) { + throw new IllegalStateException( + s"Task $taskAttemptId block $blockId more times than it was retained") + } + } + + def releaseAllBlocksForTaskAttempt(taskAttemptId: TaskAttemptId): Unit = { + val referenceCounts = blocksReferencedByTask.get(taskAttemptId) + blocksReferencedByTask.invalidate(taskAttemptId) + referenceCounts.entrySet().iterator().asScala.foreach { entry => + val blockId = entry.getElement + val taskRefCount = entry.getCount + val newRefCount = allReferencedBlocks.remove(blockId, taskRefCount) - taskRefCount + if (newRefCount < 0) { + throw new IllegalStateException( + s"Task $taskAttemptId block $blockId more times than it was retained") + } + } + } +} diff --git a/core/src/test/scala/org/apache/spark/storage/MemoryStoreReferenceCounterSuite.scala b/core/src/test/scala/org/apache/spark/storage/MemoryStoreReferenceCounterSuite.scala new file mode 100644 index 0000000000000..b1a24041982a7 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/MemoryStoreReferenceCounterSuite.scala @@ -0,0 +1,113 @@ +/* + * 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.storage + +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.SparkFunSuite + +class MemoryStoreReferenceCounterSuite extends SparkFunSuite with BeforeAndAfterEach { + + private var refCounter: MemoryStoreReferenceCounter = _ + + override protected def beforeEach(): Unit = { + super.beforeEach() + refCounter = new MemoryStoreReferenceCounter() + } + + override protected def afterEach(): Unit = { + refCounter = null + super.afterEach() + } + + test("blocks' initial reference counts are zero") { + assert(refCounter.getNumberOfMapEntries === 0) + assert(refCounter.getReferenceCountForBlock(TestBlockId("dummy")) === 0) + assert(refCounter.getNumberOfMapEntries === 0) + } + + test("error when releasing a block more times than it has been pinned") { + intercept[IllegalStateException] { + refCounter.releaseBlockForTask(taskAttemptId = 0L, TestBlockId("dummy")) + } + } + + test("retain and release block from a single task") { + val block = TestBlockId("dummy") + val taskAttemptId = 0L + refCounter.retainBlockForTask(taskAttemptId, block) + assert(refCounter.getReferenceCountForBlock(block) === 1) + refCounter.retainBlockForTask(taskAttemptId, block) + assert(refCounter.getReferenceCountForBlock(block) === 2) + refCounter.releaseBlockForTask(taskAttemptId, block) + assert(refCounter.getReferenceCountForBlock(block) === 1) + refCounter.releaseAllBlocksForTaskAttempt(taskAttemptId) + assert(refCounter.getReferenceCountForBlock(block) === 0L) + // Ensure that we didn't leak memory / map entries: + assert(refCounter.getNumberOfMapEntries === 0) + } + + test("retain and release block from multiple tasks") { + val block = TestBlockId("dummy") + val taskA = 0L + val taskB = 1L + + refCounter.retainBlockForTask(taskA, block) + refCounter.retainBlockForTask(taskA, block) + refCounter.retainBlockForTask(taskB, block) + refCounter.retainBlockForTask(taskB, block) + refCounter.retainBlockForTask(taskA, block) + + assert(refCounter.getReferenceCountForBlock(block) === 5) + + refCounter.releaseBlockForTask(taskA, block) + assert(refCounter.getReferenceCountForBlock(block) === 4) + + refCounter.releaseAllBlocksForTaskAttempt(taskA) + assert(refCounter.getReferenceCountForBlock(block) === 2) + + refCounter.releaseBlockForTask(taskB, block) + refCounter.releaseBlockForTask(taskB, block) + assert(refCounter.getReferenceCountForBlock(block) === 0) + + refCounter.releaseAllBlocksForTaskAttempt(taskB) + + // Ensure that we didn't leak memory / map entries: + assert(refCounter.getNumberOfMapEntries === 0) + } + + test("counts are per-block") { + val blockA = TestBlockId("blockA") + val blockB = TestBlockId("blockB") + val taskAttemptId = 0L + + refCounter.retainBlockForTask(taskAttemptId, blockA) + assert(refCounter.getReferenceCountForBlock(blockA) === 1) + assert(refCounter.getReferenceCountForBlock(blockB) === 0) + + refCounter.retainBlockForTask(taskAttemptId, blockB) + refCounter.retainBlockForTask(taskAttemptId, blockB) + assert(refCounter.getReferenceCountForBlock(blockA) === 1) + assert(refCounter.getReferenceCountForBlock(blockB) === 2) + + // Ensure that we didn't leak memory / map entries: + refCounter.releaseAllBlocksForTaskAttempt(taskAttemptId) + assert(refCounter.getNumberOfMapEntries === 0) + } + +} From 423faabe3a34c6021a859c93cb97ac7c946529e2 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 8 Jan 2016 13:46:13 -0800 Subject: [PATCH 02/81] Make the ReferenceCounter generic, since it's not specific to storage in any respect. --- .../storage/MemoryStoreReferenceCounter.scala | 81 ------------ .../util/collection/ReferenceCounter.scala | 122 ++++++++++++++++++ .../MemoryStoreReferenceCounterSuite.scala | 113 ---------------- .../collection/ReferenceCounterSuite.scala | 114 ++++++++++++++++ 4 files changed, 236 insertions(+), 194 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/storage/MemoryStoreReferenceCounter.scala create mode 100644 core/src/main/scala/org/apache/spark/util/collection/ReferenceCounter.scala delete mode 100644 core/src/test/scala/org/apache/spark/storage/MemoryStoreReferenceCounterSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/util/collection/ReferenceCounterSuite.scala diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStoreReferenceCounter.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStoreReferenceCounter.scala deleted file mode 100644 index 14ea381c82aea..0000000000000 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStoreReferenceCounter.scala +++ /dev/null @@ -1,81 +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.storage - -import scala.collection.JavaConverters._ - -import com.google.common.cache.{CacheBuilder, CacheLoader} -import com.google.common.collect.ConcurrentHashMultiset - -private[storage] class MemoryStoreReferenceCounter { - - type TaskAttemptId = java.lang.Long - - private[this] val allReferencedBlocks = ConcurrentHashMultiset.create[BlockId]() - private[this] val blocksReferencedByTask = { - val loader = new CacheLoader[TaskAttemptId, ConcurrentHashMultiset[BlockId]] { - override def load(t: TaskAttemptId) = ConcurrentHashMultiset.create[BlockId]() - } - CacheBuilder.newBuilder().build(loader) - } - - /** - * Return the number of map entries in this reference counter's internal data structures. - * This is used in unit tests in order to detect memory leaks. - */ - private[storage] def getNumberOfMapEntries: Long = { - allReferencedBlocks.size() + - blocksReferencedByTask.size() + - blocksReferencedByTask.asMap().asScala.map(_._2.size()).sum - } - - def getReferenceCountForBlock(blockId: BlockId): Int = allReferencedBlocks.count(blockId) - - def retainBlockForTask(taskAttemptId: TaskAttemptId, blockId: BlockId): Unit = { - blocksReferencedByTask.get(taskAttemptId).add(blockId) - allReferencedBlocks.add(blockId) - } - - def releaseBlockForTask(taskAttemptId: TaskAttemptId, blockId: BlockId): Unit = { - val countsForTask = blocksReferencedByTask.get(taskAttemptId) - val newReferenceCountForTask: Int = countsForTask.remove(blockId, 1) - 1 - val newTotalReferenceCount: Int = allReferencedBlocks.remove(blockId, 1) - 1 - if (newReferenceCountForTask < 0) { - throw new IllegalStateException( - s"Task $taskAttemptId block $blockId more times than it was retained") - } - if (newTotalReferenceCount < 0) { - throw new IllegalStateException( - s"Task $taskAttemptId block $blockId more times than it was retained") - } - } - - def releaseAllBlocksForTaskAttempt(taskAttemptId: TaskAttemptId): Unit = { - val referenceCounts = blocksReferencedByTask.get(taskAttemptId) - blocksReferencedByTask.invalidate(taskAttemptId) - referenceCounts.entrySet().iterator().asScala.foreach { entry => - val blockId = entry.getElement - val taskRefCount = entry.getCount - val newRefCount = allReferencedBlocks.remove(blockId, taskRefCount) - taskRefCount - if (newRefCount < 0) { - throw new IllegalStateException( - s"Task $taskAttemptId block $blockId more times than it was retained") - } - } - } -} diff --git a/core/src/main/scala/org/apache/spark/util/collection/ReferenceCounter.scala b/core/src/main/scala/org/apache/spark/util/collection/ReferenceCounter.scala new file mode 100644 index 0000000000000..091a902988d12 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/ReferenceCounter.scala @@ -0,0 +1,122 @@ +/* + * 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.JavaConverters._ + +import com.google.common.cache.{CacheBuilder, CacheLoader} +import com.google.common.collect.ConcurrentHashMultiset + +import org.apache.spark.TaskContext + +/** + * Thread-safe collection for maintaining both global and per-task reference counts for objects. + */ +private[spark] class ReferenceCounter[T] { + + private type TaskAttemptId = Long + + /** + * Total references across all tasks. + */ + private[this] val allReferences = ConcurrentHashMultiset.create[T]() + + /** + * Total references per task. Used to auto-release references upon task completion. + */ + private[this] val referencesByTask = { + // We need to explicitly box as java.lang.Long to avoid a type mismatch error: + val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[T]] { + override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[T]() + } + CacheBuilder.newBuilder().build(loader) + } + + /** + * Returns the total reference count, across all tasks, for the given object. + */ + def getReferenceCount(obj: T): Int = allReferences.count(obj) + + /** + * Increments the given object's reference count for the current task. + */ + def retain(obj: T): Unit = { + retainForTask(TaskContext.get().taskAttemptId(), obj) + } + + /** + * Decrements the given object's reference count for the current task. + */ + def release(obj: T): Unit = { + releaseForTask(TaskContext.get().taskAttemptId(), obj) + } + + /** + * Increments the given object's reference count for the given task. + */ + def retainForTask(taskAttemptId: TaskAttemptId, obj: T): Unit = { + referencesByTask.get(taskAttemptId).add(obj) + allReferences.add(obj) + } + + /** + * Decrements the given object's reference count for the given task. + */ + def releaseForTask(taskAttemptId: TaskAttemptId, obj: T): Unit = { + val countsForTask = referencesByTask.get(taskAttemptId) + val newReferenceCountForTask: Int = countsForTask.remove(obj, 1) - 1 + val newTotalReferenceCount: Int = allReferences.remove(obj, 1) - 1 + if (newReferenceCountForTask < 0) { + throw new IllegalStateException( + s"Task $taskAttemptId released object $obj more times than it was retained") + } + if (newTotalReferenceCount < 0) { + throw new IllegalStateException( + s"Task $taskAttemptId released object $obj more times than it was retained") + } + } + + /** + * Release all references held by the given task, clearing that task's reference bookkeeping + * structures and updating the global reference counts. This method should be called at the + * end of a task (either by a task completion handler or in `TaskRunner.run()`). + */ + def releaseAllReferencesForTask(taskAttemptId: TaskAttemptId): Unit = { + val referenceCounts = referencesByTask.get(taskAttemptId) + referencesByTask.invalidate(taskAttemptId) + referenceCounts.entrySet().iterator().asScala.foreach { entry => + val obj = entry.getElement + val taskRefCount = entry.getCount + val newRefCount = allReferences.remove(obj, taskRefCount) - taskRefCount + if (newRefCount < 0) { + throw new IllegalStateException( + s"Task $taskAttemptId released object $obj more times than it was retained") + } + } + } + + /** + * Return the number of map entries in this reference counter's internal data structures. + * This is used in unit tests in order to detect memory leaks. + */ + private[collection] def getNumberOfMapEntries: Long = { + allReferences.size() + + referencesByTask.size() + + referencesByTask.asMap().asScala.map(_._2.size()).sum + } +} diff --git a/core/src/test/scala/org/apache/spark/storage/MemoryStoreReferenceCounterSuite.scala b/core/src/test/scala/org/apache/spark/storage/MemoryStoreReferenceCounterSuite.scala deleted file mode 100644 index b1a24041982a7..0000000000000 --- a/core/src/test/scala/org/apache/spark/storage/MemoryStoreReferenceCounterSuite.scala +++ /dev/null @@ -1,113 +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.storage - -import org.scalatest.BeforeAndAfterEach - -import org.apache.spark.SparkFunSuite - -class MemoryStoreReferenceCounterSuite extends SparkFunSuite with BeforeAndAfterEach { - - private var refCounter: MemoryStoreReferenceCounter = _ - - override protected def beforeEach(): Unit = { - super.beforeEach() - refCounter = new MemoryStoreReferenceCounter() - } - - override protected def afterEach(): Unit = { - refCounter = null - super.afterEach() - } - - test("blocks' initial reference counts are zero") { - assert(refCounter.getNumberOfMapEntries === 0) - assert(refCounter.getReferenceCountForBlock(TestBlockId("dummy")) === 0) - assert(refCounter.getNumberOfMapEntries === 0) - } - - test("error when releasing a block more times than it has been pinned") { - intercept[IllegalStateException] { - refCounter.releaseBlockForTask(taskAttemptId = 0L, TestBlockId("dummy")) - } - } - - test("retain and release block from a single task") { - val block = TestBlockId("dummy") - val taskAttemptId = 0L - refCounter.retainBlockForTask(taskAttemptId, block) - assert(refCounter.getReferenceCountForBlock(block) === 1) - refCounter.retainBlockForTask(taskAttemptId, block) - assert(refCounter.getReferenceCountForBlock(block) === 2) - refCounter.releaseBlockForTask(taskAttemptId, block) - assert(refCounter.getReferenceCountForBlock(block) === 1) - refCounter.releaseAllBlocksForTaskAttempt(taskAttemptId) - assert(refCounter.getReferenceCountForBlock(block) === 0L) - // Ensure that we didn't leak memory / map entries: - assert(refCounter.getNumberOfMapEntries === 0) - } - - test("retain and release block from multiple tasks") { - val block = TestBlockId("dummy") - val taskA = 0L - val taskB = 1L - - refCounter.retainBlockForTask(taskA, block) - refCounter.retainBlockForTask(taskA, block) - refCounter.retainBlockForTask(taskB, block) - refCounter.retainBlockForTask(taskB, block) - refCounter.retainBlockForTask(taskA, block) - - assert(refCounter.getReferenceCountForBlock(block) === 5) - - refCounter.releaseBlockForTask(taskA, block) - assert(refCounter.getReferenceCountForBlock(block) === 4) - - refCounter.releaseAllBlocksForTaskAttempt(taskA) - assert(refCounter.getReferenceCountForBlock(block) === 2) - - refCounter.releaseBlockForTask(taskB, block) - refCounter.releaseBlockForTask(taskB, block) - assert(refCounter.getReferenceCountForBlock(block) === 0) - - refCounter.releaseAllBlocksForTaskAttempt(taskB) - - // Ensure that we didn't leak memory / map entries: - assert(refCounter.getNumberOfMapEntries === 0) - } - - test("counts are per-block") { - val blockA = TestBlockId("blockA") - val blockB = TestBlockId("blockB") - val taskAttemptId = 0L - - refCounter.retainBlockForTask(taskAttemptId, blockA) - assert(refCounter.getReferenceCountForBlock(blockA) === 1) - assert(refCounter.getReferenceCountForBlock(blockB) === 0) - - refCounter.retainBlockForTask(taskAttemptId, blockB) - refCounter.retainBlockForTask(taskAttemptId, blockB) - assert(refCounter.getReferenceCountForBlock(blockA) === 1) - assert(refCounter.getReferenceCountForBlock(blockB) === 2) - - // Ensure that we didn't leak memory / map entries: - refCounter.releaseAllBlocksForTaskAttempt(taskAttemptId) - assert(refCounter.getNumberOfMapEntries === 0) - } - -} diff --git a/core/src/test/scala/org/apache/spark/util/collection/ReferenceCounterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ReferenceCounterSuite.scala new file mode 100644 index 0000000000000..cbfe0bfdee7bc --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/collection/ReferenceCounterSuite.scala @@ -0,0 +1,114 @@ +/* + * 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 org.scalatest.BeforeAndAfterEach + +import org.apache.spark.SparkFunSuite +import org.apache.spark.storage.{BlockId, TestBlockId} + +class ReferenceCounterSuite extends SparkFunSuite with BeforeAndAfterEach { + + private var refCounter: ReferenceCounter[BlockId] = _ + + override protected def beforeEach(): Unit = { + super.beforeEach() + refCounter = new ReferenceCounter() + } + + override protected def afterEach(): Unit = { + refCounter = null + super.afterEach() + } + + test("initial reference counts are zero") { + assert(refCounter.getNumberOfMapEntries === 0) + assert(refCounter.getReferenceCount(TestBlockId("dummy")) === 0) + assert(refCounter.getNumberOfMapEntries === 0) + } + + test("error when releasing more times than retained") { + intercept[IllegalStateException] { + refCounter.releaseForTask(taskAttemptId = 0L, TestBlockId("dummy")) + } + } + + test("retain and release from a single task") { + val block = TestBlockId("dummy") + val taskAttemptId = 0L + refCounter.retainForTask(taskAttemptId, block) + assert(refCounter.getReferenceCount(block) === 1) + refCounter.retainForTask(taskAttemptId, block) + assert(refCounter.getReferenceCount(block) === 2) + refCounter.releaseForTask(taskAttemptId, block) + assert(refCounter.getReferenceCount(block) === 1) + refCounter.releaseAllReferencesForTask(taskAttemptId) + assert(refCounter.getReferenceCount(block) === 0L) + // Ensure that we didn't leak memory / map entries: + assert(refCounter.getNumberOfMapEntries === 0) + } + + test("retain and release from multiple tasks") { + val block = TestBlockId("dummy") + val taskA = 0L + val taskB = 1L + + refCounter.retainForTask(taskA, block) + refCounter.retainForTask(taskA, block) + refCounter.retainForTask(taskB, block) + refCounter.retainForTask(taskB, block) + refCounter.retainForTask(taskA, block) + + assert(refCounter.getReferenceCount(block) === 5) + + refCounter.releaseForTask(taskA, block) + assert(refCounter.getReferenceCount(block) === 4) + + refCounter.releaseAllReferencesForTask(taskA) + assert(refCounter.getReferenceCount(block) === 2) + + refCounter.releaseForTask(taskB, block) + refCounter.releaseForTask(taskB, block) + assert(refCounter.getReferenceCount(block) === 0) + + refCounter.releaseAllReferencesForTask(taskB) + + // Ensure that we didn't leak memory / map entries: + assert(refCounter.getNumberOfMapEntries === 0) + } + + test("counts are per-object") { + val blockA = TestBlockId("blockA") + val blockB = TestBlockId("blockB") + val taskAttemptId = 0L + + refCounter.retainForTask(taskAttemptId, blockA) + assert(refCounter.getReferenceCount(blockA) === 1) + assert(refCounter.getReferenceCount(blockB) === 0) + + refCounter.retainForTask(taskAttemptId, blockB) + refCounter.retainForTask(taskAttemptId, blockB) + assert(refCounter.getReferenceCount(blockA) === 1) + assert(refCounter.getReferenceCount(blockB) === 2) + + // Ensure that we didn't leak memory / map entries: + refCounter.releaseAllReferencesForTask(taskAttemptId) + assert(refCounter.getNumberOfMapEntries === 0) + } + +} From 76cfebd15137fb0090f89dbd1791aad9eca09902 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 8 Jan 2016 15:13:33 -0800 Subject: [PATCH 03/81] Integrate reference counter into storage eviction code. --- .../apache/spark/storage/BlockManager.scala | 38 ++++- .../apache/spark/storage/MemoryStore.scala | 11 +- .../util/collection/ReferenceCounter.scala | 10 +- .../spark/storage/BlockManagerSuite.scala | 145 ++++++++++-------- 4 files changed, 131 insertions(+), 73 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 4479e6875a731..f5fe6947c179e 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -43,6 +43,7 @@ import org.apache.spark.rpc.RpcEnv import org.apache.spark.serializer.{Serializer, SerializerInstance} import org.apache.spark.shuffle.ShuffleManager import org.apache.spark.util._ +import org.apache.spark.util.collection.ReferenceCounter private[spark] sealed trait BlockValues private[spark] case class ByteBufferValues(buffer: ByteBuffer) extends BlockValues @@ -161,6 +162,8 @@ private[spark] class BlockManager( * loaded yet. */ private lazy val compressionCodec: CompressionCodec = CompressionCodec.createCodec(conf) + private val referenceCounts = new ReferenceCounter[BlockId] + /** * Initializes the BlockManager with the given appId. This is not performed in the constructor as * the appId may not be known at BlockManager instantiation time (in particular for the driver, @@ -414,7 +417,11 @@ private[spark] class BlockManager( */ def getLocal(blockId: BlockId): Option[BlockResult] = { logDebug(s"Getting local block $blockId") - doGetLocal(blockId, asBlockResult = true).asInstanceOf[Option[BlockResult]] + val res = doGetLocal(blockId, asBlockResult = true).asInstanceOf[Option[BlockResult]] + if (res.isDefined) { + referenceCounts.retain(blockId) + } + res } /** @@ -424,7 +431,7 @@ private[spark] class BlockManager( logDebug(s"Getting local block $blockId as bytes") // As an optimization for map output fetches, if the block is for a shuffle, return it // without acquiring a lock; the disk store never deletes (recent) items so this should work - if (blockId.isShuffle) { + val res = if (blockId.isShuffle) { val shuffleBlockResolver = shuffleManager.shuffleBlockResolver // TODO: This should gracefully handle case where local block is not available. Currently // downstream code will throw an exception. @@ -433,6 +440,10 @@ private[spark] class BlockManager( } else { doGetLocal(blockId, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] } + if (res.isDefined) { + referenceCounts.retain(blockId) + } + res } private def doGetLocal(blockId: BlockId, asBlockResult: Boolean): Option[Any] = { @@ -564,7 +575,11 @@ private[spark] class BlockManager( */ def getRemote(blockId: BlockId): Option[BlockResult] = { logDebug(s"Getting remote block $blockId") - doGetRemote(blockId, asBlockResult = true).asInstanceOf[Option[BlockResult]] + val res = doGetRemote(blockId, asBlockResult = true).asInstanceOf[Option[BlockResult]] + if (res.isDefined) { + referenceCounts.retain(blockId) + } + res } /** @@ -572,7 +587,11 @@ private[spark] class BlockManager( */ def getRemoteBytes(blockId: BlockId): Option[ByteBuffer] = { logDebug(s"Getting remote block $blockId as bytes") - doGetRemote(blockId, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] + val res = doGetRemote(blockId, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] + if (res.isDefined) { + referenceCounts.retain(blockId) + } + res } /** @@ -642,6 +661,17 @@ private[spark] class BlockManager( None } + /** + * Release one reference to the given block. + */ + def release(blockId: BlockId): Unit = { + referenceCounts.release(blockId) + } + + private[storage] def getReferenceCount(blockId: BlockId): Int = { + referenceCounts.getReferenceCount(blockId) + } + def putIterator( blockId: BlockId, values: Iterator[Any], 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 bdab8c2332fae..e97f5419b2f30 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -213,6 +213,11 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo } override def remove(blockId: BlockId): Boolean = memoryManager.synchronized { + val referenceCount = blockManager.getReferenceCount(blockId) + if (referenceCount != 0) { + throw new IllegalStateException( + s"Cannot free block $blockId since it is still referenced $referenceCount times") + } val entry = entries.synchronized { entries.remove(blockId) } if (entry != null) { memoryManager.releaseStorageMemory(entry.size) @@ -425,6 +430,10 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo var freedMemory = 0L val rddToAdd = blockId.flatMap(getRddId) val selectedBlocks = new ArrayBuffer[BlockId] + def blockIsEvictable(blockId: BlockId): Boolean = { + blockManager.getReferenceCount(blockId) == 0 && + (rddToAdd.isEmpty || rddToAdd != getRddId(blockId)) + } // This is synchronized to ensure that the set of entries is not changed // (because of getValue or getBytes) while traversing the iterator, as that // can lead to exceptions. @@ -433,7 +442,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo while (freedMemory < space && iterator.hasNext) { val pair = iterator.next() val blockId = pair.getKey - if (rddToAdd.isEmpty || rddToAdd != getRddId(blockId)) { + if (blockIsEvictable(blockId)) { selectedBlocks += blockId freedMemory += pair.getValue.size } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ReferenceCounter.scala b/core/src/main/scala/org/apache/spark/util/collection/ReferenceCounter.scala index 091a902988d12..6823c1f9339ec 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ReferenceCounter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ReferenceCounter.scala @@ -55,15 +55,15 @@ private[spark] class ReferenceCounter[T] { /** * Increments the given object's reference count for the current task. */ - def retain(obj: T): Unit = { - retainForTask(TaskContext.get().taskAttemptId(), obj) - } + def retain(obj: T): Unit = retainForTask(currentTaskAttemptId, obj) /** * Decrements the given object's reference count for the current task. */ - def release(obj: T): Unit = { - releaseForTask(TaskContext.get().taskAttemptId(), obj) + def release(obj: T): Unit = releaseForTask(currentTaskAttemptId, obj) + + private def currentTaskAttemptId: TaskAttemptId = { + Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1L) } /** 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 21db3b1c9ffbd..3ac33dab45a82 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -174,8 +174,11 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // Checking whether blocks are in memory assert(store.getSingle("a1").isDefined, "a1 was not in store") + store.release("a1") assert(store.getSingle("a2").isDefined, "a2 was not in store") + store.release("a2") assert(store.getSingle("a3").isDefined, "a3 was not in store") + store.release("a3") // Checking whether master knows about the blocks or not assert(master.getLocations("a1").size > 0, "master was not told about a1") @@ -223,8 +226,11 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE 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") + store.release("a1-to-remove") assert(store.getSingle("a2-to-remove").isDefined, "a2 was not in store") + store.release("a2-to-remove") assert(store.getSingle("a3-to-remove").isDefined, "a3 was not in store") + store.release("a3-to-remove") // Checking whether master knows about the blocks or not assert(master.getLocations("a1-to-remove").size > 0, "master was not told about a1") @@ -313,9 +319,13 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // verify whether the blocks exist in both the stores Seq(driverStore, executorStore).foreach { case s => s.getLocal(broadcast0BlockId) should not be (None) + s.release(broadcast0BlockId) s.getLocal(broadcast1BlockId) should not be (None) + s.release(broadcast1BlockId) s.getLocal(broadcast2BlockId) should not be (None) + s.release(broadcast2BlockId) s.getLocal(broadcast2BlockId2) should not be (None) + s.release(broadcast2BlockId2) } // remove broadcast 0 block only from executors @@ -324,17 +334,23 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // only broadcast 0 block should be removed from the executor store executorStore.getLocal(broadcast0BlockId) should be (None) executorStore.getLocal(broadcast1BlockId) should not be (None) + executorStore.release(broadcast1BlockId) executorStore.getLocal(broadcast2BlockId) should not be (None) + executorStore.release(broadcast2BlockId) // nothing should be removed from the driver store driverStore.getLocal(broadcast0BlockId) should not be (None) + driverStore.release(broadcast0BlockId) driverStore.getLocal(broadcast1BlockId) should not be (None) + driverStore.release(broadcast1BlockId) driverStore.getLocal(broadcast2BlockId) should not be (None) + driverStore.release(broadcast2BlockId) // remove broadcast 0 block from the driver as well master.removeBroadcast(0, removeFromMaster = true, blocking = true) driverStore.getLocal(broadcast0BlockId) should be (None) driverStore.getLocal(broadcast1BlockId) should not be (None) + driverStore.release(broadcast1BlockId) // remove broadcast 1 block from both the stores asynchronously // and verify all broadcast 1 blocks have been removed @@ -505,38 +521,30 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("in-memory LRU storage") { - store = makeBlockManager(12000) - 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) - assert(store.getSingle("a2").isDefined, "a2 was not in store") - assert(store.getSingle("a3").isDefined, "a3 was not in store") - assert(store.getSingle("a1") === None, "a1 was in store") - assert(store.getSingle("a2").isDefined, "a2 was not in store") - // At this point a2 was gotten last, so LRU will getSingle rid of a3 - store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) - assert(store.getSingle("a1").isDefined, "a1 was not in store") - assert(store.getSingle("a2").isDefined, "a2 was not in store") - assert(store.getSingle("a3") === None, "a3 was in store") + testInMemoryLRUStorage(StorageLevel.MEMORY_ONLY) } test("in-memory LRU storage with serialization") { + testInMemoryLRUStorage(StorageLevel.MEMORY_ONLY_SER) + } + + private def testInMemoryLRUStorage(storageLevel: StorageLevel): Unit = { store = makeBlockManager(12000) 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) + store.putSingle("a1", a1, storageLevel) + store.putSingle("a2", a2, storageLevel) + store.putSingle("a3", a3, storageLevel) assert(store.getSingle("a2").isDefined, "a2 was not in store") + store.release("a2") assert(store.getSingle("a3").isDefined, "a3 was not in store") + store.release("a3") assert(store.getSingle("a1") === None, "a1 was in store") assert(store.getSingle("a2").isDefined, "a2 was not in store") + store.release("a2") // At this point a2 was gotten last, so LRU will getSingle rid of a3 - store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_SER) + store.putSingle("a1", a1, storageLevel) assert(store.getSingle("a1").isDefined, "a1 was not in store") assert(store.getSingle("a2").isDefined, "a2 was not in store") assert(store.getSingle("a3") === None, "a3 was in store") @@ -618,62 +626,38 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("disk and memory storage") { - store = makeBlockManager(12000) - 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) - assert(store.getSingle("a2").isDefined, "a2 was not in store") - assert(store.getSingle("a3").isDefined, "a3 was not in store") - assert(store.memoryStore.getValues("a1") == None, "a1 was in memory store") - assert(store.getSingle("a1").isDefined, "a1 was not in store") - assert(store.memoryStore.getValues("a1").isDefined, "a1 was not in memory store") + testDiskAndMemoryStorage(StorageLevel.MEMORY_AND_DISK, _.getSingle) } test("disk and memory storage with getLocalBytes") { - store = makeBlockManager(12000) - 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) - assert(store.getLocalBytes("a2").isDefined, "a2 was not in store") - assert(store.getLocalBytes("a3").isDefined, "a3 was not in store") - assert(store.memoryStore.getValues("a1") == None, "a1 was in memory store") - assert(store.getLocalBytes("a1").isDefined, "a1 was not in store") - assert(store.memoryStore.getValues("a1").isDefined, "a1 was not in memory store") + testDiskAndMemoryStorage(StorageLevel.MEMORY_AND_DISK, _.getLocalBytes) } test("disk and memory storage with serialization") { - store = makeBlockManager(12000) - 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) - assert(store.getSingle("a2").isDefined, "a2 was not in store") - assert(store.getSingle("a3").isDefined, "a3 was not in store") - assert(store.memoryStore.getValues("a1") == None, "a1 was in memory store") - assert(store.getSingle("a1").isDefined, "a1 was not in store") - assert(store.memoryStore.getValues("a1").isDefined, "a1 was not in memory store") + testDiskAndMemoryStorage(StorageLevel.MEMORY_AND_DISK_SER, _.getSingle) } test("disk and memory storage with serialization and getLocalBytes") { + testDiskAndMemoryStorage(StorageLevel.MEMORY_AND_DISK_SER, _.getLocalBytes) + } + + def testDiskAndMemoryStorage( + storageLevel: StorageLevel, + accessMethod: BlockManager => BlockId => Option[_]): Unit = { store = makeBlockManager(12000) 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) - assert(store.getLocalBytes("a2").isDefined, "a2 was not in store") - assert(store.getLocalBytes("a3").isDefined, "a3 was not in store") - assert(store.memoryStore.getValues("a1") == None, "a1 was in memory store") - assert(store.getLocalBytes("a1").isDefined, "a1 was not in store") + store.putSingle("a1", a1, storageLevel) + store.putSingle("a2", a2, storageLevel) + store.putSingle("a3", a3, storageLevel) + assert(accessMethod(store)("a2").isDefined, "a2 was not in store") + store.release("a2") + assert(accessMethod(store)("a3").isDefined, "a3 was not in store") + store.release("a3") + assert(store.memoryStore.getValues("a1").isEmpty, "a1 was in memory store") + assert(accessMethod(store)("a1").isDefined, "a1 was not in store") + store.release("a1") assert(store.memoryStore.getValues("a1").isDefined, "a1 was not in memory store") } @@ -689,14 +673,20 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE store.putSingle("a3", a3, StorageLevel.DISK_ONLY) // At this point LRU should not kick in because a3 is only on disk assert(store.getSingle("a1").isDefined, "a1 was not in store") + store.release("a1") assert(store.getSingle("a2").isDefined, "a2 was not in store") + store.release("a2") assert(store.getSingle("a3").isDefined, "a3 was not in store") + store.release("a3") // Now let's add in a4, which uses both disk and memory; a1 should drop out store.putSingle("a4", a4, StorageLevel.MEMORY_AND_DISK_SER) assert(store.getSingle("a1") == None, "a1 was in store") assert(store.getSingle("a2").isDefined, "a2 was not in store") + store.release("a2") assert(store.getSingle("a3").isDefined, "a3 was not in store") + store.release("a3") assert(store.getSingle("a4").isDefined, "a4 was not in store") + store.release("a4") } test("in-memory LRU with streams") { @@ -709,17 +699,27 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE 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) + store.release("list2") + store.release("list2") assert(store.get("list3").isDefined, "list3 was not in store") assert(store.get("list3").get.data.size === 2) + store.release("list3") + store.release("list3") 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) + store.release("list2") + store.release("list2") // At this point list2 was gotten last, so LRU will getSingle rid of list3 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) + store.release("list1") + store.release("list1") assert(store.get("list2").isDefined, "list2 was not in store") assert(store.get("list2").get.data.size === 2) + store.release("list2") + store.release("list2") assert(store.get("list3") === None, "list1 was in store") } @@ -739,25 +739,43 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // At this point LRU should not kick in because list3 is only on disk assert(store.get("list1").isDefined, "list1 was not in store") assert(store.get("list1").get.data.size === 2) + store.release("list1") + store.release("list1") assert(store.get("list2").isDefined, "list2 was not in store") assert(store.get("list2").get.data.size === 2) + store.release("list2") + store.release("list2") assert(store.get("list3").isDefined, "list3 was not in store") assert(store.get("list3").get.data.size === 2) + store.release("list3") + store.release("list3") assert(store.get("list1").isDefined, "list1 was not in store") assert(store.get("list1").get.data.size === 2) + store.release("list1") + store.release("list1") assert(store.get("list2").isDefined, "list2 was not in store") assert(store.get("list2").get.data.size === 2) + store.release("list2") + store.release("list2") assert(store.get("list3").isDefined, "list3 was not in store") assert(store.get("list3").get.data.size === 2) + store.release("list3") + store.release("list3") // Now let's add in list4, which uses both disk and memory; list1 should drop out 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) + store.release("list2") + store.release("list2") assert(store.get("list3").isDefined, "list3 was not in store") assert(store.get("list3").get.data.size === 2) + store.release("list3") + store.release("list3") assert(store.get("list4").isDefined, "list4 was not in store") assert(store.get("list4").get.data.size === 2) + store.release("list4") + store.release("list4") } test("negative byte values in ByteBufferInputStream") { @@ -1059,6 +1077,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE 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") + store.release(rdd(1, 0)) // According to the same-RDD rule, rdd_1_0 should be replaced here. 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. From 7cad770025a3ab4f3d2e312482ff7baf20a159cb Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 14 Jan 2016 12:07:13 -0800 Subject: [PATCH 04/81] Fix BlockManagerReplicationSuite tests. --- .../org/apache/spark/storage/BlockManagerReplicationSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala index 3fd6fb4560465..f5758a4a51c57 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -367,6 +367,7 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo }.foreach { testStore => val testStoreName = testStore.blockManagerId.executorId assert(testStore.getLocal(blockId).isDefined, s"$blockId was not found in $testStoreName") + testStore.release(blockId) assert(master.getLocations(blockId).map(_.executorId).toSet.contains(testStoreName), s"master does not have status for ${blockId.name} in $testStoreName") From 8ae88b0a4904e55e3f685c1a6bd1abe2fa0f4f35 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 14 Jan 2016 14:43:10 -0800 Subject: [PATCH 05/81] Add unit test for pinCount > 0 preventing eviction. --- .../spark/storage/BlockManagerSuite.scala | 37 +++++++++++++++---- 1 file changed, 29 insertions(+), 8 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 3ac33dab45a82..f6b9fc79dbae2 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -56,9 +56,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val mapOutputTracker = new MapOutputTrackerMaster(conf) val shuffleManager = new HashShuffleManager(conf) - // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test conf.set("spark.kryoserializer.buffer", "1m") - val serializer = new KryoSerializer(conf) // Implicitly convert strings to BlockIds for test clarity. implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value) @@ -68,6 +66,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE maxMem: Long, name: String = SparkContext.DRIVER_IDENTIFIER, master: BlockManagerMaster = this.master): BlockManager = { + val serializer = new KryoSerializer(conf) val transfer = new NettyBlockTransferService(conf, securityMgr, numCores = 1) val memManager = new StaticMemoryManager(conf, Long.MaxValue, maxMem, numCores = 1) val blockManager = new BlockManager(name, rpcEnv, master, serializer, conf, @@ -498,19 +497,16 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val list1 = List(new Array[Byte](4000)) store2.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) store3.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - var list1Get = store.getRemoteBytes("list1") - assert(list1Get.isDefined, "list1Get expected to be fetched") + assert(store.getRemoteBytes("list1").isDefined, "list1Get expected to be fetched") // block manager exit store2.stop() store2 = null - list1Get = store.getRemoteBytes("list1") - // get `list1` block - assert(list1Get.isDefined, "list1Get expected to be fetched") + assert(store.getRemoteBytes("list1").isDefined, "list1Get expected to be fetched") store3.stop() store3 = null // exception throw because there is no locations intercept[BlockFetchException] { - list1Get = store.getRemoteBytes("list1") + store.getRemoteBytes("list1") } } finally { origTimeoutOpt match { @@ -1342,4 +1338,29 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(result.data === Right(bytes)) assert(result.droppedBlocks === Nil) } + + test("blocks with non-zero reference counts cannot be evicted from the MemoryStore") { + store = makeBlockManager(12000) + val arr = new Array[Byte](4000) + // First store a1 and a2, both in memory, and a3, on disk only + store.putSingle("a1", arr, StorageLevel.MEMORY_ONLY_SER) + store.putSingle("a2", arr, StorageLevel.MEMORY_ONLY_SER) + assert(store.getSingle("a1").isDefined, "a1 was not in store") + assert(store.getSingle("a2").isDefined, "a2 was not in store") + // This put should fail because both a1 and a2 have non-zero reference counts: + store.putSingle("a3", arr, StorageLevel.MEMORY_ONLY_SER) + assert(store.getSingle("a3").isEmpty, "a3 was in store") + assert(store.getSingle("a1").isDefined, "a1 was not in store") + assert(store.getSingle("a2").isDefined, "a2 was not in store") + // Release both references to block a2: + store.release("a2") + store.release("a2") + assert(store.getReferenceCount("a2") === 0) + // Block a1 is the least-recently accessed, so an LRU eviction policy would evict it before + // block a2. However, a1 still has references, so this put of a3 should evict a2 instead: + store.putSingle("a3", arr, StorageLevel.MEMORY_ONLY_SER) + assert(store.getSingle("a2").isEmpty, "a2 was in store") + assert(store.getSingle("a1").isDefined, "a1 was not in store") + assert(store.getSingle("a3").isDefined, "a3 was not in store") + } } From c1a8d85e6404f5db56ced0c2f4dc0d18dbfee955 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 14 Jan 2016 15:07:51 -0800 Subject: [PATCH 06/81] Minimal changes to release refs on task completion. --- core/src/main/scala/org/apache/spark/executor/Executor.scala | 1 + .../main/scala/org/apache/spark/storage/BlockManager.scala | 4 ++++ 2 files changed, 5 insertions(+) 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 9b14184364246..17471e77b40f0 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -218,6 +218,7 @@ private[spark] class Executor( threwException = false res } finally { + env.blockManager.releaseAllReferencesForTask(taskId) val freedMemory = taskMemoryManager.cleanUpAllAllocatedMemory() if (freedMemory > 0) { val errMsg = s"Managed memory leak detected; size = $freedMemory bytes, TID = $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 f5fe6947c179e..bea7d7f79e0c5 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -668,6 +668,10 @@ private[spark] class BlockManager( referenceCounts.release(blockId) } + def releaseAllReferencesForTask(taskAttemptId: Long) = { + referenceCounts.releaseAllReferencesForTask(taskAttemptId) + } + private[storage] def getReferenceCount(blockId: BlockId): Int = { referenceCounts.getReferenceCount(blockId) } From 575a47bbd0154a2d9eccc37c86f8e35931bc95d0 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 14 Jan 2016 16:00:00 -0800 Subject: [PATCH 07/81] Fix Scalastyle. --- core/src/main/scala/org/apache/spark/storage/BlockManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 bea7d7f79e0c5..0a35e26290f30 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -668,7 +668,7 @@ private[spark] class BlockManager( referenceCounts.release(blockId) } - def releaseAllReferencesForTask(taskAttemptId: Long) = { + def releaseAllReferencesForTask(taskAttemptId: Long): Unit = { referenceCounts.releaseAllReferencesForTask(taskAttemptId) } From 7f289107fff5de5af4f6bd192fd681e11d10bff0 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 20 Jan 2016 17:54:13 -0800 Subject: [PATCH 08/81] Fix CachedTableSuite tests. --- .../test/scala/org/apache/spark/sql/CachedTableSuite.scala | 4 +++- .../scala/org/apache/spark/sql/hive/CachedTableSuite.scala | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index e8d0678989d88..33f105892e482 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -46,7 +46,9 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } def isMaterialized(rddId: Int): Boolean = { - sparkContext.env.blockManager.get(RDDBlockId(rddId, 0)).nonEmpty + val maybeBlock = sparkContext.env.blockManager.get(RDDBlockId(rddId, 0)) + maybeBlock.foreach(_ => sparkContext.env.blockManager.release(RDDBlockId(rddId, 0))) + maybeBlock.nonEmpty } test("withColumn doesn't invalidate cached dataframe") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index 11863caffed75..5350d3a321292 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -40,7 +40,9 @@ class CachedTableSuite extends QueryTest with TestHiveSingleton { } def isMaterialized(rddId: Int): Boolean = { - sparkContext.env.blockManager.get(RDDBlockId(rddId, 0)).nonEmpty + val maybeBlock = sparkContext.env.blockManager.get(RDDBlockId(rddId, 0)) + maybeBlock.foreach(_ => sparkContext.env.blockManager.release(RDDBlockId(rddId, 0))) + maybeBlock.nonEmpty } test("cache table") { From 12ed0841b5d5cf171e9db9325bf9f61f3dd8046b Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 20 Jan 2016 18:05:53 -0800 Subject: [PATCH 09/81] Fix TaskResultGetterSuite. --- .../main/scala/org/apache/spark/scheduler/TaskResultGetter.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index f4965994d8277..9a2f49a263814 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -78,6 +78,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul } val deserializedResult = serializer.get().deserialize[DirectTaskResult[_]]( serializedTaskResult.get) + sparkEnv.blockManager.release(blockId) sparkEnv.blockManager.master.removeBlock(blockId) (deserializedResult, size) } From 1b18226db118de25f6ff836236d5f9b0877ea004 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 24 Jan 2016 18:59:20 -0800 Subject: [PATCH 10/81] Terminology update: reference -> pin. --- .../apache/spark/storage/BlockManager.scala | 21 +++--- .../apache/spark/storage/MemoryStore.scala | 5 +- ...eferenceCounter.scala => PinCounter.scala} | 64 +++++++++---------- .../spark/storage/BlockManagerSuite.scala | 2 +- ...unterSuite.scala => PinCounterSuite.scala} | 42 ++++++------ 5 files changed, 68 insertions(+), 66 deletions(-) rename core/src/main/scala/org/apache/spark/util/collection/{ReferenceCounter.scala => PinCounter.scala} (56%) rename core/src/test/scala/org/apache/spark/util/collection/{ReferenceCounterSuite.scala => PinCounterSuite.scala} (72%) 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 61dacd473162a..8bad500d3d9f9 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -43,7 +43,7 @@ import org.apache.spark.rpc.RpcEnv import org.apache.spark.serializer.{Serializer, SerializerInstance} import org.apache.spark.shuffle.ShuffleManager import org.apache.spark.util._ -import org.apache.spark.util.collection.ReferenceCounter +import org.apache.spark.util.collection.PinCounter private[spark] sealed trait BlockValues private[spark] case class ByteBufferValues(buffer: ByteBuffer) extends BlockValues @@ -157,7 +157,7 @@ private[spark] class BlockManager( * loaded yet. */ private lazy val compressionCodec: CompressionCodec = CompressionCodec.createCodec(conf) - private val referenceCounts = new ReferenceCounter[BlockId] + private val pinCounts = new PinCounter[BlockId] /** * Initializes the BlockManager with the given appId. This is not performed in the constructor as @@ -408,7 +408,7 @@ private[spark] class BlockManager( logDebug(s"Getting local block $blockId") val res = doGetLocal(blockId, asBlockResult = true).asInstanceOf[Option[BlockResult]] if (res.isDefined) { - referenceCounts.retain(blockId) + pinCounts.pin(blockId) } res } @@ -430,7 +430,7 @@ private[spark] class BlockManager( doGetLocal(blockId, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] } if (res.isDefined) { - referenceCounts.retain(blockId) + pinCounts.pin(blockId) } res } @@ -547,7 +547,7 @@ private[spark] class BlockManager( logDebug(s"Getting remote block $blockId") val res = doGetRemote(blockId, asBlockResult = true).asInstanceOf[Option[BlockResult]] if (res.isDefined) { - referenceCounts.retain(blockId) + pinCounts.pin(blockId) } res } @@ -559,7 +559,7 @@ private[spark] class BlockManager( logDebug(s"Getting remote block $blockId as bytes") val res = doGetRemote(blockId, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] if (res.isDefined) { - referenceCounts.retain(blockId) + pinCounts.pin(blockId) } res } @@ -635,15 +635,15 @@ private[spark] class BlockManager( * Release one reference to the given block. */ def release(blockId: BlockId): Unit = { - referenceCounts.release(blockId) + pinCounts.unpin(blockId) } def releaseAllReferencesForTask(taskAttemptId: Long): Unit = { - referenceCounts.releaseAllReferencesForTask(taskAttemptId) + pinCounts.releaseAllPinsForTask(taskAttemptId) } - private[storage] def getReferenceCount(blockId: BlockId): Int = { - referenceCounts.getReferenceCount(blockId) + private[storage] def getPinCount(blockId: BlockId): Int = { + pinCounts.getPinCount(blockId) } /** @@ -1088,6 +1088,7 @@ private[spark] class BlockManager( /** * Remove all blocks belonging to the given RDD. + * * @return The number of blocks removed. */ def removeRdd(rddId: Int): Int = { 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 b520f081c8b51..7fa8d7af91f3e 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -208,7 +208,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo } override def remove(blockId: BlockId): Boolean = memoryManager.synchronized { - val referenceCount = blockManager.getReferenceCount(blockId) + val referenceCount = blockManager.getPinCount(blockId) if (referenceCount != 0) { throw new IllegalStateException( s"Cannot free block $blockId since it is still referenced $referenceCount times") @@ -412,7 +412,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo val rddToAdd = blockId.flatMap(getRddId) val selectedBlocks = new ArrayBuffer[BlockId] def blockIsEvictable(blockId: BlockId): Boolean = { - blockManager.getReferenceCount(blockId) == 0 && + blockManager.getPinCount(blockId) == 0 && (rddToAdd.isEmpty || rddToAdd != getRddId(blockId)) } // This is synchronized to ensure that the set of entries is not changed @@ -468,6 +468,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo /** * Reserve memory for unrolling the given block for this task. + * * @return whether the request is granted. */ def reserveUnrollMemoryForThisTask(blockId: BlockId, memory: Long): Boolean = { diff --git a/core/src/main/scala/org/apache/spark/util/collection/ReferenceCounter.scala b/core/src/main/scala/org/apache/spark/util/collection/PinCounter.scala similarity index 56% rename from core/src/main/scala/org/apache/spark/util/collection/ReferenceCounter.scala rename to core/src/main/scala/org/apache/spark/util/collection/PinCounter.scala index 6823c1f9339ec..4c5accb6ed0b3 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ReferenceCounter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/PinCounter.scala @@ -25,21 +25,21 @@ import com.google.common.collect.ConcurrentHashMultiset import org.apache.spark.TaskContext /** - * Thread-safe collection for maintaining both global and per-task reference counts for objects. + * Thread-safe collection for maintaining both global and per-task pin counts for objects. */ -private[spark] class ReferenceCounter[T] { +private[spark] class PinCounter[T] { private type TaskAttemptId = Long /** - * Total references across all tasks. + * Total pins across all tasks. */ - private[this] val allReferences = ConcurrentHashMultiset.create[T]() + private[this] val allPins = ConcurrentHashMultiset.create[T]() /** - * Total references per task. Used to auto-release references upon task completion. + * Total pins per task. Used to auto-release pins upon task completion. */ - private[this] val referencesByTask = { + private[this] val pinsByTask = { // We need to explicitly box as java.lang.Long to avoid a type mismatch error: val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[T]] { override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[T]() @@ -48,61 +48,61 @@ private[spark] class ReferenceCounter[T] { } /** - * Returns the total reference count, across all tasks, for the given object. + * Returns the total pin count, across all tasks, for the given object. */ - def getReferenceCount(obj: T): Int = allReferences.count(obj) + def getPinCount(obj: T): Int = allPins.count(obj) /** - * Increments the given object's reference count for the current task. + * Increments the given object's pin count for the current task. */ - def retain(obj: T): Unit = retainForTask(currentTaskAttemptId, obj) + def pin(obj: T): Unit = retainForTask(currentTaskAttemptId, obj) /** - * Decrements the given object's reference count for the current task. + * Decrements the given object's pin count for the current task. */ - def release(obj: T): Unit = releaseForTask(currentTaskAttemptId, obj) + def unpin(obj: T): Unit = releaseForTask(currentTaskAttemptId, obj) private def currentTaskAttemptId: TaskAttemptId = { Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1L) } /** - * Increments the given object's reference count for the given task. + * Increments the given object's pin count for the given task. */ def retainForTask(taskAttemptId: TaskAttemptId, obj: T): Unit = { - referencesByTask.get(taskAttemptId).add(obj) - allReferences.add(obj) + pinsByTask.get(taskAttemptId).add(obj) + allPins.add(obj) } /** - * Decrements the given object's reference count for the given task. + * Decrements the given object's pin count for the given task. */ def releaseForTask(taskAttemptId: TaskAttemptId, obj: T): Unit = { - val countsForTask = referencesByTask.get(taskAttemptId) - val newReferenceCountForTask: Int = countsForTask.remove(obj, 1) - 1 - val newTotalReferenceCount: Int = allReferences.remove(obj, 1) - 1 - if (newReferenceCountForTask < 0) { + val countsForTask = pinsByTask.get(taskAttemptId) + val newPinCountForTask: Int = countsForTask.remove(obj, 1) - 1 + val newTotalPinCount: Int = allPins.remove(obj, 1) - 1 + if (newPinCountForTask < 0) { throw new IllegalStateException( s"Task $taskAttemptId released object $obj more times than it was retained") } - if (newTotalReferenceCount < 0) { + if (newTotalPinCount < 0) { throw new IllegalStateException( s"Task $taskAttemptId released object $obj more times than it was retained") } } /** - * Release all references held by the given task, clearing that task's reference bookkeeping - * structures and updating the global reference counts. This method should be called at the + * Release all pins held by the given task, clearing that task's pin bookkeeping + * structures and updating the global pin counts. This method should be called at the * end of a task (either by a task completion handler or in `TaskRunner.run()`). */ - def releaseAllReferencesForTask(taskAttemptId: TaskAttemptId): Unit = { - val referenceCounts = referencesByTask.get(taskAttemptId) - referencesByTask.invalidate(taskAttemptId) - referenceCounts.entrySet().iterator().asScala.foreach { entry => + def releaseAllPinsForTask(taskAttemptId: TaskAttemptId): Unit = { + val PinCounts = pinsByTask.get(taskAttemptId) + pinsByTask.invalidate(taskAttemptId) + PinCounts.entrySet().iterator().asScala.foreach { entry => val obj = entry.getElement val taskRefCount = entry.getCount - val newRefCount = allReferences.remove(obj, taskRefCount) - taskRefCount + val newRefCount = allPins.remove(obj, taskRefCount) - taskRefCount if (newRefCount < 0) { throw new IllegalStateException( s"Task $taskAttemptId released object $obj more times than it was retained") @@ -111,12 +111,12 @@ private[spark] class ReferenceCounter[T] { } /** - * Return the number of map entries in this reference counter's internal data structures. + * Return the number of map entries in this pin counter's internal data structures. * This is used in unit tests in order to detect memory leaks. */ private[collection] def getNumberOfMapEntries: Long = { - allReferences.size() + - referencesByTask.size() + - referencesByTask.asMap().asScala.map(_._2.size()).sum + allPins.size() + + pinsByTask.size() + + pinsByTask.asMap().asScala.map(_._2.size()).sum } } 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 8507917fb7e43..4f5ef30198c8e 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -1338,7 +1338,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // Release both references to block a2: store.release("a2") store.release("a2") - assert(store.getReferenceCount("a2") === 0) + assert(store.getPinCount("a2") === 0) // Block a1 is the least-recently accessed, so an LRU eviction policy would evict it before // block a2. However, a1 still has references, so this put of a3 should evict a2 instead: store.putSingle("a3", arr, StorageLevel.MEMORY_ONLY_SER) diff --git a/core/src/test/scala/org/apache/spark/util/collection/ReferenceCounterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/PinCounterSuite.scala similarity index 72% rename from core/src/test/scala/org/apache/spark/util/collection/ReferenceCounterSuite.scala rename to core/src/test/scala/org/apache/spark/util/collection/PinCounterSuite.scala index cbfe0bfdee7bc..87904df7d793f 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ReferenceCounterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/PinCounterSuite.scala @@ -22,13 +22,13 @@ import org.scalatest.BeforeAndAfterEach import org.apache.spark.SparkFunSuite import org.apache.spark.storage.{BlockId, TestBlockId} -class ReferenceCounterSuite extends SparkFunSuite with BeforeAndAfterEach { +class PinCounterSuite extends SparkFunSuite with BeforeAndAfterEach { - private var refCounter: ReferenceCounter[BlockId] = _ + private var refCounter: PinCounter[BlockId] = _ override protected def beforeEach(): Unit = { super.beforeEach() - refCounter = new ReferenceCounter() + refCounter = new PinCounter() } override protected def afterEach(): Unit = { @@ -36,9 +36,9 @@ class ReferenceCounterSuite extends SparkFunSuite with BeforeAndAfterEach { super.afterEach() } - test("initial reference counts are zero") { + test("initial pin counts are zero") { assert(refCounter.getNumberOfMapEntries === 0) - assert(refCounter.getReferenceCount(TestBlockId("dummy")) === 0) + assert(refCounter.getPinCount(TestBlockId("dummy")) === 0) assert(refCounter.getNumberOfMapEntries === 0) } @@ -52,13 +52,13 @@ class ReferenceCounterSuite extends SparkFunSuite with BeforeAndAfterEach { val block = TestBlockId("dummy") val taskAttemptId = 0L refCounter.retainForTask(taskAttemptId, block) - assert(refCounter.getReferenceCount(block) === 1) + assert(refCounter.getPinCount(block) === 1) refCounter.retainForTask(taskAttemptId, block) - assert(refCounter.getReferenceCount(block) === 2) + assert(refCounter.getPinCount(block) === 2) refCounter.releaseForTask(taskAttemptId, block) - assert(refCounter.getReferenceCount(block) === 1) - refCounter.releaseAllReferencesForTask(taskAttemptId) - assert(refCounter.getReferenceCount(block) === 0L) + assert(refCounter.getPinCount(block) === 1) + refCounter.releaseAllPinsForTask(taskAttemptId) + assert(refCounter.getPinCount(block) === 0L) // Ensure that we didn't leak memory / map entries: assert(refCounter.getNumberOfMapEntries === 0) } @@ -74,19 +74,19 @@ class ReferenceCounterSuite extends SparkFunSuite with BeforeAndAfterEach { refCounter.retainForTask(taskB, block) refCounter.retainForTask(taskA, block) - assert(refCounter.getReferenceCount(block) === 5) + assert(refCounter.getPinCount(block) === 5) refCounter.releaseForTask(taskA, block) - assert(refCounter.getReferenceCount(block) === 4) + assert(refCounter.getPinCount(block) === 4) - refCounter.releaseAllReferencesForTask(taskA) - assert(refCounter.getReferenceCount(block) === 2) + refCounter.releaseAllPinsForTask(taskA) + assert(refCounter.getPinCount(block) === 2) refCounter.releaseForTask(taskB, block) refCounter.releaseForTask(taskB, block) - assert(refCounter.getReferenceCount(block) === 0) + assert(refCounter.getPinCount(block) === 0) - refCounter.releaseAllReferencesForTask(taskB) + refCounter.releaseAllPinsForTask(taskB) // Ensure that we didn't leak memory / map entries: assert(refCounter.getNumberOfMapEntries === 0) @@ -98,16 +98,16 @@ class ReferenceCounterSuite extends SparkFunSuite with BeforeAndAfterEach { val taskAttemptId = 0L refCounter.retainForTask(taskAttemptId, blockA) - assert(refCounter.getReferenceCount(blockA) === 1) - assert(refCounter.getReferenceCount(blockB) === 0) + assert(refCounter.getPinCount(blockA) === 1) + assert(refCounter.getPinCount(blockB) === 0) refCounter.retainForTask(taskAttemptId, blockB) refCounter.retainForTask(taskAttemptId, blockB) - assert(refCounter.getReferenceCount(blockA) === 1) - assert(refCounter.getReferenceCount(blockB) === 2) + assert(refCounter.getPinCount(blockA) === 1) + assert(refCounter.getPinCount(blockB) === 2) // Ensure that we didn't leak memory / map entries: - refCounter.releaseAllReferencesForTask(taskAttemptId) + refCounter.releaseAllPinsForTask(taskAttemptId) assert(refCounter.getNumberOfMapEntries === 0) } From 8d45da608a5b8d72259b2a275271f2740b1ebdec Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 24 Jan 2016 19:03:51 -0800 Subject: [PATCH 11/81] More terminology updates. --- .../org/apache/spark/executor/Executor.scala | 2 +- .../spark/scheduler/TaskResultGetter.scala | 2 +- .../apache/spark/storage/BlockManager.scala | 6 +- .../BlockManagerReplicationSuite.scala | 2 +- .../spark/storage/BlockManagerSuite.scala | 118 +++++++++--------- .../apache/spark/sql/CachedTableSuite.scala | 2 +- .../spark/sql/hive/CachedTableSuite.scala | 2 +- 7 files changed, 67 insertions(+), 67 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 f1018f2c8b4ec..b8c8f88b1e5ee 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -218,7 +218,7 @@ private[spark] class Executor( threwException = false res } finally { - env.blockManager.releaseAllReferencesForTask(taskId) + env.blockManager.releaseAllPinsForTask(taskId) val freedMemory = taskMemoryManager.cleanUpAllAllocatedMemory() if (freedMemory > 0) { val errMsg = s"Managed memory leak detected; size = $freedMemory bytes, TID = $taskId" diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 9a2f49a263814..53412cecff751 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -78,7 +78,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul } val deserializedResult = serializer.get().deserialize[DirectTaskResult[_]]( serializedTaskResult.get) - sparkEnv.blockManager.release(blockId) + sparkEnv.blockManager.unpin(blockId) sparkEnv.blockManager.master.removeBlock(blockId) (deserializedResult, size) } 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 8bad500d3d9f9..1b76c7405ef91 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -632,13 +632,13 @@ private[spark] class BlockManager( } /** - * Release one reference to the given block. + * Release one pin of the given block. */ - def release(blockId: BlockId): Unit = { + def unpin(blockId: BlockId): Unit = { pinCounts.unpin(blockId) } - def releaseAllReferencesForTask(taskAttemptId: Long): Unit = { + def releaseAllPinsForTask(taskAttemptId: Long): Unit = { pinCounts.releaseAllPinsForTask(taskAttemptId) } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala index f5758a4a51c57..63e8b95941164 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -367,7 +367,7 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo }.foreach { testStore => val testStoreName = testStore.blockManagerId.executorId assert(testStore.getLocal(blockId).isDefined, s"$blockId was not found in $testStoreName") - testStore.release(blockId) + testStore.unpin(blockId) assert(master.getLocations(blockId).map(_.executorId).toSet.contains(testStoreName), s"master does not have status for ${blockId.name} in $testStoreName") 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 4f5ef30198c8e..6eab0c14af0b6 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -176,11 +176,11 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // Checking whether blocks are in memory assert(store.getSingle("a1").isDefined, "a1 was not in store") - store.release("a1") + store.unpin("a1") assert(store.getSingle("a2").isDefined, "a2 was not in store") - store.release("a2") + store.unpin("a2") assert(store.getSingle("a3").isDefined, "a3 was not in store") - store.release("a3") + store.unpin("a3") // Checking whether master knows about the blocks or not assert(master.getLocations("a1").size > 0, "master was not told about a1") @@ -228,11 +228,11 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE 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") - store.release("a1-to-remove") + store.unpin("a1-to-remove") assert(store.getSingle("a2-to-remove").isDefined, "a2 was not in store") - store.release("a2-to-remove") + store.unpin("a2-to-remove") assert(store.getSingle("a3-to-remove").isDefined, "a3 was not in store") - store.release("a3-to-remove") + store.unpin("a3-to-remove") // Checking whether master knows about the blocks or not assert(master.getLocations("a1-to-remove").size > 0, "master was not told about a1") @@ -321,13 +321,13 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // verify whether the blocks exist in both the stores Seq(driverStore, executorStore).foreach { case s => s.getLocal(broadcast0BlockId) should not be (None) - s.release(broadcast0BlockId) + s.unpin(broadcast0BlockId) s.getLocal(broadcast1BlockId) should not be (None) - s.release(broadcast1BlockId) + s.unpin(broadcast1BlockId) s.getLocal(broadcast2BlockId) should not be (None) - s.release(broadcast2BlockId) + s.unpin(broadcast2BlockId) s.getLocal(broadcast2BlockId2) should not be (None) - s.release(broadcast2BlockId2) + s.unpin(broadcast2BlockId2) } // remove broadcast 0 block only from executors @@ -336,23 +336,23 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // only broadcast 0 block should be removed from the executor store executorStore.getLocal(broadcast0BlockId) should be (None) executorStore.getLocal(broadcast1BlockId) should not be (None) - executorStore.release(broadcast1BlockId) + executorStore.unpin(broadcast1BlockId) executorStore.getLocal(broadcast2BlockId) should not be (None) - executorStore.release(broadcast2BlockId) + executorStore.unpin(broadcast2BlockId) // nothing should be removed from the driver store driverStore.getLocal(broadcast0BlockId) should not be (None) - driverStore.release(broadcast0BlockId) + driverStore.unpin(broadcast0BlockId) driverStore.getLocal(broadcast1BlockId) should not be (None) - driverStore.release(broadcast1BlockId) + driverStore.unpin(broadcast1BlockId) driverStore.getLocal(broadcast2BlockId) should not be (None) - driverStore.release(broadcast2BlockId) + driverStore.unpin(broadcast2BlockId) // remove broadcast 0 block from the driver as well master.removeBroadcast(0, removeFromMaster = true, blocking = true) driverStore.getLocal(broadcast0BlockId) should be (None) driverStore.getLocal(broadcast1BlockId) should not be (None) - driverStore.release(broadcast1BlockId) + driverStore.unpin(broadcast1BlockId) // remove broadcast 1 block from both the stores asynchronously // and verify all broadcast 1 blocks have been removed @@ -527,12 +527,12 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE store.putSingle("a2", a2, storageLevel) store.putSingle("a3", a3, storageLevel) assert(store.getSingle("a2").isDefined, "a2 was not in store") - store.release("a2") + store.unpin("a2") assert(store.getSingle("a3").isDefined, "a3 was not in store") - store.release("a3") + store.unpin("a3") assert(store.getSingle("a1") === None, "a1 was in store") assert(store.getSingle("a2").isDefined, "a2 was not in store") - store.release("a2") + store.unpin("a2") // At this point a2 was gotten last, so LRU will getSingle rid of a3 store.putSingle("a1", a1, storageLevel) assert(store.getSingle("a1").isDefined, "a1 was not in store") @@ -622,12 +622,12 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE store.putSingle("a2", a2, storageLevel) store.putSingle("a3", a3, storageLevel) assert(accessMethod(store)("a2").isDefined, "a2 was not in store") - store.release("a2") + store.unpin("a2") assert(accessMethod(store)("a3").isDefined, "a3 was not in store") - store.release("a3") + store.unpin("a3") assert(store.memoryStore.getValues("a1").isEmpty, "a1 was in memory store") assert(accessMethod(store)("a1").isDefined, "a1 was not in store") - store.release("a1") + store.unpin("a1") assert(store.memoryStore.getValues("a1").isDefined, "a1 was not in memory store") } @@ -643,20 +643,20 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE store.putSingle("a3", a3, StorageLevel.DISK_ONLY) // At this point LRU should not kick in because a3 is only on disk assert(store.getSingle("a1").isDefined, "a1 was not in store") - store.release("a1") + store.unpin("a1") assert(store.getSingle("a2").isDefined, "a2 was not in store") - store.release("a2") + store.unpin("a2") assert(store.getSingle("a3").isDefined, "a3 was not in store") - store.release("a3") + store.unpin("a3") // Now let's add in a4, which uses both disk and memory; a1 should drop out store.putSingle("a4", a4, StorageLevel.MEMORY_AND_DISK_SER) assert(store.getSingle("a1") == None, "a1 was in store") assert(store.getSingle("a2").isDefined, "a2 was not in store") - store.release("a2") + store.unpin("a2") assert(store.getSingle("a3").isDefined, "a3 was not in store") - store.release("a3") + store.unpin("a3") assert(store.getSingle("a4").isDefined, "a4 was not in store") - store.release("a4") + store.unpin("a4") } test("in-memory LRU with streams") { @@ -669,27 +669,27 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE 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) - store.release("list2") - store.release("list2") + store.unpin("list2") + store.unpin("list2") assert(store.get("list3").isDefined, "list3 was not in store") assert(store.get("list3").get.data.size === 2) - store.release("list3") - store.release("list3") + store.unpin("list3") + store.unpin("list3") 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) - store.release("list2") - store.release("list2") + store.unpin("list2") + store.unpin("list2") // At this point list2 was gotten last, so LRU will getSingle rid of list3 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) - store.release("list1") - store.release("list1") + store.unpin("list1") + store.unpin("list1") assert(store.get("list2").isDefined, "list2 was not in store") assert(store.get("list2").get.data.size === 2) - store.release("list2") - store.release("list2") + store.unpin("list2") + store.unpin("list2") assert(store.get("list3") === None, "list1 was in store") } @@ -709,43 +709,43 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // At this point LRU should not kick in because list3 is only on disk assert(store.get("list1").isDefined, "list1 was not in store") assert(store.get("list1").get.data.size === 2) - store.release("list1") - store.release("list1") + store.unpin("list1") + store.unpin("list1") assert(store.get("list2").isDefined, "list2 was not in store") assert(store.get("list2").get.data.size === 2) - store.release("list2") - store.release("list2") + store.unpin("list2") + store.unpin("list2") assert(store.get("list3").isDefined, "list3 was not in store") assert(store.get("list3").get.data.size === 2) - store.release("list3") - store.release("list3") + store.unpin("list3") + store.unpin("list3") assert(store.get("list1").isDefined, "list1 was not in store") assert(store.get("list1").get.data.size === 2) - store.release("list1") - store.release("list1") + store.unpin("list1") + store.unpin("list1") assert(store.get("list2").isDefined, "list2 was not in store") assert(store.get("list2").get.data.size === 2) - store.release("list2") - store.release("list2") + store.unpin("list2") + store.unpin("list2") assert(store.get("list3").isDefined, "list3 was not in store") assert(store.get("list3").get.data.size === 2) - store.release("list3") - store.release("list3") + store.unpin("list3") + store.unpin("list3") // Now let's add in list4, which uses both disk and memory; list1 should drop out 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) - store.release("list2") - store.release("list2") + store.unpin("list2") + store.unpin("list2") assert(store.get("list3").isDefined, "list3 was not in store") assert(store.get("list3").get.data.size === 2) - store.release("list3") - store.release("list3") + store.unpin("list3") + store.unpin("list3") assert(store.get("list4").isDefined, "list4 was not in store") assert(store.get("list4").get.data.size === 2) - store.release("list4") - store.release("list4") + store.unpin("list4") + store.unpin("list4") } test("negative byte values in ByteBufferInputStream") { @@ -1063,7 +1063,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE 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") - store.release(rdd(1, 0)) + store.unpin(rdd(1, 0)) // According to the same-RDD rule, rdd_1_0 should be replaced here. 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. @@ -1336,8 +1336,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(store.getSingle("a1").isDefined, "a1 was not in store") assert(store.getSingle("a2").isDefined, "a2 was not in store") // Release both references to block a2: - store.release("a2") - store.release("a2") + store.unpin("a2") + store.unpin("a2") assert(store.getPinCount("a2") === 0) // Block a1 is the least-recently accessed, so an LRU eviction policy would evict it before // block a2. However, a1 still has references, so this put of a3 should evict a2 instead: diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 33f105892e482..5d62460dd0b1f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -47,7 +47,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext def isMaterialized(rddId: Int): Boolean = { val maybeBlock = sparkContext.env.blockManager.get(RDDBlockId(rddId, 0)) - maybeBlock.foreach(_ => sparkContext.env.blockManager.release(RDDBlockId(rddId, 0))) + maybeBlock.foreach(_ => sparkContext.env.blockManager.unpin(RDDBlockId(rddId, 0))) maybeBlock.nonEmpty } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index 5350d3a321292..694df185f19e1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -41,7 +41,7 @@ class CachedTableSuite extends QueryTest with TestHiveSingleton { def isMaterialized(rddId: Int): Boolean = { val maybeBlock = sparkContext.env.blockManager.get(RDDBlockId(rddId, 0)) - maybeBlock.foreach(_ => sparkContext.env.blockManager.release(RDDBlockId(rddId, 0))) + maybeBlock.foreach(_ => sparkContext.env.blockManager.unpin(RDDBlockId(rddId, 0))) maybeBlock.nonEmpty } From 8a52f58ef1b6657a371b4301909228f16fa4952a Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 24 Jan 2016 19:40:36 -0800 Subject: [PATCH 12/81] Fix flaky BlockManagerSuite test: There was a race between the remove() call and the check which tested whether the removal succeeded; if the check arrived before the async call then its implicit pin might prevent block eviction. --- .../apache/spark/storage/BlockManager.scala | 9 ++++ .../spark/storage/BlockManagerSuite.scala | 46 ++++++++----------- 2 files changed, 27 insertions(+), 28 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 1b76c7405ef91..de6914a742cfa 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -413,6 +413,15 @@ private[spark] class BlockManager( res } + /** + * Return true if local block manager contains the given block and false otherwise. + * Calling this method will _not_ implicitly pin the block, making this method useful + * when writing tests. + */ + def hasLocalBlock(blockId: BlockId): Boolean = { + doGetLocal(blockId, asBlockResult = true).asInstanceOf[Option[BlockResult]].isDefined + } + /** * Get block from the local block manager as serialized bytes. */ 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 6eab0c14af0b6..b06b0b84696d3 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -320,56 +320,46 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // verify whether the blocks exist in both the stores Seq(driverStore, executorStore).foreach { case s => - s.getLocal(broadcast0BlockId) should not be (None) - s.unpin(broadcast0BlockId) - s.getLocal(broadcast1BlockId) should not be (None) - s.unpin(broadcast1BlockId) - s.getLocal(broadcast2BlockId) should not be (None) - s.unpin(broadcast2BlockId) - s.getLocal(broadcast2BlockId2) should not be (None) - s.unpin(broadcast2BlockId2) + assert(s.hasLocalBlock(broadcast0BlockId)) + assert(s.hasLocalBlock(broadcast1BlockId)) + assert(s.hasLocalBlock(broadcast2BlockId)) + assert(s.hasLocalBlock(broadcast2BlockId2)) } // remove broadcast 0 block only from executors master.removeBroadcast(0, removeFromMaster = false, blocking = true) // only broadcast 0 block should be removed from the executor store - executorStore.getLocal(broadcast0BlockId) should be (None) - executorStore.getLocal(broadcast1BlockId) should not be (None) - executorStore.unpin(broadcast1BlockId) - executorStore.getLocal(broadcast2BlockId) should not be (None) - executorStore.unpin(broadcast2BlockId) + assert(!executorStore.hasLocalBlock(broadcast0BlockId)) + assert(executorStore.hasLocalBlock(broadcast1BlockId)) + assert(executorStore.hasLocalBlock(broadcast2BlockId)) // nothing should be removed from the driver store - driverStore.getLocal(broadcast0BlockId) should not be (None) - driverStore.unpin(broadcast0BlockId) - driverStore.getLocal(broadcast1BlockId) should not be (None) - driverStore.unpin(broadcast1BlockId) - driverStore.getLocal(broadcast2BlockId) should not be (None) - driverStore.unpin(broadcast2BlockId) + assert(driverStore.hasLocalBlock(broadcast0BlockId)) + assert(driverStore.hasLocalBlock(broadcast1BlockId)) + assert(driverStore.hasLocalBlock(broadcast2BlockId)) // remove broadcast 0 block from the driver as well master.removeBroadcast(0, removeFromMaster = true, blocking = true) - driverStore.getLocal(broadcast0BlockId) should be (None) - driverStore.getLocal(broadcast1BlockId) should not be (None) - driverStore.unpin(broadcast1BlockId) + assert(!driverStore.hasLocalBlock(broadcast0BlockId)) + assert(driverStore.hasLocalBlock(broadcast1BlockId)) // remove broadcast 1 block from both the stores asynchronously // and verify all broadcast 1 blocks have been removed master.removeBroadcast(1, removeFromMaster = true, blocking = false) eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { - driverStore.getLocal(broadcast1BlockId) should be (None) - executorStore.getLocal(broadcast1BlockId) should be (None) + assert(!driverStore.hasLocalBlock(broadcast1BlockId)) + assert(!executorStore.hasLocalBlock(broadcast1BlockId)) } // remove broadcast 2 from both the stores asynchronously // and verify all broadcast 2 blocks have been removed master.removeBroadcast(2, removeFromMaster = true, blocking = false) eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { - driverStore.getLocal(broadcast2BlockId) should be (None) - driverStore.getLocal(broadcast2BlockId2) should be (None) - executorStore.getLocal(broadcast2BlockId) should be (None) - executorStore.getLocal(broadcast2BlockId2) should be (None) + assert(!driverStore.hasLocalBlock(broadcast2BlockId)) + assert(!driverStore.hasLocalBlock(broadcast2BlockId2)) + assert(!executorStore.hasLocalBlock(broadcast2BlockId)) + assert(!executorStore.hasLocalBlock(broadcast2BlockId2)) } executorStore.stop() driverStore.stop() From 36253dfe6879023562e404d9daf0b2c0c364e718 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 24 Jan 2016 19:51:04 -0800 Subject: [PATCH 13/81] Update very last occurrences of old terminology. --- .../main/scala/org/apache/spark/storage/MemoryStore.scala | 8 ++++---- .../org/apache/spark/storage/BlockManagerSuite.scala | 8 ++++---- 2 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 7fa8d7af91f3e..29b9a48bec897 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -208,10 +208,10 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo } override def remove(blockId: BlockId): Boolean = memoryManager.synchronized { - val referenceCount = blockManager.getPinCount(blockId) - if (referenceCount != 0) { + val pinCount = blockManager.getPinCount(blockId) + if (pinCount != 0) { throw new IllegalStateException( - s"Cannot free block $blockId since it is still referenced $referenceCount times") + s"Cannot free block $blockId since it is still pinned $pinCount times") } val entry = entries.synchronized { entries.remove(blockId) } if (entry != null) { @@ -468,7 +468,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo /** * Reserve memory for unrolling the given block for this task. - * + * * @return whether the request is granted. */ def reserveUnrollMemoryForThisTask(blockId: BlockId, memory: Long): Boolean = { 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 b06b0b84696d3..454093f9033ee 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -1312,7 +1312,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(result.data === Right(bytes)) } - test("blocks with non-zero reference counts cannot be evicted from the MemoryStore") { + test("blocks with non-zero pin counts cannot be evicted from the MemoryStore") { store = makeBlockManager(12000) val arr = new Array[Byte](4000) // First store a1 and a2, both in memory, and a3, on disk only @@ -1320,17 +1320,17 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE store.putSingle("a2", arr, StorageLevel.MEMORY_ONLY_SER) assert(store.getSingle("a1").isDefined, "a1 was not in store") assert(store.getSingle("a2").isDefined, "a2 was not in store") - // This put should fail because both a1 and a2 have non-zero reference counts: + // This put should fail because both a1 and a2 have non-zero pin counts: store.putSingle("a3", arr, StorageLevel.MEMORY_ONLY_SER) assert(store.getSingle("a3").isEmpty, "a3 was in store") assert(store.getSingle("a1").isDefined, "a1 was not in store") assert(store.getSingle("a2").isDefined, "a2 was not in store") - // Release both references to block a2: + // Release both pins of block a2: store.unpin("a2") store.unpin("a2") assert(store.getPinCount("a2") === 0) // Block a1 is the least-recently accessed, so an LRU eviction policy would evict it before - // block a2. However, a1 still has references, so this put of a3 should evict a2 instead: + // block a2. However, a1 is still pinned so this put of a3 should evict a2 instead: store.putSingle("a3", arr, StorageLevel.MEMORY_ONLY_SER) assert(store.getSingle("a2").isEmpty, "a2 was in store") assert(store.getSingle("a1").isDefined, "a1 was not in store") From 77d8c5caff50ce462b458bd440474fd5f3234966 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 25 Jan 2016 10:21:58 -0800 Subject: [PATCH 14/81] More test flakiness fixes --- .../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 454093f9033ee..b1890f34c538c 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -245,15 +245,15 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE master.removeBlock("a3-to-remove") eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { - store.getSingle("a1-to-remove") should be (None) + assert(!store.hasLocalBlock("a1-to-remove")) master.getLocations("a1-to-remove") should have size 0 } eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { - store.getSingle("a2-to-remove") should be (None) + assert(!store.hasLocalBlock("a2-to-remove")) master.getLocations("a2-to-remove") should have size 0 } eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { - store.getSingle("a3-to-remove") should not be (None) + assert(store.hasLocalBlock("a3-to-remove")) master.getLocations("a3-to-remove") should have size 0 } eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { From 2cf815730c7bdb44bf7f51e68e105e9e772e213b Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 26 Jan 2016 16:19:49 -0800 Subject: [PATCH 15/81] Detect leaked pins at end of tasks. --- .../scala/org/apache/spark/executor/Executor.scala | 12 +++++++++++- .../org/apache/spark/storage/BlockManager.scala | 6 +++++- .../apache/spark/util/collection/PinCounter.scala | 10 +++++++--- pom.xml | 2 ++ project/SparkBuild.scala | 1 + 5 files changed, 26 insertions(+), 5 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 b8c8f88b1e5ee..e05e6d1565478 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -218,8 +218,9 @@ private[spark] class Executor( threwException = false res } finally { - env.blockManager.releaseAllPinsForTask(taskId) + val releasedPins = env.blockManager.releaseAllPinsForTask(taskId) val freedMemory = taskMemoryManager.cleanUpAllAllocatedMemory() + if (freedMemory > 0) { val errMsg = s"Managed memory leak detected; size = $freedMemory bytes, TID = $taskId" if (conf.getBoolean("spark.unsafe.exceptionOnMemoryLeak", false) && !threwException) { @@ -228,6 +229,15 @@ private[spark] class Executor( logError(errMsg) } } + + if (releasedPins > 0) { + val errMsg = s"$releasedPins block pins were not released, TID = $taskId" + if (conf.getBoolean("spark.storage.exceptionOnPinLeak", false) && !threwException) { + throw new SparkException(errMsg) + } else { + logError(errMsg) + } + } } val taskFinish = System.currentTimeMillis() 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 de6914a742cfa..a6c2704c4b8b2 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -647,7 +647,11 @@ private[spark] class BlockManager( pinCounts.unpin(blockId) } - def releaseAllPinsForTask(taskAttemptId: Long): Unit = { + /** + * Release all pins for the given task. + * @return the total number of pins released. + */ + def releaseAllPinsForTask(taskAttemptId: Long): Int = { pinCounts.releaseAllPinsForTask(taskAttemptId) } diff --git a/core/src/main/scala/org/apache/spark/util/collection/PinCounter.scala b/core/src/main/scala/org/apache/spark/util/collection/PinCounter.scala index 4c5accb6ed0b3..038287ef1a53c 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/PinCounter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/PinCounter.scala @@ -95,11 +95,14 @@ private[spark] class PinCounter[T] { * Release all pins held by the given task, clearing that task's pin bookkeeping * structures and updating the global pin counts. This method should be called at the * end of a task (either by a task completion handler or in `TaskRunner.run()`). + * + * @return the number of pins released */ - def releaseAllPinsForTask(taskAttemptId: TaskAttemptId): Unit = { - val PinCounts = pinsByTask.get(taskAttemptId) + def releaseAllPinsForTask(taskAttemptId: TaskAttemptId): Int = { + val pinCounts = pinsByTask.get(taskAttemptId) pinsByTask.invalidate(taskAttemptId) - PinCounts.entrySet().iterator().asScala.foreach { entry => + val totalPinCountForTask = pinCounts.size() + pinCounts.entrySet().iterator().asScala.foreach { entry => val obj = entry.getElement val taskRefCount = entry.getCount val newRefCount = allPins.remove(obj, taskRefCount) - taskRefCount @@ -108,6 +111,7 @@ private[spark] class PinCounter[T] { s"Task $taskAttemptId released object $obj more times than it was retained") } } + totalPinCountForTask } /** diff --git a/pom.xml b/pom.xml index fb7750602c425..c72c3fb054082 100644 --- a/pom.xml +++ b/pom.xml @@ -1935,6 +1935,7 @@ false false true + true src @@ -1973,6 +1974,7 @@ false false true + true __not_used__ diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 4224a65a822b8..d7e8142041f88 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -730,6 +730,7 @@ object TestSettings { javaOptions in Test += "-Dspark.ui.enabled=false", javaOptions in Test += "-Dspark.ui.showConsoleProgress=false", javaOptions in Test += "-Dspark.unsafe.exceptionOnMemoryLeak=true", + javaOptions in Test += "-Dspark.storage.exceptionOnPinLeak=true", javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true", javaOptions in Test += "-Dderby.system.durability=test", javaOptions in Test ++= System.getProperties.asScala.filter(_._1.startsWith("spark")) From 150c6e1d390d6d0346817b31ddd0911c82e72baf Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 26 Jan 2016 16:56:27 -0800 Subject: [PATCH 16/81] Add unpin calls in more places. --- .../main/scala/org/apache/spark/CacheManager.scala | 13 ++++++++++--- .../apache/spark/broadcast/TorrentBroadcast.scala | 12 +++++++++++- 2 files changed, 21 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index fa8e2b953835b..e7c83d415d543 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -21,6 +21,7 @@ import scala.collection.mutable import org.apache.spark.rdd.RDD import org.apache.spark.storage._ +import org.apache.spark.util.CompletionIterator /** * Spark class responsible for passing RDDs partition contents to the BlockManager and making @@ -46,7 +47,9 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { val existingMetrics = context.taskMetrics().registerInputMetrics(blockResult.readMethod) existingMetrics.incBytesRead(blockResult.bytes) - val iter = blockResult.data.asInstanceOf[Iterator[T]] + val iter = CompletionIterator[T, Iterator[T]]( + blockResult.data.asInstanceOf[Iterator[T]], + blockManager.unpin(key)) new InterruptibleIterator[T](context, iter) { override def next(): T = { existingMetrics.incRecordsRead(1) @@ -58,7 +61,8 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { // If another thread already holds the lock, wait for it to finish return its results val storedValues = acquireLockForPartition[T](key) if (storedValues.isDefined) { - return new InterruptibleIterator[T](context, storedValues.get) + val iter = CompletionIterator[T, Iterator[T]](storedValues.get, blockManager.unpin(key)) + return new InterruptibleIterator[T](context, iter) } // Otherwise, we have to load the partition ourselves @@ -136,7 +140,10 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { */ blockManager.putIterator(key, values, level, tellMaster = true, effectiveStorageLevel) blockManager.get(key) match { - case Some(v) => v.data.asInstanceOf[Iterator[T]] + case Some(v) => + CompletionIterator[T, Iterator[T]]( + v.data.asInstanceOf[Iterator[T]], + blockManager.unpin(key)) case None => logInfo(s"Failure to store $key") throw new BlockException(key, s"Block manager failed to return cached value for $key!") diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 9bd69727f6086..6a30b768e91b1 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._ import scala.reflect.ClassTag import scala.util.Random -import org.apache.spark.{Logging, SparkConf, SparkEnv, SparkException} +import org.apache.spark._ import org.apache.spark.io.CompressionCodec import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} @@ -137,6 +137,11 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) val block: ByteBuffer = getLocal.orElse(getRemote).getOrElse( throw new SparkException(s"Failed to get $pieceId of $broadcastId")) blocks(pid) = block + // On the driver broadcast variables may be loaded when computing rdd.partitions(), which + // takes place outside of the context of a task, so we need to use an option here: + Option(TaskContext.get()).foreach { taskContext => + taskContext.addTaskCompletionListener(_ => SparkEnv.get.blockManager.unpin(pieceId)) + } } blocks } @@ -167,6 +172,11 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) setConf(SparkEnv.get.conf) SparkEnv.get.blockManager.getLocal(broadcastId).map(_.data.next()) match { case Some(x) => + // On the driver broadcast variables may be loaded when computing rdd.partitions(), which + // takes place outside of the context of a task, so we need to use an option here: + Option(TaskContext.get()).foreach { taskContext => + taskContext.addTaskCompletionListener(_ => SparkEnv.get.blockManager.unpin(broadcastId)) + } x.asInstanceOf[T] case None => From 1828757755f0461cf4011f0d0aaba0441ab4a026 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 2 Feb 2016 14:18:09 -0800 Subject: [PATCH 17/81] Disable leak detection in tests for now. --- pom.xml | 2 -- project/SparkBuild.scala | 1 - 2 files changed, 3 deletions(-) diff --git a/pom.xml b/pom.xml index 38da486a8db4d..d0387aca66d0d 100644 --- a/pom.xml +++ b/pom.xml @@ -1935,7 +1935,6 @@ false false true - true src @@ -1974,7 +1973,6 @@ false false true - true __not_used__ diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index ec3b694f2c7e1..550b5bad8a46a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -730,7 +730,6 @@ object TestSettings { javaOptions in Test += "-Dspark.ui.enabled=false", javaOptions in Test += "-Dspark.ui.showConsoleProgress=false", javaOptions in Test += "-Dspark.unsafe.exceptionOnMemoryLeak=true", - javaOptions in Test += "-Dspark.storage.exceptionOnPinLeak=true", javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true", javaOptions in Test += "-Dderby.system.durability=test", javaOptions in Test ++= System.getProperties.asScala.filter(_._1.startsWith("spark")) From 76fc9f5580b384d6f9fc990078f7f080bf298ada Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 2 Feb 2016 14:28:06 -0800 Subject: [PATCH 18/81] More renaming. --- .../spark/util/collection/PinCounter.scala | 8 ++--- .../util/collection/PinCounterSuite.scala | 30 +++++++++---------- 2 files changed, 19 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/PinCounter.scala b/core/src/main/scala/org/apache/spark/util/collection/PinCounter.scala index 038287ef1a53c..7165ce24f7e06 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/PinCounter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/PinCounter.scala @@ -55,12 +55,12 @@ private[spark] class PinCounter[T] { /** * Increments the given object's pin count for the current task. */ - def pin(obj: T): Unit = retainForTask(currentTaskAttemptId, obj) + def pin(obj: T): Unit = pinForTask(currentTaskAttemptId, obj) /** * Decrements the given object's pin count for the current task. */ - def unpin(obj: T): Unit = releaseForTask(currentTaskAttemptId, obj) + def unpin(obj: T): Unit = unpinForTask(currentTaskAttemptId, obj) private def currentTaskAttemptId: TaskAttemptId = { Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1L) @@ -69,7 +69,7 @@ private[spark] class PinCounter[T] { /** * Increments the given object's pin count for the given task. */ - def retainForTask(taskAttemptId: TaskAttemptId, obj: T): Unit = { + def pinForTask(taskAttemptId: TaskAttemptId, obj: T): Unit = { pinsByTask.get(taskAttemptId).add(obj) allPins.add(obj) } @@ -77,7 +77,7 @@ private[spark] class PinCounter[T] { /** * Decrements the given object's pin count for the given task. */ - def releaseForTask(taskAttemptId: TaskAttemptId, obj: T): Unit = { + def unpinForTask(taskAttemptId: TaskAttemptId, obj: T): Unit = { val countsForTask = pinsByTask.get(taskAttemptId) val newPinCountForTask: Int = countsForTask.remove(obj, 1) - 1 val newTotalPinCount: Int = allPins.remove(obj, 1) - 1 diff --git a/core/src/test/scala/org/apache/spark/util/collection/PinCounterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/PinCounterSuite.scala index 87904df7d793f..ee3b5ab741fcc 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/PinCounterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/PinCounterSuite.scala @@ -44,18 +44,18 @@ class PinCounterSuite extends SparkFunSuite with BeforeAndAfterEach { test("error when releasing more times than retained") { intercept[IllegalStateException] { - refCounter.releaseForTask(taskAttemptId = 0L, TestBlockId("dummy")) + refCounter.unpinForTask(taskAttemptId = 0L, TestBlockId("dummy")) } } test("retain and release from a single task") { val block = TestBlockId("dummy") val taskAttemptId = 0L - refCounter.retainForTask(taskAttemptId, block) + refCounter.pinForTask(taskAttemptId, block) assert(refCounter.getPinCount(block) === 1) - refCounter.retainForTask(taskAttemptId, block) + refCounter.pinForTask(taskAttemptId, block) assert(refCounter.getPinCount(block) === 2) - refCounter.releaseForTask(taskAttemptId, block) + refCounter.unpinForTask(taskAttemptId, block) assert(refCounter.getPinCount(block) === 1) refCounter.releaseAllPinsForTask(taskAttemptId) assert(refCounter.getPinCount(block) === 0L) @@ -68,22 +68,22 @@ class PinCounterSuite extends SparkFunSuite with BeforeAndAfterEach { val taskA = 0L val taskB = 1L - refCounter.retainForTask(taskA, block) - refCounter.retainForTask(taskA, block) - refCounter.retainForTask(taskB, block) - refCounter.retainForTask(taskB, block) - refCounter.retainForTask(taskA, block) + refCounter.pinForTask(taskA, block) + refCounter.pinForTask(taskA, block) + refCounter.pinForTask(taskB, block) + refCounter.pinForTask(taskB, block) + refCounter.pinForTask(taskA, block) assert(refCounter.getPinCount(block) === 5) - refCounter.releaseForTask(taskA, block) + refCounter.unpinForTask(taskA, block) assert(refCounter.getPinCount(block) === 4) refCounter.releaseAllPinsForTask(taskA) assert(refCounter.getPinCount(block) === 2) - refCounter.releaseForTask(taskB, block) - refCounter.releaseForTask(taskB, block) + refCounter.unpinForTask(taskB, block) + refCounter.unpinForTask(taskB, block) assert(refCounter.getPinCount(block) === 0) refCounter.releaseAllPinsForTask(taskB) @@ -97,12 +97,12 @@ class PinCounterSuite extends SparkFunSuite with BeforeAndAfterEach { val blockB = TestBlockId("blockB") val taskAttemptId = 0L - refCounter.retainForTask(taskAttemptId, blockA) + refCounter.pinForTask(taskAttemptId, blockA) assert(refCounter.getPinCount(blockA) === 1) assert(refCounter.getPinCount(blockB) === 0) - refCounter.retainForTask(taskAttemptId, blockB) - refCounter.retainForTask(taskAttemptId, blockB) + refCounter.pinForTask(taskAttemptId, blockB) + refCounter.pinForTask(taskAttemptId, blockB) assert(refCounter.getPinCount(blockA) === 1) assert(refCounter.getPinCount(blockB) === 2) From 2942b2457a62b737530e560d148ec0c5b490b94c Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 2 Feb 2016 17:20:42 -0800 Subject: [PATCH 19/81] WIP. --- .../org/apache/spark/storage/BlockStore.scala | 2 ++ .../apache/spark/storage/MemoryStore.scala | 33 ++++++++++++++++--- 2 files changed, 30 insertions(+), 5 deletions(-) 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 69985c9759e2d..6f6a6773ba4fd 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala @@ -60,8 +60,10 @@ private[spark] abstract class BlockStore(val blockManager: BlockManager) extends /** * Remove a block, if it exists. + * * @param blockId the block to remove. * @return True if the block was found and removed, False otherwise. + * @throws IllegalStateException if the block is pinned by a task. */ def remove(blockId: BlockId): Boolean 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 29b9a48bec897..cbda1de150eaa 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -208,12 +208,35 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo } override def remove(blockId: BlockId): Boolean = memoryManager.synchronized { - val pinCount = blockManager.getPinCount(blockId) - if (pinCount != 0) { - throw new IllegalStateException( - s"Cannot free block $blockId since it is still pinned $pinCount times") + // This method is called from two different places: + // + // 1. When removing a block in BlockManager.removeBlock(). This is called by ContextCleaner + // cleanup code (e.g. when removing blocks from RDDs which are have fallen out of scope on + // the driver) or when a user explicitly unpersists an RDD or deletes a broadcast variable. + // + // 2. When dropping a block memory in BlockManager.dropFromMemory(), which is called by the + // MemoryStore when dropping blocks to free up space. The MemoryStore will never evict a + // pinned block. + // + // As a result, the only situation where `pinCount != 0` in this block is if the user performed + // an unsafe manual block eviction (which currently has undefined semantics), so in that case + // we choose to fail the user-initiated eviction rather than possibly crash running tasks by + // deleting data that they're using. + // + // Regarding thread-safety: + // + // - We want to avoid a race where we see that the pin count is zero, begin removing a block, + // then have a new read which re-pins the block right as we start removing it here. + // - In the code below, we synchronize on `entries` before checking the pin count. + // - In order for a + val entry = entries.synchronized { + val pinCount = blockManager.getPinCount(blockId) + if (pinCount != 0) { + throw new IllegalStateException( + s"Cannot free block $blockId since it is still pinned $pinCount times") + } + entries.remove(blockId) } - val entry = entries.synchronized { entries.remove(blockId) } if (entry != null) { memoryManager.releaseStorageMemory(entry.size) logDebug(s"Block $blockId of size ${entry.size} dropped " + From 62f667125498c3dadd98fc5ba17272db1a1cbf19 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 10 Feb 2016 18:05:10 -0800 Subject: [PATCH 20/81] BlockInfoManager WIP. --- .../scala/org/apache/spark/CacheManager.scala | 7 +- .../spark/broadcast/TorrentBroadcast.scala | 5 +- .../org/apache/spark/executor/Executor.scala | 2 +- .../spark/scheduler/TaskResultGetter.scala | 2 +- .../org/apache/spark/storage/BlockInfo.scala | 83 ------- .../spark/storage/BlockInfoManager.scala | 214 ++++++++++++++++++ .../apache/spark/storage/BlockManager.scala | 160 +++++-------- .../apache/spark/storage/MemoryStore.scala | 14 +- .../spark/util/collection/PinCounter.scala | 126 ----------- .../spark/storage/BlockInfoManagerSuite.scala | 183 +++++++++++++++ .../BlockManagerReplicationSuite.scala | 2 +- .../spark/storage/BlockManagerSuite.scala | 100 ++++---- .../util/collection/PinCounterSuite.scala | 114 ---------- 13 files changed, 513 insertions(+), 499 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/storage/BlockInfo.scala create mode 100644 core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala delete mode 100644 core/src/main/scala/org/apache/spark/util/collection/PinCounter.scala create mode 100644 core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/util/collection/PinCounterSuite.scala diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index e7c83d415d543..c79de14146e7e 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -49,7 +49,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { val iter = CompletionIterator[T, Iterator[T]]( blockResult.data.asInstanceOf[Iterator[T]], - blockManager.unpin(key)) + blockManager.releaseLock(key)) new InterruptibleIterator[T](context, iter) { override def next(): T = { existingMetrics.incRecordsRead(1) @@ -61,7 +61,8 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { // If another thread already holds the lock, wait for it to finish return its results val storedValues = acquireLockForPartition[T](key) if (storedValues.isDefined) { - val iter = CompletionIterator[T, Iterator[T]](storedValues.get, blockManager.unpin(key)) + val iter = + CompletionIterator[T, Iterator[T]](storedValues.get, blockManager.releaseLock(key)) return new InterruptibleIterator[T](context, iter) } @@ -143,7 +144,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { case Some(v) => CompletionIterator[T, Iterator[T]]( v.data.asInstanceOf[Iterator[T]], - blockManager.unpin(key)) + blockManager.releaseLock(key)) case None => logInfo(s"Failure to store $key") throw new BlockException(key, s"Block manager failed to return cached value for $key!") diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 6a30b768e91b1..a2872207bde9e 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -140,7 +140,7 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) // On the driver broadcast variables may be loaded when computing rdd.partitions(), which // takes place outside of the context of a task, so we need to use an option here: Option(TaskContext.get()).foreach { taskContext => - taskContext.addTaskCompletionListener(_ => SparkEnv.get.blockManager.unpin(pieceId)) + taskContext.addTaskCompletionListener(_ => SparkEnv.get.blockManager.releaseLock(pieceId)) } } blocks @@ -175,7 +175,8 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) // On the driver broadcast variables may be loaded when computing rdd.partitions(), which // takes place outside of the context of a task, so we need to use an option here: Option(TaskContext.get()).foreach { taskContext => - taskContext.addTaskCompletionListener(_ => SparkEnv.get.blockManager.unpin(broadcastId)) + taskContext.addTaskCompletionListener(_ => + SparkEnv.get.blockManager.releaseLock(broadcastId)) } x.asInstanceOf[T] 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 cfe64d0f12a96..0da4c5a2832c8 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -218,7 +218,7 @@ private[spark] class Executor( threwException = false res } finally { - val releasedPins = env.blockManager.releaseAllPinsForTask(taskId) + val releasedPins = env.blockManager.releaseAllLocksForTask(taskId) val freedMemory = taskMemoryManager.cleanUpAllAllocatedMemory() if (freedMemory > 0) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 0a7073f2263f4..541bc256f983b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -83,7 +83,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul } val deserializedResult = serializer.get().deserialize[DirectTaskResult[_]]( serializedTaskResult.get) - sparkEnv.blockManager.unpin(blockId) + sparkEnv.blockManager.releaseLock(blockId) sparkEnv.blockManager.master.removeBlock(blockId) (deserializedResult, size) } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala deleted file mode 100644 index 22fdf73e9d1f4..0000000000000 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfo.scala +++ /dev/null @@ -1,83 +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.storage - -import java.util.concurrent.ConcurrentHashMap - -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) - - setInitThread() - - private def setInitThread() { - /* Set current thread as init thread - waitForReady will not block this thread - * (in case there is non trivial initialization which ends up calling waitForReady - * as part of initialization itself) */ - BlockInfo.blockInfoInitThreads.put(this, Thread.currentThread()) - } - - /** - * Wait for this BlockInfo to be marked as ready (i.e. block is finished writing). - * Return true if the block is available, false otherwise. - */ - def waitForReady(): Boolean = { - if (pending && initThread != Thread.currentThread()) { - synchronized { - while (pending) { - this.wait() - } - } - } - !failed - } - - /** Mark this BlockInfo as ready (i.e. block is finished writing) */ - def markReady(sizeInBytes: Long) { - require(sizeInBytes >= 0, s"sizeInBytes was negative: $sizeInBytes") - assert(pending) - size = sizeInBytes - BlockInfo.blockInfoInitThreads.remove(this) - synchronized { - this.notifyAll() - } - } - - /** Mark this BlockInfo as ready but failed */ - def markFailure() { - assert(pending) - size = BlockInfo.BLOCK_FAILED - BlockInfo.blockInfoInitThreads.remove(this) - synchronized { - this.notifyAll() - } - } -} - -private object BlockInfo { - /* initThread is logically a BlockInfo field, but we store it here because - * it's only needed while this block is in the 'pending' state and we want - * to minimize BlockInfo's memory footprint. */ - private val blockInfoInitThreads = new ConcurrentHashMap[BlockInfo, Thread] - - private val BLOCK_PENDING: Long = -1L - private val BLOCK_FAILED: Long = -2L -} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala new file mode 100644 index 0000000000000..9afee9cf0c9e2 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -0,0 +1,214 @@ +/* + * 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.storage + +import java.lang +import javax.annotation.concurrent.GuardedBy + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} +import com.google.common.collect.ConcurrentHashMultiset + +import org.apache.spark.{Logging, TaskContext} + + +private[storage] class BlockInfo( + val level: StorageLevel, + val tellMaster: Boolean) { + var size: Long = 0 + var readerCount: Int = 0 + var writerTask: Long = -1 +} + + +private[storage] class BlockInfoManager extends Logging { + + private type TaskAttemptId = Long + + @GuardedBy("this") + private[this] val infos = new mutable.HashMap[BlockId, BlockInfo] + + @GuardedBy("this") + private[this] val writeLocksByTask = + new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]] + with mutable.MultiMap[TaskAttemptId, BlockId] + + private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = { + // We need to explicitly box as java.lang.Long to avoid a type mismatch error: + val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] { + override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]() + } + CacheBuilder.newBuilder().build(loader) + } + + // ---------------------------------------------------------------------------------------------- + + private def currentTaskAttemptId: TaskAttemptId = { + Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1L) + } + + /** + * Todo: document blocking / locking semantics. + * + * @param blockId + * @return + */ + def getAndLockForReading( + blockId: BlockId, + blocking: Boolean = true): Option[BlockInfo] = synchronized { + // TODO: eagerness + infos.get(blockId).map { info => + while (info.writerTask != -1) { + if (blocking) wait() else return None + } + // TODO: try to remember why you need actualInfo / the extra get() here. + val actualInfo = infos.get(blockId) + actualInfo.foreach { i => + i.readerCount += 1 + readLocksByTask(currentTaskAttemptId).add(blockId) + } + info + } + } + + def getAndLockForWriting( + blockId: BlockId, + blocking: Boolean = true): Option[BlockInfo] = synchronized { + infos.get(blockId).map { info => + while (info.writerTask != -1 || info.readerCount != 0) { + if (blocking) wait() else return None + } + val actualInfo = infos.get(blockId) + actualInfo.foreach { i => + i.writerTask = currentTaskAttemptId + } + info + } + } + + def get(blockId: BlockId): Option[BlockInfo] = synchronized { + infos.get(blockId) + } + + def downgradeLock(blockId: BlockId): Unit = synchronized { + + } + + def releaseLock(blockId: BlockId): Unit = synchronized { + val info = get(blockId).get + if (info.writerTask != -1) { + info.writerTask = -1 + } else { + assert(info.readerCount > 0) + info.readerCount -= 1 + val countsForTask = readLocksByTask.get(currentTaskAttemptId) + val newPinCountForTask: Int = countsForTask.remove(blockId, 1) - 1 + assert(newPinCountForTask >= 0) + } + notifyAll() + } + + def putAndLockForWritingIfAbsent( + blockId: BlockId, + newBlockInfo: BlockInfo): Boolean = synchronized { + val actualInfo = infos.getOrElseUpdate(blockId, newBlockInfo) + if (actualInfo eq newBlockInfo) { + actualInfo.writerTask = currentTaskAttemptId + true + } else { + false + } + } + + /** + * Release all pins held by the given task, clearing that task's pin bookkeeping + * structures and updating the global pin counts. This method should be called at the + * end of a task (either by a task completion handler or in `TaskRunner.run()`). + * + * @return the number of pins released + */ + def releaseAllPinsForTask(taskAttemptId: TaskAttemptId): Int = { + synchronized { + writeLocksByTask.remove(taskAttemptId).foreach { locks => + for (lock <- locks) { + infos.get(lock).foreach { info => + assert(info.writerTask == taskAttemptId) + info.writerTask = -1 + } + } + } + notifyAll() + } + val readLocks = readLocksByTask.get(taskAttemptId) + readLocksByTask.invalidate(taskAttemptId) + val totalPinCountForTask = readLocks.size() + readLocks.entrySet().iterator().asScala.foreach { entry => + val blockId = entry.getElement + val lockCount = entry.getCount + synchronized { + get(blockId).foreach { info => + info.readerCount -= lockCount + assert(info.readerCount >= 0) + } + } + } + synchronized { + notifyAll() + } + totalPinCountForTask + } + + + def size: Int = synchronized { + infos.size + } + + /** + * Return the number of map entries in this pin counter's internal data structures. + * This is used in unit tests in order to detect memory leaks. + */ + private[storage] def getNumberOfMapEntries: Long = synchronized { + size + + readLocksByTask.size() + + readLocksByTask.asMap().asScala.map(_._2.size()).sum + + writeLocksByTask.size + + writeLocksByTask.map(_._2.size).sum + } + + // This implicitly drops all locks. + def remove(blockId: BlockId): Unit = synchronized { + // TODO: Should probably have safety checks here + infos.remove(blockId) + // May also want to mark the blockInfo as removed so that gets() don't keep references to it? + // This should also implicitly release locks? No. + } + + def clear(): Unit = synchronized { + infos.clear() + readLocksByTask.invalidateAll() + writeLocksByTask.clear() + } + + + def entries: Iterator[(BlockId, BlockInfo)] = synchronized { + infos.iterator.toArray.toIterator + } + +} 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 a6c2704c4b8b2..496f2b1496115 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -21,7 +21,6 @@ import java.io._ import java.nio.{ByteBuffer, MappedByteBuffer} import java.util.concurrent.ConcurrentHashMap -import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.concurrent.{Await, ExecutionContext, Future} import scala.concurrent.duration._ @@ -43,7 +42,6 @@ import org.apache.spark.rpc.RpcEnv import org.apache.spark.serializer.{Serializer, SerializerInstance} import org.apache.spark.shuffle.ShuffleManager import org.apache.spark.util._ -import org.apache.spark.util.collection.PinCounter private[spark] sealed trait BlockValues private[spark] case class ByteBufferValues(buffer: ByteBuffer) extends BlockValues @@ -78,7 +76,7 @@ private[spark] class BlockManager( val diskBlockManager = new DiskBlockManager(this, conf) - private val blockInfo = new ConcurrentHashMap[BlockId, BlockInfo] + private[storage] val blockInfoManager = new BlockInfoManager private val futureExecutionContext = ExecutionContext.fromExecutorService( ThreadUtils.newDaemonCachedThreadPool("block-manager-future", 128)) @@ -157,8 +155,6 @@ private[spark] class BlockManager( * loaded yet. */ private lazy val compressionCodec: CompressionCodec = CompressionCodec.createCodec(conf) - private val pinCounts = new PinCounter[BlockId] - /** * Initializes the BlockManager with the given appId. This is not performed in the constructor as * the appId may not be known at BlockManager instantiation time (in particular for the driver, @@ -226,8 +222,8 @@ private[spark] class BlockManager( * will be made then. */ private def reportAllBlocks(): Unit = { - logInfo(s"Reporting ${blockInfo.size} blocks to the master.") - for ((blockId, info) <- blockInfo.asScala) { + logInfo(s"Reporting ${blockInfoManager.size} blocks to the master.") + for ((blockId, info) <- blockInfoManager.entries) { val status = getCurrentBlockStatus(blockId, info) if (!tryToReportBlockStatus(blockId, info, status)) { logError(s"Failed to report $blockId to master; giving up.") @@ -308,7 +304,7 @@ private[spark] class BlockManager( * NOTE: This is mainly for testing, and it doesn't fetch information from external block store. */ def getStatus(blockId: BlockId): Option[BlockStatus] = { - blockInfo.asScala.get(blockId).map { info => + blockInfoManager.get(blockId).map { info => val memSize = if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L val diskSize = if (diskStore.contains(blockId)) diskStore.getSize(blockId) else 0L BlockStatus(info.level, memSize = memSize, diskSize = diskSize) @@ -321,7 +317,7 @@ private[spark] class BlockManager( * may not know of). */ def getMatchingBlockIds(filter: BlockId => Boolean): Seq[BlockId] = { - (blockInfo.asScala.keys ++ diskBlockManager.getAllBlocks()).filter(filter).toSeq + (blockInfoManager.entries.map(_._1) ++ diskBlockManager.getAllBlocks()).filter(filter).toSeq } /** @@ -406,11 +402,7 @@ private[spark] class BlockManager( */ def getLocal(blockId: BlockId): Option[BlockResult] = { logDebug(s"Getting local block $blockId") - val res = doGetLocal(blockId, asBlockResult = true).asInstanceOf[Option[BlockResult]] - if (res.isDefined) { - pinCounts.pin(blockId) - } - res + doGetLocal(blockId, asBlockResult = true).asInstanceOf[Option[BlockResult]] } /** @@ -419,6 +411,7 @@ private[spark] class BlockManager( * when writing tests. */ def hasLocalBlock(blockId: BlockId): Boolean = { + // TODO(josh): this needs to be reworked doGetLocal(blockId, asBlockResult = true).asInstanceOf[Option[BlockResult]].isDefined } @@ -429,7 +422,7 @@ private[spark] class BlockManager( logDebug(s"Getting local block $blockId as bytes") // As an optimization for map output fetches, if the block is for a shuffle, return it // without acquiring a lock; the disk store never deletes (recent) items so this should work - val res = if (blockId.isShuffle) { + if (blockId.isShuffle) { val shuffleBlockResolver = shuffleManager.shuffleBlockResolver // TODO: This should gracefully handle case where local block is not available. Currently // downstream code will throw an exception. @@ -438,33 +431,14 @@ private[spark] class BlockManager( } else { doGetLocal(blockId, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] } - if (res.isDefined) { - pinCounts.pin(blockId) - } - res } private def doGetLocal(blockId: BlockId, asBlockResult: Boolean): Option[Any] = { - val info = blockInfo.get(blockId) - if (info != null) { - info.synchronized { - // Double check to make sure the block is still there. There is a small chance that the - // block has been removed by removeBlock (which also synchronizes on the blockInfo object). - // Note that this only checks metadata tracking. If user intentionally deleted the block - // on disk or from off heap storage without using removeBlock, this conditional check will - // still pass but eventually we will get an exception because we can't find the block. - if (blockInfo.asScala.get(blockId).isEmpty) { - logWarning(s"Block $blockId had been removed") - return None - } - - // If another thread is writing the block, wait for it to become ready. - if (!info.waitForReady()) { - // If we get here, the block write failed. - logWarning(s"Block $blockId was marked as failure.") - return None - } - + blockInfoManager.getAndLockForReading(blockId) match { + case None => + logDebug(s"Block $blockId was not found") + None + case Some(info) => val level = info.level logDebug(s"Level for block $blockId is $level") @@ -541,12 +515,10 @@ private[spark] class BlockManager( } } } + } else { + None } - } - } else { - logDebug(s"Block $blockId not registered locally") } - None } /** @@ -554,11 +526,7 @@ private[spark] class BlockManager( */ def getRemote(blockId: BlockId): Option[BlockResult] = { logDebug(s"Getting remote block $blockId") - val res = doGetRemote(blockId, asBlockResult = true).asInstanceOf[Option[BlockResult]] - if (res.isDefined) { - pinCounts.pin(blockId) - } - res + doGetRemote(blockId, asBlockResult = true).asInstanceOf[Option[BlockResult]] } /** @@ -566,11 +534,7 @@ private[spark] class BlockManager( */ def getRemoteBytes(blockId: BlockId): Option[ByteBuffer] = { logDebug(s"Getting remote block $blockId as bytes") - val res = doGetRemote(blockId, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] - if (res.isDefined) { - pinCounts.pin(blockId) - } - res + doGetRemote(blockId, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] } /** @@ -641,22 +605,19 @@ private[spark] class BlockManager( } /** - * Release one pin of the given block. + * Release a lock on the given block. */ - def unpin(blockId: BlockId): Unit = { - pinCounts.unpin(blockId) + def releaseLock(blockId: BlockId): Unit = { + blockInfoManager.releaseLock(blockId) } /** * Release all pins for the given task. + * * @return the total number of pins released. */ - def releaseAllPinsForTask(taskAttemptId: Long): Int = { - pinCounts.releaseAllPinsForTask(taskAttemptId) - } - - private[storage] def getPinCount(blockId: BlockId): Int = { - pinCounts.getPinCount(blockId) + def releaseAllLocksForTask(taskAttemptId: Long): Int = { + blockInfoManager.releaseAllPinsForTask(taskAttemptId) } /** @@ -750,19 +711,13 @@ private[spark] class BlockManager( * to be dropped right after it got put into memory. Note, however, that other threads will * not be able to get() this block until we call markReady on its BlockInfo. */ val putBlockInfo = { - val tinfo = new BlockInfo(level, tellMaster) - // Do atomically ! - val oldBlockOpt = Option(blockInfo.putIfAbsent(blockId, tinfo)) - if (oldBlockOpt.isDefined) { - if (oldBlockOpt.get.waitForReady()) { - logWarning(s"Block $blockId already exists on this machine; not re-adding it") - return false - } - // TODO: So the block info exists - but previous attempt to load it (?) failed. - // What do we do now ? Retry on it ? - oldBlockOpt.get + // TODO(josh): if an existing put is in progress, do we block to see if it's done / succeeds? + val newInfo = new BlockInfo(level, tellMaster) + if (blockInfoManager.putAndLockForWritingIfAbsent(blockId, newInfo)) { + newInfo } else { - tinfo + logWarning(s"Block $blockId already exists on this machine; not re-adding it") + return false } } @@ -797,7 +752,7 @@ private[spark] class BlockManager( case _ => null } - var marked = false + var blockWasSuccessfullyStored = false putBlockInfo.synchronized { logTrace("Put for block %s took %s to get into synchronized block" @@ -842,8 +797,8 @@ private[spark] class BlockManager( if (putBlockStatus.storageLevel != StorageLevel.NONE) { // Now that the block is in either the memory, externalBlockStore, or disk store, // let other threads read it, and tell the master about it. - marked = true - putBlockInfo.markReady(size) + blockWasSuccessfullyStored = true + blockInfoManager.downgradeLock(blockId) if (tellMaster) { reportBlockStatus(blockId, putBlockInfo, putBlockStatus) } @@ -854,11 +809,10 @@ private[spark] class BlockManager( } finally { // If we failed in putting the block to memory/disk, notify other possible readers // that it has failed, and then remove it from the block info map. - if (!marked) { + if (!blockWasSuccessfullyStored) { // Note that the remove must happen before markFailure otherwise another thread // could've inserted a new BlockInfo before we remove it. - blockInfo.remove(blockId) - putBlockInfo.markFailure() + blockInfoManager.remove(blockId) logWarning(s"Putting block $blockId failed") } } @@ -899,7 +853,7 @@ private[spark] class BlockManager( .format(blockId, Utils.getUsedTimeMs(startTimeMs))) } - marked + blockWasSuccessfullyStored } /** @@ -1042,21 +996,10 @@ private[spark] class BlockManager( data: () => Either[Array[Any], ByteBuffer]): Unit = { logInfo(s"Dropping block $blockId from memory") - val info = blockInfo.get(blockId) - - // If the block has not already been dropped - if (info != null) { - info.synchronized { - // required ? As of now, this will be invoked only for blocks which are ready - // But in case this changes in future, adding for consistency sake. - if (!info.waitForReady()) { - // If we get here, the block write failed. - logWarning(s"Block $blockId was marked as failure. Nothing to drop") - return - } else if (blockInfo.asScala.get(blockId).isEmpty) { - logWarning(s"Block $blockId was already dropped.") - return - } + blockInfoManager.getAndLockForWriting(blockId) match { + case None => + logDebug(s"Block $blockId has already been dropped") + case Some(info) => var blockIsUpdated = false val level = info.level @@ -1088,14 +1031,15 @@ private[spark] class BlockManager( } if (!level.useDisk) { // The block is completely gone from this node; forget it so we can put() it again later. - blockInfo.remove(blockId) + blockInfoManager.remove(blockId) + } else { + blockInfoManager.releaseLock(blockId) } if (blockIsUpdated) { Option(TaskContext.get()).foreach { c => c.taskMetrics().incUpdatedBlockStatuses(Seq((blockId, status))) } } - } } } @@ -1107,7 +1051,7 @@ private[spark] class BlockManager( def removeRdd(rddId: Int): Int = { // TODO: Avoid a linear scan by creating another mapping of RDD.id to blocks. logInfo(s"Removing RDD $rddId") - val blocksToRemove = blockInfo.asScala.keys.flatMap(_.asRDDId).filter(_.rddId == rddId) + val blocksToRemove = blockInfoManager.entries.flatMap(_._1.asRDDId).filter(_.rddId == rddId) blocksToRemove.foreach { blockId => removeBlock(blockId, tellMaster = false) } blocksToRemove.size } @@ -1117,7 +1061,7 @@ private[spark] class BlockManager( */ def removeBroadcast(broadcastId: Long, tellMaster: Boolean): Int = { logDebug(s"Removing broadcast $broadcastId") - val blocksToRemove = blockInfo.asScala.keys.collect { + val blocksToRemove = blockInfoManager.entries.map(_._1).collect { case bid @ BroadcastBlockId(`broadcastId`, _) => bid } blocksToRemove.foreach { blockId => removeBlock(blockId, tellMaster) } @@ -1129,9 +1073,11 @@ private[spark] class BlockManager( */ def removeBlock(blockId: BlockId, tellMaster: Boolean = true): Unit = { logDebug(s"Removing block $blockId") - val info = blockInfo.get(blockId) - if (info != null) { - info.synchronized { + blockInfoManager.getAndLockForWriting(blockId) match { + case None => + // The block has already been removed; do nothing. + logWarning(s"Asked to remove block $blockId, which does not exist") + case Some(info) => // Removals are idempotent in disk store and memory store. At worst, we get a warning. val removedFromMemory = memoryStore.remove(blockId) val removedFromDisk = diskStore.remove(blockId) @@ -1139,15 +1085,11 @@ private[spark] class BlockManager( logWarning(s"Block $blockId could not be removed as it was not found in either " + "the disk, memory, or external block store") } - blockInfo.remove(blockId) + blockInfoManager.remove(blockId) if (tellMaster && info.tellMaster) { val status = getCurrentBlockStatus(blockId, info) reportBlockStatus(blockId, info, status) } - } - } else { - // The block has already been removed; do nothing. - logWarning(s"Asked to remove block $blockId, which does not exist") } } @@ -1222,7 +1164,7 @@ private[spark] class BlockManager( } diskBlockManager.stop() rpcEnv.stop(slaveEndpoint) - blockInfo.clear() + blockInfoManager.clear() memoryStore.clear() diskStore.clear() futureExecutionContext.shutdownNow() 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 cbda1de150eaa..c7d0b71fed230 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -230,11 +230,6 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo // - In the code below, we synchronize on `entries` before checking the pin count. // - In order for a val entry = entries.synchronized { - val pinCount = blockManager.getPinCount(blockId) - if (pinCount != 0) { - throw new IllegalStateException( - s"Cannot free block $blockId since it is still pinned $pinCount times") - } entries.remove(blockId) } if (entry != null) { @@ -435,8 +430,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo val rddToAdd = blockId.flatMap(getRddId) val selectedBlocks = new ArrayBuffer[BlockId] def blockIsEvictable(blockId: BlockId): Boolean = { - blockManager.getPinCount(blockId) == 0 && - (rddToAdd.isEmpty || rddToAdd != getRddId(blockId)) + rddToAdd.isEmpty || rddToAdd != getRddId(blockId) } // This is synchronized to ensure that the set of entries is not changed // (because of getValue or getBytes) while traversing the iterator, as that @@ -447,8 +441,10 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo val pair = iterator.next() val blockId = pair.getKey if (blockIsEvictable(blockId)) { - selectedBlocks += blockId - freedMemory += pair.getValue.size + if (blockManager.blockInfoManager.getAndLockForWriting(blockId, false).isDefined) { + selectedBlocks += blockId + freedMemory += pair.getValue.size + } } } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/PinCounter.scala b/core/src/main/scala/org/apache/spark/util/collection/PinCounter.scala deleted file mode 100644 index 7165ce24f7e06..0000000000000 --- a/core/src/main/scala/org/apache/spark/util/collection/PinCounter.scala +++ /dev/null @@ -1,126 +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.collection - -import scala.collection.JavaConverters._ - -import com.google.common.cache.{CacheBuilder, CacheLoader} -import com.google.common.collect.ConcurrentHashMultiset - -import org.apache.spark.TaskContext - -/** - * Thread-safe collection for maintaining both global and per-task pin counts for objects. - */ -private[spark] class PinCounter[T] { - - private type TaskAttemptId = Long - - /** - * Total pins across all tasks. - */ - private[this] val allPins = ConcurrentHashMultiset.create[T]() - - /** - * Total pins per task. Used to auto-release pins upon task completion. - */ - private[this] val pinsByTask = { - // We need to explicitly box as java.lang.Long to avoid a type mismatch error: - val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[T]] { - override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[T]() - } - CacheBuilder.newBuilder().build(loader) - } - - /** - * Returns the total pin count, across all tasks, for the given object. - */ - def getPinCount(obj: T): Int = allPins.count(obj) - - /** - * Increments the given object's pin count for the current task. - */ - def pin(obj: T): Unit = pinForTask(currentTaskAttemptId, obj) - - /** - * Decrements the given object's pin count for the current task. - */ - def unpin(obj: T): Unit = unpinForTask(currentTaskAttemptId, obj) - - private def currentTaskAttemptId: TaskAttemptId = { - Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1L) - } - - /** - * Increments the given object's pin count for the given task. - */ - def pinForTask(taskAttemptId: TaskAttemptId, obj: T): Unit = { - pinsByTask.get(taskAttemptId).add(obj) - allPins.add(obj) - } - - /** - * Decrements the given object's pin count for the given task. - */ - def unpinForTask(taskAttemptId: TaskAttemptId, obj: T): Unit = { - val countsForTask = pinsByTask.get(taskAttemptId) - val newPinCountForTask: Int = countsForTask.remove(obj, 1) - 1 - val newTotalPinCount: Int = allPins.remove(obj, 1) - 1 - if (newPinCountForTask < 0) { - throw new IllegalStateException( - s"Task $taskAttemptId released object $obj more times than it was retained") - } - if (newTotalPinCount < 0) { - throw new IllegalStateException( - s"Task $taskAttemptId released object $obj more times than it was retained") - } - } - - /** - * Release all pins held by the given task, clearing that task's pin bookkeeping - * structures and updating the global pin counts. This method should be called at the - * end of a task (either by a task completion handler or in `TaskRunner.run()`). - * - * @return the number of pins released - */ - def releaseAllPinsForTask(taskAttemptId: TaskAttemptId): Int = { - val pinCounts = pinsByTask.get(taskAttemptId) - pinsByTask.invalidate(taskAttemptId) - val totalPinCountForTask = pinCounts.size() - pinCounts.entrySet().iterator().asScala.foreach { entry => - val obj = entry.getElement - val taskRefCount = entry.getCount - val newRefCount = allPins.remove(obj, taskRefCount) - taskRefCount - if (newRefCount < 0) { - throw new IllegalStateException( - s"Task $taskAttemptId released object $obj more times than it was retained") - } - } - totalPinCountForTask - } - - /** - * Return the number of map entries in this pin counter's internal data structures. - * This is used in unit tests in order to detect memory leaks. - */ - private[collection] def getNumberOfMapEntries: Long = { - allPins.size() + - pinsByTask.size() + - pinsByTask.asMap().asScala.map(_._2.size()).sum - } -} diff --git a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala new file mode 100644 index 0000000000000..0ea8789207a8e --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala @@ -0,0 +1,183 @@ +/* + * 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.storage + +import scala.concurrent.{Await, ExecutionContext, Future} +import scala.language.implicitConversions + +import org.scalatest.BeforeAndAfterEach +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.{SparkFunSuite, TaskContext, TaskContextImpl} + + +class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { + + private implicit val ec = ExecutionContext.global + private var blockInfoManager: BlockInfoManager = _ + + override protected def beforeEach(): Unit = { + super.beforeEach() + blockInfoManager = new BlockInfoManager() + } + + override protected def afterEach(): Unit = { + blockInfoManager = null + super.afterEach() + } + + private implicit def stringToBlockId(str: String): BlockId = { + TestBlockId(str) + } + + private def newBlockInfo(): BlockInfo = { + new BlockInfo(StorageLevel.MEMORY_ONLY, tellMaster = false) + } + + private def withTaskId[T](taskAttemptId: Long)(block: => T): T = { + try { + TaskContext.setTaskContext(new TaskContextImpl(0, 0, taskAttemptId, 0, null, null)) + block + } finally { + TaskContext.unset() + } + } + + test("initial memory usage") { + assert(blockInfoManager.getNumberOfMapEntries === 0) + assert(blockInfoManager.size === 0) + } + + test("get non-existent block") { + assert(blockInfoManager.get("non-existent-block").isEmpty) + assert(blockInfoManager.getAndLockForReading("non-existent-block").isEmpty) + assert(blockInfoManager.getAndLockForWriting("non-existent-block").isEmpty) + } + + test("basic putAndLockForWritingIfAbsent") { + val blockInfo = newBlockInfo() + withTaskId(1) { + assert(blockInfoManager.putAndLockForWritingIfAbsent("block", blockInfo)) + assert(blockInfoManager.get("block").get eq blockInfo) + assert(!blockInfoManager.putAndLockForWritingIfAbsent("block", newBlockInfo())) + assert(blockInfoManager.get("block").get eq blockInfo) + assert(blockInfo.readerCount === 0) + assert(blockInfo.writerTask === 1) + blockInfoManager.releaseLock("block") + assert(blockInfo.readerCount === 0) + assert(blockInfo.writerTask === -1) + } + assert(blockInfoManager.size === 1) + assert(blockInfoManager.getNumberOfMapEntries === 1) + } + + test("read locks are reentrant") { + withTaskId(1) { + assert(blockInfoManager.putAndLockForWritingIfAbsent("block", newBlockInfo())) + blockInfoManager.releaseLock("block") + assert(blockInfoManager.getAndLockForReading("block", blocking = false).isDefined) + assert(blockInfoManager.getAndLockForReading("block", blocking = false).isDefined) + assert(blockInfoManager.get("block").get.readerCount === 2) + assert(blockInfoManager.get("block").get.writerTask === -1) + blockInfoManager.releaseLock("block") + assert(blockInfoManager.get("block").get.readerCount === 1) + blockInfoManager.releaseLock("block") + assert(blockInfoManager.get("block").get.readerCount === 0) + } + } + + test("multiple tasks can hold read locks") { + withTaskId(0) { + assert(blockInfoManager.putAndLockForWritingIfAbsent("block", newBlockInfo())) + blockInfoManager.releaseLock("block") + } + withTaskId(1) { assert(blockInfoManager.getAndLockForReading("block").isDefined) } + withTaskId(2) { assert(blockInfoManager.getAndLockForReading("block").isDefined) } + withTaskId(3) { assert(blockInfoManager.getAndLockForReading("block").isDefined) } + withTaskId(4) { assert(blockInfoManager.getAndLockForReading("block").isDefined) } + assert(blockInfoManager.get("block").get.readerCount === 4) + } + + test("single task can hold write lock") { + withTaskId(0) { + assert(blockInfoManager.putAndLockForWritingIfAbsent("block", newBlockInfo())) + blockInfoManager.releaseLock("block") + } + withTaskId(1) { + assert(blockInfoManager.getAndLockForWriting("block").isDefined) + assert(blockInfoManager.get("block").get.writerTask === 1) + } + withTaskId(2) { + assert(blockInfoManager.getAndLockForWriting("block", blocking = false).isEmpty) + assert(blockInfoManager.get("block").get.writerTask === 1) + } + } + + test("write lock will block readers") { + withTaskId(0) { + assert(blockInfoManager.putAndLockForWritingIfAbsent("block", newBlockInfo())) + } + val get1Future = Future { + withTaskId(1) { + blockInfoManager.getAndLockForReading("block") + } + } + val get2Future = Future { + withTaskId(2) { + blockInfoManager.getAndLockForReading("block") + } + } + Thread.sleep(300) // Hack to try to ensure that both future tasks are waiting + withTaskId(0) { + blockInfoManager.releaseLock("block") + } + assert(Await.result(get1Future, 1.seconds).isDefined) + assert(Await.result(get2Future, 1.seconds).isDefined) + assert(blockInfoManager.get("block").get.readerCount === 2) + } + + test("read locks will block writer") { + withTaskId(0) { + assert(blockInfoManager.putAndLockForWritingIfAbsent("block", newBlockInfo())) + blockInfoManager.releaseLock("block") + blockInfoManager.getAndLockForReading("block") + } + val write1Future = Future { + withTaskId(1) { + blockInfoManager.getAndLockForWriting("block") + } + } + val write2Future = Future { + withTaskId(2) { + blockInfoManager.getAndLockForWriting("block") + } + } + Thread.sleep(300) // Hack to try to ensure that both future tasks are waiting + withTaskId(0) { + blockInfoManager.releaseLock("block") + } + assert( + Await.result(Future.firstCompletedOf(Seq(write1Future, write2Future)), 1.seconds).isDefined) + val firstWriteWinner = if (write1Future.isCompleted) 1 else 2 + withTaskId(firstWriteWinner) { + blockInfoManager.releaseLock("block") + } + assert(Await.result(write1Future, 1.seconds).isDefined) + assert(Await.result(write2Future, 1.seconds).isDefined) + } +} \ No newline at end of file diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala index 63e8b95941164..ae1faf5d98f38 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -367,7 +367,7 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo }.foreach { testStore => val testStoreName = testStore.blockManagerId.executorId assert(testStore.getLocal(blockId).isDefined, s"$blockId was not found in $testStoreName") - testStore.unpin(blockId) + testStore.releaseLock(blockId) assert(master.getLocations(blockId).map(_.executorId).toSet.contains(testStoreName), s"master does not have status for ${blockId.name} in $testStoreName") 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 b1890f34c538c..bb0bb35f9fc84 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -176,11 +176,11 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // Checking whether blocks are in memory assert(store.getSingle("a1").isDefined, "a1 was not in store") - store.unpin("a1") + store.releaseLock("a1") assert(store.getSingle("a2").isDefined, "a2 was not in store") - store.unpin("a2") + store.releaseLock("a2") assert(store.getSingle("a3").isDefined, "a3 was not in store") - store.unpin("a3") + store.releaseLock("a3") // Checking whether master knows about the blocks or not assert(master.getLocations("a1").size > 0, "master was not told about a1") @@ -228,11 +228,11 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE 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") - store.unpin("a1-to-remove") + store.releaseLock("a1-to-remove") assert(store.getSingle("a2-to-remove").isDefined, "a2 was not in store") - store.unpin("a2-to-remove") + store.releaseLock("a2-to-remove") assert(store.getSingle("a3-to-remove").isDefined, "a3 was not in store") - store.unpin("a3-to-remove") + store.releaseLock("a3-to-remove") // Checking whether master knows about the blocks or not assert(master.getLocations("a1-to-remove").size > 0, "master was not told about a1") @@ -517,12 +517,12 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE store.putSingle("a2", a2, storageLevel) store.putSingle("a3", a3, storageLevel) assert(store.getSingle("a2").isDefined, "a2 was not in store") - store.unpin("a2") + store.releaseLock("a2") assert(store.getSingle("a3").isDefined, "a3 was not in store") - store.unpin("a3") + store.releaseLock("a3") assert(store.getSingle("a1") === None, "a1 was in store") assert(store.getSingle("a2").isDefined, "a2 was not in store") - store.unpin("a2") + store.releaseLock("a2") // At this point a2 was gotten last, so LRU will getSingle rid of a3 store.putSingle("a1", a1, storageLevel) assert(store.getSingle("a1").isDefined, "a1 was not in store") @@ -612,12 +612,12 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE store.putSingle("a2", a2, storageLevel) store.putSingle("a3", a3, storageLevel) assert(accessMethod(store)("a2").isDefined, "a2 was not in store") - store.unpin("a2") + store.releaseLock("a2") assert(accessMethod(store)("a3").isDefined, "a3 was not in store") - store.unpin("a3") + store.releaseLock("a3") assert(store.memoryStore.getValues("a1").isEmpty, "a1 was in memory store") assert(accessMethod(store)("a1").isDefined, "a1 was not in store") - store.unpin("a1") + store.releaseLock("a1") assert(store.memoryStore.getValues("a1").isDefined, "a1 was not in memory store") } @@ -633,20 +633,20 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE store.putSingle("a3", a3, StorageLevel.DISK_ONLY) // At this point LRU should not kick in because a3 is only on disk assert(store.getSingle("a1").isDefined, "a1 was not in store") - store.unpin("a1") + store.releaseLock("a1") assert(store.getSingle("a2").isDefined, "a2 was not in store") - store.unpin("a2") + store.releaseLock("a2") assert(store.getSingle("a3").isDefined, "a3 was not in store") - store.unpin("a3") + store.releaseLock("a3") // Now let's add in a4, which uses both disk and memory; a1 should drop out store.putSingle("a4", a4, StorageLevel.MEMORY_AND_DISK_SER) assert(store.getSingle("a1") == None, "a1 was in store") assert(store.getSingle("a2").isDefined, "a2 was not in store") - store.unpin("a2") + store.releaseLock("a2") assert(store.getSingle("a3").isDefined, "a3 was not in store") - store.unpin("a3") + store.releaseLock("a3") assert(store.getSingle("a4").isDefined, "a4 was not in store") - store.unpin("a4") + store.releaseLock("a4") } test("in-memory LRU with streams") { @@ -659,27 +659,27 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE 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) - store.unpin("list2") - store.unpin("list2") + store.releaseLock("list2") + store.releaseLock("list2") assert(store.get("list3").isDefined, "list3 was not in store") assert(store.get("list3").get.data.size === 2) - store.unpin("list3") - store.unpin("list3") + store.releaseLock("list3") + store.releaseLock("list3") 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) - store.unpin("list2") - store.unpin("list2") + store.releaseLock("list2") + store.releaseLock("list2") // At this point list2 was gotten last, so LRU will getSingle rid of list3 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) - store.unpin("list1") - store.unpin("list1") + store.releaseLock("list1") + store.releaseLock("list1") assert(store.get("list2").isDefined, "list2 was not in store") assert(store.get("list2").get.data.size === 2) - store.unpin("list2") - store.unpin("list2") + store.releaseLock("list2") + store.releaseLock("list2") assert(store.get("list3") === None, "list1 was in store") } @@ -699,43 +699,43 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // At this point LRU should not kick in because list3 is only on disk assert(store.get("list1").isDefined, "list1 was not in store") assert(store.get("list1").get.data.size === 2) - store.unpin("list1") - store.unpin("list1") + store.releaseLock("list1") + store.releaseLock("list1") assert(store.get("list2").isDefined, "list2 was not in store") assert(store.get("list2").get.data.size === 2) - store.unpin("list2") - store.unpin("list2") + store.releaseLock("list2") + store.releaseLock("list2") assert(store.get("list3").isDefined, "list3 was not in store") assert(store.get("list3").get.data.size === 2) - store.unpin("list3") - store.unpin("list3") + store.releaseLock("list3") + store.releaseLock("list3") assert(store.get("list1").isDefined, "list1 was not in store") assert(store.get("list1").get.data.size === 2) - store.unpin("list1") - store.unpin("list1") + store.releaseLock("list1") + store.releaseLock("list1") assert(store.get("list2").isDefined, "list2 was not in store") assert(store.get("list2").get.data.size === 2) - store.unpin("list2") - store.unpin("list2") + store.releaseLock("list2") + store.releaseLock("list2") assert(store.get("list3").isDefined, "list3 was not in store") assert(store.get("list3").get.data.size === 2) - store.unpin("list3") - store.unpin("list3") + store.releaseLock("list3") + store.releaseLock("list3") // Now let's add in list4, which uses both disk and memory; list1 should drop out 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) - store.unpin("list2") - store.unpin("list2") + store.releaseLock("list2") + store.releaseLock("list2") assert(store.get("list3").isDefined, "list3 was not in store") assert(store.get("list3").get.data.size === 2) - store.unpin("list3") - store.unpin("list3") + store.releaseLock("list3") + store.releaseLock("list3") assert(store.get("list4").isDefined, "list4 was not in store") assert(store.get("list4").get.data.size === 2) - store.unpin("list4") - store.unpin("list4") + store.releaseLock("list4") + store.releaseLock("list4") } test("negative byte values in ByteBufferInputStream") { @@ -1053,7 +1053,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE 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") - store.unpin(rdd(1, 0)) + store.releaseLock(rdd(1, 0)) // According to the same-RDD rule, rdd_1_0 should be replaced here. 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. @@ -1326,9 +1326,9 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(store.getSingle("a1").isDefined, "a1 was not in store") assert(store.getSingle("a2").isDefined, "a2 was not in store") // Release both pins of block a2: - store.unpin("a2") - store.unpin("a2") - assert(store.getPinCount("a2") === 0) + store.releaseLock("a2") + store.releaseLock("a2") + // TODO(josh): assert(store.getPinCount("a2") === 0) // Block a1 is the least-recently accessed, so an LRU eviction policy would evict it before // block a2. However, a1 is still pinned so this put of a3 should evict a2 instead: store.putSingle("a3", arr, StorageLevel.MEMORY_ONLY_SER) diff --git a/core/src/test/scala/org/apache/spark/util/collection/PinCounterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/PinCounterSuite.scala deleted file mode 100644 index ee3b5ab741fcc..0000000000000 --- a/core/src/test/scala/org/apache/spark/util/collection/PinCounterSuite.scala +++ /dev/null @@ -1,114 +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.collection - -import org.scalatest.BeforeAndAfterEach - -import org.apache.spark.SparkFunSuite -import org.apache.spark.storage.{BlockId, TestBlockId} - -class PinCounterSuite extends SparkFunSuite with BeforeAndAfterEach { - - private var refCounter: PinCounter[BlockId] = _ - - override protected def beforeEach(): Unit = { - super.beforeEach() - refCounter = new PinCounter() - } - - override protected def afterEach(): Unit = { - refCounter = null - super.afterEach() - } - - test("initial pin counts are zero") { - assert(refCounter.getNumberOfMapEntries === 0) - assert(refCounter.getPinCount(TestBlockId("dummy")) === 0) - assert(refCounter.getNumberOfMapEntries === 0) - } - - test("error when releasing more times than retained") { - intercept[IllegalStateException] { - refCounter.unpinForTask(taskAttemptId = 0L, TestBlockId("dummy")) - } - } - - test("retain and release from a single task") { - val block = TestBlockId("dummy") - val taskAttemptId = 0L - refCounter.pinForTask(taskAttemptId, block) - assert(refCounter.getPinCount(block) === 1) - refCounter.pinForTask(taskAttemptId, block) - assert(refCounter.getPinCount(block) === 2) - refCounter.unpinForTask(taskAttemptId, block) - assert(refCounter.getPinCount(block) === 1) - refCounter.releaseAllPinsForTask(taskAttemptId) - assert(refCounter.getPinCount(block) === 0L) - // Ensure that we didn't leak memory / map entries: - assert(refCounter.getNumberOfMapEntries === 0) - } - - test("retain and release from multiple tasks") { - val block = TestBlockId("dummy") - val taskA = 0L - val taskB = 1L - - refCounter.pinForTask(taskA, block) - refCounter.pinForTask(taskA, block) - refCounter.pinForTask(taskB, block) - refCounter.pinForTask(taskB, block) - refCounter.pinForTask(taskA, block) - - assert(refCounter.getPinCount(block) === 5) - - refCounter.unpinForTask(taskA, block) - assert(refCounter.getPinCount(block) === 4) - - refCounter.releaseAllPinsForTask(taskA) - assert(refCounter.getPinCount(block) === 2) - - refCounter.unpinForTask(taskB, block) - refCounter.unpinForTask(taskB, block) - assert(refCounter.getPinCount(block) === 0) - - refCounter.releaseAllPinsForTask(taskB) - - // Ensure that we didn't leak memory / map entries: - assert(refCounter.getNumberOfMapEntries === 0) - } - - test("counts are per-object") { - val blockA = TestBlockId("blockA") - val blockB = TestBlockId("blockB") - val taskAttemptId = 0L - - refCounter.pinForTask(taskAttemptId, blockA) - assert(refCounter.getPinCount(blockA) === 1) - assert(refCounter.getPinCount(blockB) === 0) - - refCounter.pinForTask(taskAttemptId, blockB) - refCounter.pinForTask(taskAttemptId, blockB) - assert(refCounter.getPinCount(blockA) === 1) - assert(refCounter.getPinCount(blockB) === 2) - - // Ensure that we didn't leak memory / map entries: - refCounter.releaseAllPinsForTask(taskAttemptId) - assert(refCounter.getNumberOfMapEntries === 0) - } - -} From 47f3174c8ec9206d6d13ea5548275f8e165afca7 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 10 Feb 2016 18:23:19 -0800 Subject: [PATCH 21/81] Implement lock downgrading --- .../apache/spark/storage/BlockInfoManager.scala | 8 ++++++-- .../spark/storage/BlockInfoManagerSuite.scala | 16 ++++++++++++++-- 2 files changed, 20 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index 9afee9cf0c9e2..9de87948bb85a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -108,7 +108,12 @@ private[storage] class BlockInfoManager extends Logging { } def downgradeLock(blockId: BlockId): Unit = synchronized { - + val info = get(blockId).get + require(info.writerTask == currentTaskAttemptId, + s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold") + releaseLock(blockId) + getAndLockForReading(blockId, blocking = false) + notifyAll() } def releaseLock(blockId: BlockId): Unit = synchronized { @@ -206,7 +211,6 @@ private[storage] class BlockInfoManager extends Logging { writeLocksByTask.clear() } - def entries: Iterator[(BlockId, BlockInfo)] = synchronized { infos.iterator.toArray.toIterator } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala index 0ea8789207a8e..b2c3599e30448 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala @@ -90,8 +90,8 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { withTaskId(1) { assert(blockInfoManager.putAndLockForWritingIfAbsent("block", newBlockInfo())) blockInfoManager.releaseLock("block") - assert(blockInfoManager.getAndLockForReading("block", blocking = false).isDefined) - assert(blockInfoManager.getAndLockForReading("block", blocking = false).isDefined) + assert(blockInfoManager.getAndLockForReading("block").isDefined) + assert(blockInfoManager.getAndLockForReading("block").isDefined) assert(blockInfoManager.get("block").get.readerCount === 2) assert(blockInfoManager.get("block").get.writerTask === -1) blockInfoManager.releaseLock("block") @@ -128,6 +128,18 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { } } + test("downgrade lock") { + withTaskId(0) { + assert(blockInfoManager.putAndLockForWritingIfAbsent("block", newBlockInfo())) + blockInfoManager.downgradeLock("block") + } + withTaskId(1) { + assert(blockInfoManager.getAndLockForReading("block").isDefined) + } + assert(blockInfoManager.get("block").get.readerCount === 2) + assert(blockInfoManager.get("block").get.writerTask === -1) + } + test("write lock will block readers") { withTaskId(0) { assert(blockInfoManager.putAndLockForWritingIfAbsent("block", newBlockInfo())) From 4591308226f09135db7696bcd1e8b01b53652b0a Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 10 Feb 2016 18:32:19 -0800 Subject: [PATCH 22/81] Implement remove() --- .../spark/storage/BlockInfoManager.scala | 10 +++++--- .../spark/storage/BlockInfoManagerSuite.scala | 24 +++++++++++++++++++ 2 files changed, 31 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index 9de87948bb85a..e75ba594a390b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -35,6 +35,7 @@ private[storage] class BlockInfo( var size: Long = 0 var readerCount: Int = 0 var writerTask: Long = -1 + var removed: Boolean = false } @@ -76,6 +77,7 @@ private[storage] class BlockInfoManager extends Logging { // TODO: eagerness infos.get(blockId).map { info => while (info.writerTask != -1) { + if (info.removed) return None if (blocking) wait() else return None } // TODO: try to remember why you need actualInfo / the extra get() here. @@ -93,6 +95,7 @@ private[storage] class BlockInfoManager extends Logging { blocking: Boolean = true): Option[BlockInfo] = synchronized { infos.get(blockId).map { info => while (info.writerTask != -1 || info.readerCount != 0) { + if (info.removed) return None if (blocking) wait() else return None } val actualInfo = infos.get(blockId) @@ -200,9 +203,10 @@ private[storage] class BlockInfoManager extends Logging { // This implicitly drops all locks. def remove(blockId: BlockId): Unit = synchronized { // TODO: Should probably have safety checks here - infos.remove(blockId) - // May also want to mark the blockInfo as removed so that gets() don't keep references to it? - // This should also implicitly release locks? No. + infos.remove(blockId).foreach { info => + info.removed = true + } + notifyAll() } def clear(): Unit = synchronized { diff --git a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala index b2c3599e30448..23d6d29f9d9e1 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala @@ -192,4 +192,28 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { assert(Await.result(write1Future, 1.seconds).isDefined) assert(Await.result(write2Future, 1.seconds).isDefined) } + + test("removing a block causes blocked callers to receive None") { + withTaskId(0) { + assert(blockInfoManager.putAndLockForWritingIfAbsent("block", newBlockInfo())) + } + val getFuture = Future { + withTaskId(1) { + blockInfoManager.getAndLockForReading("block") + } + } + val writeFuture = Future { + withTaskId(2) { + blockInfoManager.getAndLockForWriting("block") + } + } + Thread.sleep(300) // Hack to try to ensure that both future tasks are waiting + withTaskId(0) { + blockInfoManager.remove("block") + } + assert(Await.result(getFuture, 1.seconds).isEmpty) + assert(Await.result(writeFuture, 1.seconds).isEmpty) + assert(blockInfoManager.getNumberOfMapEntries === 0) + assert(blockInfoManager.size === 0) + } } \ No newline at end of file From 77939c2bf60b3425b2df682b7dab8352ef7edfdf Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 11 Feb 2016 12:46:37 -0800 Subject: [PATCH 23/81] BlockInfoManagerSuite tests now pass. --- .../spark/storage/BlockInfoManager.scala | 40 +- .../apache/spark/storage/BlockManager.scala | 16 +- .../spark/storage/BlockInfoManagerSuite.scala | 11 +- .../spark/storage/BlockManagerSuite.scala | 509 ++++++++++-------- 4 files changed, 320 insertions(+), 256 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index e75ba594a390b..04a8c61aa4ec0 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -62,7 +62,7 @@ private[storage] class BlockInfoManager extends Logging { // ---------------------------------------------------------------------------------------------- private def currentTaskAttemptId: TaskAttemptId = { - Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1L) + Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L) } /** @@ -74,7 +74,7 @@ private[storage] class BlockInfoManager extends Logging { def getAndLockForReading( blockId: BlockId, blocking: Boolean = true): Option[BlockInfo] = synchronized { - // TODO: eagerness + logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId") infos.get(blockId).map { info => while (info.writerTask != -1) { if (info.removed) return None @@ -86,6 +86,7 @@ private[storage] class BlockInfoManager extends Logging { i.readerCount += 1 readLocksByTask(currentTaskAttemptId).add(blockId) } + logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId") info } } @@ -93,15 +94,20 @@ private[storage] class BlockInfoManager extends Logging { def getAndLockForWriting( blockId: BlockId, blocking: Boolean = true): Option[BlockInfo] = synchronized { + logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId") infos.get(blockId).map { info => - while (info.writerTask != -1 || info.readerCount != 0) { - if (info.removed) return None - if (blocking) wait() else return None + if (info.writerTask != currentTaskAttemptId) { + while (info.writerTask != -1 || info.readerCount != 0) { + if (info.removed) return None + if (blocking) wait() else return None + } } val actualInfo = infos.get(blockId) actualInfo.foreach { i => i.writerTask = currentTaskAttemptId + writeLocksByTask.addBinding(currentTaskAttemptId, blockId) } + logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId") info } } @@ -111,6 +117,8 @@ private[storage] class BlockInfoManager extends Logging { } def downgradeLock(blockId: BlockId): Unit = synchronized { + logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId") + // TODO: refactor this code so that log messages aren't confusing. val info = get(blockId).get require(info.writerTask == currentTaskAttemptId, s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold") @@ -120,15 +128,20 @@ private[storage] class BlockInfoManager extends Logging { } def releaseLock(blockId: BlockId): Unit = synchronized { - val info = get(blockId).get + logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId") + val info = get(blockId).getOrElse { + throw new IllegalStateException(s"Block $blockId not found") + } if (info.writerTask != -1) { info.writerTask = -1 + writeLocksByTask.removeBinding(currentTaskAttemptId, blockId) } else { - assert(info.readerCount > 0) + assert(info.readerCount > 0, s"Block $blockId is not locked for reading") info.readerCount -= 1 val countsForTask = readLocksByTask.get(currentTaskAttemptId) val newPinCountForTask: Int = countsForTask.remove(blockId, 1) - 1 - assert(newPinCountForTask >= 0) + assert(newPinCountForTask >= 0, + s"Task $currentTaskAttemptId release lock on block $blockId more times than it acquired it") } notifyAll() } @@ -136,9 +149,11 @@ private[storage] class BlockInfoManager extends Logging { def putAndLockForWritingIfAbsent( blockId: BlockId, newBlockInfo: BlockInfo): Boolean = synchronized { + logTrace(s"Task $currentTaskAttemptId trying to put $blockId") val actualInfo = infos.getOrElseUpdate(blockId, newBlockInfo) if (actualInfo eq newBlockInfo) { actualInfo.writerTask = currentTaskAttemptId + writeLocksByTask.addBinding(currentTaskAttemptId, blockId) true } else { false @@ -152,11 +167,11 @@ private[storage] class BlockInfoManager extends Logging { * * @return the number of pins released */ - def releaseAllPinsForTask(taskAttemptId: TaskAttemptId): Int = { + def releaseAllLocksForTask(taskAttemptId: TaskAttemptId): Int = { synchronized { writeLocksByTask.remove(taskAttemptId).foreach { locks => - for (lock <- locks) { - infos.get(lock).foreach { info => + for (blockId <- locks) { + infos.get(blockId).foreach { info => assert(info.writerTask == taskAttemptId) info.writerTask = -1 } @@ -192,7 +207,7 @@ private[storage] class BlockInfoManager extends Logging { * Return the number of map entries in this pin counter's internal data structures. * This is used in unit tests in order to detect memory leaks. */ - private[storage] def getNumberOfMapEntries: Long = synchronized { + private[storage] def getNumberOfMapEntries: Long = synchronized { size + readLocksByTask.size() + readLocksByTask.asMap().asScala.map(_._2.size()).sum + @@ -202,6 +217,7 @@ private[storage] class BlockInfoManager extends Logging { // This implicitly drops all locks. def remove(blockId: BlockId): Unit = synchronized { + logTrace(s"Task $currentTaskAttemptId trying to remove block $blockId") // TODO: Should probably have safety checks here infos.remove(blockId).foreach { info => info.removed = true 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 496f2b1496115..4efd5e4ce7399 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -19,7 +19,6 @@ package org.apache.spark.storage import java.io._ import java.nio.{ByteBuffer, MappedByteBuffer} -import java.util.concurrent.ConcurrentHashMap import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.concurrent.{Await, ExecutionContext, Future} @@ -405,16 +404,6 @@ private[spark] class BlockManager( doGetLocal(blockId, asBlockResult = true).asInstanceOf[Option[BlockResult]] } - /** - * Return true if local block manager contains the given block and false otherwise. - * Calling this method will _not_ implicitly pin the block, making this method useful - * when writing tests. - */ - def hasLocalBlock(blockId: BlockId): Boolean = { - // TODO(josh): this needs to be reworked - doGetLocal(blockId, asBlockResult = true).asInstanceOf[Option[BlockResult]].isDefined - } - /** * Get block from the local block manager as serialized bytes. */ @@ -464,6 +453,7 @@ private[spark] class BlockManager( val bytes: ByteBuffer = diskStore.getBytes(blockId) match { case Some(b) => b case None => + releaseLock(blockId) throw new BlockException( blockId, s"Block $blockId not found on disk, though it should be") } @@ -516,6 +506,7 @@ private[spark] class BlockManager( } } } else { + releaseLock(blockId) None } } @@ -617,7 +608,7 @@ private[spark] class BlockManager( * @return the total number of pins released. */ def releaseAllLocksForTask(taskAttemptId: Long): Int = { - blockInfoManager.releaseAllPinsForTask(taskAttemptId) + blockInfoManager.releaseAllLocksForTask(taskAttemptId) } /** @@ -798,6 +789,7 @@ private[spark] class BlockManager( // Now that the block is in either the memory, externalBlockStore, or disk store, // let other threads read it, and tell the master about it. blockWasSuccessfullyStored = true + putBlockInfo.size = size blockInfoManager.downgradeLock(blockId) if (tellMaster) { reportBlockStatus(blockId, putBlockInfo, putBlockStatus) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala index 23d6d29f9d9e1..5c5fc6f937a3d 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala @@ -213,7 +213,14 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { } assert(Await.result(getFuture, 1.seconds).isEmpty) assert(Await.result(writeFuture, 1.seconds).isEmpty) - assert(blockInfoManager.getNumberOfMapEntries === 0) - assert(blockInfoManager.size === 0) + } + + test("releaseAllLocksForTask releases write locks") { + withTaskId(0) { + assert(blockInfoManager.putAndLockForWritingIfAbsent("block", newBlockInfo())) + } + assert(blockInfoManager.getNumberOfMapEntries === 3) + blockInfoManager.releaseAllLocksForTask(0) + assert(blockInfoManager.getNumberOfMapEntries === 1) } } \ No newline at end of file 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 bb0bb35f9fc84..6c40541101169 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -45,6 +45,8 @@ import org.apache.spark.util._ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterEach with PrivateMethodTester with ResetSystemProperties { + import BlockManagerSuite._ + var conf: SparkConf = null var store: BlockManager = null var store2: BlockManager = null @@ -170,17 +172,14 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE 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) - store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY) - store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY, tellMaster = false) + store.putSingleAndReleaseLock("a1", a1, StorageLevel.MEMORY_ONLY) + store.putSingleAndReleaseLock("a2", a2, StorageLevel.MEMORY_ONLY) + store.putSingleAndReleaseLock("a3", a3, StorageLevel.MEMORY_ONLY, tellMaster = false) // Checking whether blocks are in memory - assert(store.getSingle("a1").isDefined, "a1 was not in store") - store.releaseLock("a1") - assert(store.getSingle("a2").isDefined, "a2 was not in store") - store.releaseLock("a2") - assert(store.getSingle("a3").isDefined, "a3 was not in store") - store.releaseLock("a3") + assert(store.getSingleAndReleaseLock("a1").isDefined, "a1 was not in store") + assert(store.getSingleAndReleaseLock("a2").isDefined, "a2 was not in store") + assert(store.getSingleAndReleaseLock("a3").isDefined, "a3 was not in store") // Checking whether master knows about the blocks or not assert(master.getLocations("a1").size > 0, "master was not told about a1") @@ -190,8 +189,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // Drop a1 and a2 from memory; this should be reported back to the master store.dropFromMemory("a1", () => null: Either[Array[Any], ByteBuffer]) store.dropFromMemory("a2", () => null: Either[Array[Any], ByteBuffer]) - assert(store.getSingle("a1") === None, "a1 not removed from store") - assert(store.getSingle("a2") === None, "a2 not removed from store") + assert(store.getSingleAndReleaseLock("a1") === None, "a1 not removed from store") + assert(store.getSingleAndReleaseLock("a2") === None, "a2 not removed from store") assert(master.getLocations("a1").size === 0, "master did not remove a1") assert(master.getLocations("a2").size === 0, "master did not remove a2") } @@ -206,7 +205,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) - store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_2) + store.putSingleAndReleaseLock("a1", a1, StorageLevel.MEMORY_ONLY_2) store2.putSingle("a2", a2, StorageLevel.MEMORY_ONLY_2) assert(master.getLocations("a1").size === 2, "master did not report 2 locations for a1") assert(master.getLocations("a2").size === 2, "master did not report 2 locations for a2") @@ -219,20 +218,17 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE 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) - store.putSingle("a2-to-remove", a2, StorageLevel.MEMORY_ONLY) - store.putSingle("a3-to-remove", a3, StorageLevel.MEMORY_ONLY, tellMaster = false) + store.putSingleAndReleaseLock("a1-to-remove", a1, StorageLevel.MEMORY_ONLY) + store.putSingleAndReleaseLock("a2-to-remove", a2, StorageLevel.MEMORY_ONLY) + store.putSingleAndReleaseLock("a3-to-remove", a3, StorageLevel.MEMORY_ONLY, tellMaster = false) // Checking whether blocks are in memory and memory size val memStatus = master.getMemoryStatus.head._2 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") - store.releaseLock("a1-to-remove") - assert(store.getSingle("a2-to-remove").isDefined, "a2 was not in store") - store.releaseLock("a2-to-remove") - assert(store.getSingle("a3-to-remove").isDefined, "a3 was not in store") - store.releaseLock("a3-to-remove") + assert(store.getSingleAndReleaseLock("a1-to-remove").isDefined, "a1 was not in store") + assert(store.getSingleAndReleaseLock("a2-to-remove").isDefined, "a2 was not in store") + assert(store.getSingleAndReleaseLock("a3-to-remove").isDefined, "a3 was not in store") // Checking whether master knows about the blocks or not assert(master.getLocations("a1-to-remove").size > 0, "master was not told about a1") @@ -269,30 +265,30 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE 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) - store.putSingle("nonrddblock", a3, StorageLevel.MEMORY_ONLY) + store.putSingleAndReleaseLock(rdd(0, 0), a1, StorageLevel.MEMORY_ONLY) + store.putSingleAndReleaseLock(rdd(0, 1), a2, StorageLevel.MEMORY_ONLY) + store.putSingleAndReleaseLock("nonrddblock", a3, StorageLevel.MEMORY_ONLY) master.removeRdd(0, blocking = false) eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { - store.getSingle(rdd(0, 0)) should be (None) + store.getSingleAndReleaseLock(rdd(0, 0)) should be (None) master.getLocations(rdd(0, 0)) should have size 0 } eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { - store.getSingle(rdd(0, 1)) should be (None) + store.getSingleAndReleaseLock(rdd(0, 1)) should be (None) master.getLocations(rdd(0, 1)) should have size 0 } eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { - store.getSingle("nonrddblock") should not be (None) + store.getSingleAndReleaseLock("nonrddblock") should not be (None) master.getLocations("nonrddblock") should have size (1) } - store.putSingle(rdd(0, 0), a1, StorageLevel.MEMORY_ONLY) - store.putSingle(rdd(0, 1), a2, StorageLevel.MEMORY_ONLY) + store.putSingleAndReleaseLock(rdd(0, 0), a1, StorageLevel.MEMORY_ONLY) + store.putSingleAndReleaseLock(rdd(0, 1), a2, StorageLevel.MEMORY_ONLY) master.removeRdd(0, blocking = true) - store.getSingle(rdd(0, 0)) should be (None) + store.getSingleAndReleaseLock(rdd(0, 0)) should be (None) master.getLocations(rdd(0, 0)) should have size 0 - store.getSingle(rdd(0, 1)) should be (None) + store.getSingleAndReleaseLock(rdd(0, 1)) should be (None) master.getLocations(rdd(0, 1)) should have size 0 } @@ -312,10 +308,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // insert broadcast blocks in both the stores Seq(driverStore, executorStore).foreach { case s => - s.putSingle(broadcast0BlockId, a1, StorageLevel.DISK_ONLY) - s.putSingle(broadcast1BlockId, a2, StorageLevel.DISK_ONLY) - s.putSingle(broadcast2BlockId, a3, StorageLevel.DISK_ONLY) - s.putSingle(broadcast2BlockId2, a4, StorageLevel.DISK_ONLY) + s.putSingleAndReleaseLock(broadcast0BlockId, a1, StorageLevel.DISK_ONLY) + s.putSingleAndReleaseLock(broadcast1BlockId, a2, StorageLevel.DISK_ONLY) + s.putSingleAndReleaseLock(broadcast2BlockId, a3, StorageLevel.DISK_ONLY) + s.putSingleAndReleaseLock(broadcast2BlockId2, a4, StorageLevel.DISK_ONLY) } // verify whether the blocks exist in both the stores @@ -370,9 +366,9 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE store = makeBlockManager(2000) val a1 = new Array[Byte](400) - store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) + store.putSingleAndReleaseLock("a1", a1, StorageLevel.MEMORY_ONLY) - assert(store.getSingle("a1").isDefined, "a1 was not in store") + assert(store.getSingleAndReleaseLock("a1").isDefined, "a1 was not in store") assert(master.getLocations("a1").size > 0, "master was not told about a1") master.removeExecutor(store.blockManagerId.executorId) @@ -388,13 +384,13 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) - store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) + store.putSingleAndReleaseLock("a1", a1, StorageLevel.MEMORY_ONLY) assert(master.getLocations("a1").size > 0, "master was not told about a1") master.removeExecutor(store.blockManagerId.executorId) assert(master.getLocations("a1").size == 0, "a1 was not removed from master") - store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY) + store.putSingleAndReleaseLock("a2", a2, StorageLevel.MEMORY_ONLY) store.waitForAsyncReregister() assert(master.getLocations("a1").size > 0, "a1 was not reregistered with master") @@ -411,12 +407,13 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE master.removeExecutor(store.blockManagerId.executorId) val t1 = new Thread { override def run() { - store.putIterator("a2", a2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIteratorAndReleaseLock( + "a2", a2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) } } val t2 = new Thread { override def run() { - store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) + store.putSingleAndReleaseLock("a1", a1, StorageLevel.MEMORY_ONLY) } } val t3 = new Thread { @@ -444,20 +441,23 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE 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.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") + store.putIteratorAndReleaseLock( + "list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIteratorAndReleaseLock( + "list2memory", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIteratorAndReleaseLock( + "list2disk", list2.iterator, StorageLevel.DISK_ONLY, tellMaster = true) + val list1Get = store.getAndReleaseLock("list1") assert(list1Get.isDefined, "list1 expected to be in store") assert(list1Get.get.data.size === 2) assert(list1Get.get.bytes === list1SizeEstimate) assert(list1Get.get.readMethod === DataReadMethod.Memory) - val list2MemoryGet = store.get("list2memory") + val list2MemoryGet = store.getAndReleaseLock("list2memory") assert(list2MemoryGet.isDefined, "list2memory expected to be in store") assert(list2MemoryGet.get.data.size === 3) assert(list2MemoryGet.get.bytes === list2SizeEstimate) assert(list2MemoryGet.get.readMethod === DataReadMethod.Memory) - val list2DiskGet = store.get("list2disk") + val list2DiskGet = store.getAndReleaseLock("list2disk") assert(list2DiskGet.isDefined, "list2memory expected to be in store") assert(list2DiskGet.get.data.size === 3) // We don't know the exact size of the data on disk, but it should certainly be > 0. @@ -486,8 +486,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE store2 = makeBlockManager(8000, "executor2") store3 = makeBlockManager(8000, "executor3") val list1 = List(new Array[Byte](4000)) - store2.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store3.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store2.putIteratorAndReleaseLock( + "list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store3.putIteratorAndReleaseLock( + "list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(store.getRemoteBytes("list1").isDefined, "list1Get expected to be fetched") store2.stop() store2 = null @@ -513,21 +515,18 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) - store.putSingle("a1", a1, storageLevel) - store.putSingle("a2", a2, storageLevel) - store.putSingle("a3", a3, storageLevel) - assert(store.getSingle("a2").isDefined, "a2 was not in store") - store.releaseLock("a2") - assert(store.getSingle("a3").isDefined, "a3 was not in store") - store.releaseLock("a3") - assert(store.getSingle("a1") === None, "a1 was in store") - assert(store.getSingle("a2").isDefined, "a2 was not in store") - store.releaseLock("a2") + store.putSingleAndReleaseLock("a1", a1, storageLevel) + store.putSingleAndReleaseLock("a2", a2, storageLevel) + store.putSingleAndReleaseLock("a3", a3, storageLevel) + assert(store.getSingleAndReleaseLock("a2").isDefined, "a2 was not in store") + assert(store.getSingleAndReleaseLock("a3").isDefined, "a3 was not in store") + assert(store.getSingleAndReleaseLock("a1") === None, "a1 was in store") + assert(store.getSingleAndReleaseLock("a2").isDefined, "a2 was not in store") // At this point a2 was gotten last, so LRU will getSingle rid of a3 - store.putSingle("a1", a1, storageLevel) - assert(store.getSingle("a1").isDefined, "a1 was not in store") - assert(store.getSingle("a2").isDefined, "a2 was not in store") - assert(store.getSingle("a3") === None, "a3 was in store") + store.putSingleAndReleaseLock("a1", a1, storageLevel) + assert(store.getSingleAndReleaseLock("a1").isDefined, "a1 was not in store") + assert(store.getSingleAndReleaseLock("a2").isDefined, "a2 was not in store") + assert(store.getSingleAndReleaseLock("a3") === None, "a3 was in store") } test("in-memory LRU for partitions of same RDD") { @@ -535,34 +534,34 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE 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) + store.putSingleAndReleaseLock(rdd(0, 1), a1, StorageLevel.MEMORY_ONLY) + store.putSingleAndReleaseLock(rdd(0, 2), a2, StorageLevel.MEMORY_ONLY) + store.putSingleAndReleaseLock(rdd(0, 3), a3, StorageLevel.MEMORY_ONLY) // Even though we accessed rdd_0_3 last, it should not have replaced partitions 1 and 2 // from the same RDD - assert(store.getSingle(rdd(0, 3)) === None, "rdd_0_3 was in store") - assert(store.getSingle(rdd(0, 2)).isDefined, "rdd_0_2 was not in store") - assert(store.getSingle(rdd(0, 1)).isDefined, "rdd_0_1 was not in store") + assert(store.getSingleAndReleaseLock(rdd(0, 3)) === None, "rdd_0_3 was in store") + assert(store.getSingleAndReleaseLock(rdd(0, 2)).isDefined, "rdd_0_2 was not in store") + assert(store.getSingleAndReleaseLock(rdd(0, 1)).isDefined, "rdd_0_1 was not in store") // Check that rdd_0_3 doesn't replace them even after further accesses - assert(store.getSingle(rdd(0, 3)) === None, "rdd_0_3 was in store") - assert(store.getSingle(rdd(0, 3)) === None, "rdd_0_3 was in store") - assert(store.getSingle(rdd(0, 3)) === None, "rdd_0_3 was in store") + assert(store.getSingleAndReleaseLock(rdd(0, 3)) === None, "rdd_0_3 was in store") + assert(store.getSingleAndReleaseLock(rdd(0, 3)) === None, "rdd_0_3 was in store") + assert(store.getSingleAndReleaseLock(rdd(0, 3)) === None, "rdd_0_3 was in store") } test("in-memory LRU for partitions of multiple RDDs") { 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) + store.putSingleAndReleaseLock(rdd(0, 1), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) + store.putSingleAndReleaseLock(rdd(0, 2), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) + store.putSingleAndReleaseLock(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") assert(store.memoryStore.contains(rdd(0, 2)), "rdd_0_2 was not in store") // 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") + assert(store.getSingleAndReleaseLock(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](4000), StorageLevel.MEMORY_ONLY) - store.putSingle(rdd(0, 4), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) + store.putSingleAndReleaseLock(rdd(0, 3), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) + store.putSingleAndReleaseLock(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") @@ -577,28 +576,28 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) - store.putSingle("a1", a1, StorageLevel.DISK_ONLY) - store.putSingle("a2", a2, StorageLevel.DISK_ONLY) - store.putSingle("a3", a3, StorageLevel.DISK_ONLY) - assert(store.getSingle("a2").isDefined, "a2 was in store") - assert(store.getSingle("a3").isDefined, "a3 was in store") - assert(store.getSingle("a1").isDefined, "a1 was in store") + store.putSingleAndReleaseLock("a1", a1, StorageLevel.DISK_ONLY) + store.putSingleAndReleaseLock("a2", a2, StorageLevel.DISK_ONLY) + store.putSingleAndReleaseLock("a3", a3, StorageLevel.DISK_ONLY) + assert(store.getSingleAndReleaseLock("a2").isDefined, "a2 was in store") + assert(store.getSingleAndReleaseLock("a3").isDefined, "a3 was in store") + assert(store.getSingleAndReleaseLock("a1").isDefined, "a1 was in store") } test("disk and memory storage") { - testDiskAndMemoryStorage(StorageLevel.MEMORY_AND_DISK, _.getSingle) + testDiskAndMemoryStorage(StorageLevel.MEMORY_AND_DISK, _.getSingleAndReleaseLock) } test("disk and memory storage with getLocalBytes") { - testDiskAndMemoryStorage(StorageLevel.MEMORY_AND_DISK, _.getLocalBytes) + testDiskAndMemoryStorage(StorageLevel.MEMORY_AND_DISK, _.getLocalBytesAndReleaseLock) } test("disk and memory storage with serialization") { - testDiskAndMemoryStorage(StorageLevel.MEMORY_AND_DISK_SER, _.getSingle) + testDiskAndMemoryStorage(StorageLevel.MEMORY_AND_DISK_SER, _.getSingleAndReleaseLock) } test("disk and memory storage with serialization and getLocalBytes") { - testDiskAndMemoryStorage(StorageLevel.MEMORY_AND_DISK_SER, _.getLocalBytes) + testDiskAndMemoryStorage(StorageLevel.MEMORY_AND_DISK_SER, _.getLocalBytesAndReleaseLock) } def testDiskAndMemoryStorage( @@ -608,16 +607,13 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) - store.putSingle("a1", a1, storageLevel) - store.putSingle("a2", a2, storageLevel) - store.putSingle("a3", a3, storageLevel) + store.putSingleAndReleaseLock("a1", a1, storageLevel) + store.putSingleAndReleaseLock("a2", a2, storageLevel) + store.putSingleAndReleaseLock("a3", a3, storageLevel) assert(accessMethod(store)("a2").isDefined, "a2 was not in store") - store.releaseLock("a2") assert(accessMethod(store)("a3").isDefined, "a3 was not in store") - store.releaseLock("a3") assert(store.memoryStore.getValues("a1").isEmpty, "a1 was in memory store") assert(accessMethod(store)("a1").isDefined, "a1 was not in store") - store.releaseLock("a1") assert(store.memoryStore.getValues("a1").isDefined, "a1 was not in memory store") } @@ -628,25 +624,19 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE 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) - store.putSingle("a3", a3, StorageLevel.DISK_ONLY) + store.putSingleAndReleaseLock("a1", a1, StorageLevel.MEMORY_ONLY_SER) + store.putSingleAndReleaseLock("a2", a2, StorageLevel.MEMORY_ONLY_SER) + store.putSingleAndReleaseLock("a3", a3, StorageLevel.DISK_ONLY) // At this point LRU should not kick in because a3 is only on disk - assert(store.getSingle("a1").isDefined, "a1 was not in store") - store.releaseLock("a1") - assert(store.getSingle("a2").isDefined, "a2 was not in store") - store.releaseLock("a2") - assert(store.getSingle("a3").isDefined, "a3 was not in store") - store.releaseLock("a3") + assert(store.getSingleAndReleaseLock("a1").isDefined, "a1 was not in store") + assert(store.getSingleAndReleaseLock("a2").isDefined, "a2 was not in store") + assert(store.getSingleAndReleaseLock("a3").isDefined, "a3 was not in store") // Now let's add in a4, which uses both disk and memory; a1 should drop out - store.putSingle("a4", a4, StorageLevel.MEMORY_AND_DISK_SER) - assert(store.getSingle("a1") == None, "a1 was in store") - assert(store.getSingle("a2").isDefined, "a2 was not in store") - store.releaseLock("a2") - assert(store.getSingle("a3").isDefined, "a3 was not in store") - store.releaseLock("a3") - assert(store.getSingle("a4").isDefined, "a4 was not in store") - store.releaseLock("a4") + store.putSingleAndReleaseLock("a4", a4, StorageLevel.MEMORY_AND_DISK_SER) + assert(store.getSingleAndReleaseLock("a1") == None, "a1 was in store") + assert(store.getSingleAndReleaseLock("a2").isDefined, "a2 was not in store") + assert(store.getSingleAndReleaseLock("a3").isDefined, "a3 was not in store") + assert(store.getSingleAndReleaseLock("a4").isDefined, "a4 was not in store") } test("in-memory LRU with streams") { @@ -654,33 +644,27 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE 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.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) - store.releaseLock("list2") - store.releaseLock("list2") - assert(store.get("list3").isDefined, "list3 was not in store") - assert(store.get("list3").get.data.size === 2) - store.releaseLock("list3") - store.releaseLock("list3") - 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) - store.releaseLock("list2") - store.releaseLock("list2") + store.putIteratorAndReleaseLock( + "list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIteratorAndReleaseLock( + "list2", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIteratorAndReleaseLock( + "list3", list3.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + assert(store.getAndReleaseLock("list2").isDefined, "list2 was not in store") + assert(store.getAndReleaseLock("list2").get.data.size === 2) + assert(store.getAndReleaseLock("list3").isDefined, "list3 was not in store") + assert(store.getAndReleaseLock("list3").get.data.size === 2) + assert(store.getAndReleaseLock("list1") === None, "list1 was in store") + assert(store.getAndReleaseLock("list2").isDefined, "list2 was not in store") + assert(store.getAndReleaseLock("list2").get.data.size === 2) // At this point list2 was gotten last, so LRU will getSingle rid of list3 - 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) - store.releaseLock("list1") - store.releaseLock("list1") - assert(store.get("list2").isDefined, "list2 was not in store") - assert(store.get("list2").get.data.size === 2) - store.releaseLock("list2") - store.releaseLock("list2") - assert(store.get("list3") === None, "list1 was in store") + store.putIteratorAndReleaseLock( + "list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + assert(store.getAndReleaseLock("list1").isDefined, "list1 was not in store") + assert(store.getAndReleaseLock("list1").get.data.size === 2) + assert(store.getAndReleaseLock("list2").isDefined, "list2 was not in store") + assert(store.getAndReleaseLock("list2").get.data.size === 2) + assert(store.getAndReleaseLock("list3") === None, "list1 was in store") } test("LRU with mixed storage levels and streams") { @@ -690,52 +674,38 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE 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.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) + store.putIteratorAndReleaseLock( + "list1", list1.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) + store.putIteratorAndReleaseLock( + "list2", list2.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) + store.putIteratorAndReleaseLock( + "list3", list3.iterator, StorageLevel.DISK_ONLY, tellMaster = true) val listForSizeEstimate = new ArrayBuffer[Any] listForSizeEstimate ++= list1.iterator val listSize = SizeEstimator.estimate(listForSizeEstimate) // At this point LRU should not kick in because list3 is only on disk - assert(store.get("list1").isDefined, "list1 was not in store") - assert(store.get("list1").get.data.size === 2) - store.releaseLock("list1") - store.releaseLock("list1") - assert(store.get("list2").isDefined, "list2 was not in store") - assert(store.get("list2").get.data.size === 2) - store.releaseLock("list2") - store.releaseLock("list2") - assert(store.get("list3").isDefined, "list3 was not in store") - assert(store.get("list3").get.data.size === 2) - store.releaseLock("list3") - store.releaseLock("list3") - assert(store.get("list1").isDefined, "list1 was not in store") - assert(store.get("list1").get.data.size === 2) - store.releaseLock("list1") - store.releaseLock("list1") - assert(store.get("list2").isDefined, "list2 was not in store") - assert(store.get("list2").get.data.size === 2) - store.releaseLock("list2") - store.releaseLock("list2") - assert(store.get("list3").isDefined, "list3 was not in store") - assert(store.get("list3").get.data.size === 2) - store.releaseLock("list3") - store.releaseLock("list3") + assert(store.getAndReleaseLock("list1").isDefined, "list1 was not in store") + assert(store.getAndReleaseLock("list1").get.data.size === 2) + assert(store.getAndReleaseLock("list2").isDefined, "list2 was not in store") + assert(store.getAndReleaseLock("list2").get.data.size === 2) + assert(store.getAndReleaseLock("list3").isDefined, "list3 was not in store") + assert(store.getAndReleaseLock("list3").get.data.size === 2) + assert(store.getAndReleaseLock("list1").isDefined, "list1 was not in store") + assert(store.getAndReleaseLock("list1").get.data.size === 2) + assert(store.getAndReleaseLock("list2").isDefined, "list2 was not in store") + assert(store.getAndReleaseLock("list2").get.data.size === 2) + assert(store.getAndReleaseLock("list3").isDefined, "list3 was not in store") + assert(store.getAndReleaseLock("list3").get.data.size === 2) // Now let's add in list4, which uses both disk and memory; list1 should drop out - 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) - store.releaseLock("list2") - store.releaseLock("list2") - assert(store.get("list3").isDefined, "list3 was not in store") - assert(store.get("list3").get.data.size === 2) - store.releaseLock("list3") - store.releaseLock("list3") - assert(store.get("list4").isDefined, "list4 was not in store") - assert(store.get("list4").get.data.size === 2) - store.releaseLock("list4") - store.releaseLock("list4") + store.putIteratorAndReleaseLock( + "list4", list4.iterator, StorageLevel.MEMORY_AND_DISK_SER, tellMaster = true) + assert(store.getAndReleaseLock("list1") === None, "list1 was in store") + assert(store.getAndReleaseLock("list2").isDefined, "list2 was not in store") + assert(store.getAndReleaseLock("list2").get.data.size === 2) + assert(store.getAndReleaseLock("list3").isDefined, "list3 was not in store") + assert(store.getAndReleaseLock("list3").get.data.size === 2) + assert(store.getAndReleaseLock("list4").isDefined, "list4 was not in store") + assert(store.getAndReleaseLock("list4").get.data.size === 2) } test("negative byte values in ByteBufferInputStream") { @@ -752,18 +722,19 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE test("overly large block") { 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) + store.putSingleAndReleaseLock("a1", new Array[Byte](10000), StorageLevel.MEMORY_ONLY) + assert(store.getSingleAndReleaseLock("a1") === None, "a1 was in store") + store.putSingleAndReleaseLock("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") + assert(store.getSingleAndReleaseLock("a2").isDefined, "a2 was not in store") } test("block compression") { try { conf.set("spark.shuffle.compress", "true") store = makeBlockManager(20000, "exec1") - store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) + store.putSingleAndReleaseLock( + 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") store.stop() @@ -771,7 +742,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE conf.set("spark.shuffle.compress", "false") store = makeBlockManager(20000, "exec2") - store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + store.putSingleAndReleaseLock( + 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() @@ -779,7 +751,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE conf.set("spark.broadcast.compress", "true") store = makeBlockManager(20000, "exec3") - store.putSingle(BroadcastBlockId(0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + store.putSingleAndReleaseLock( + BroadcastBlockId(0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(BroadcastBlockId(0)) <= 1000, "broadcast_0 was not compressed") store.stop() @@ -787,28 +760,29 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE conf.set("spark.broadcast.compress", "false") store = makeBlockManager(20000, "exec4") - store.putSingle(BroadcastBlockId(0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + store.putSingleAndReleaseLock( + 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 = makeBlockManager(20000, "exec5") - store.putSingle(rdd(0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + store.putSingleAndReleaseLock(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 = makeBlockManager(20000, "exec6") - store.putSingle(rdd(0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + store.putSingleAndReleaseLock(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 = makeBlockManager(20000, "exec7") - store.putSingle("other_block", new Array[Byte](10000), StorageLevel.MEMORY_ONLY) + store.putSingleAndReleaseLock("other_block", new Array[Byte](10000), StorageLevel.MEMORY_ONLY) assert(store.memoryStore.getSize("other_block") >= 10000, "other_block was compressed") store.stop() store = null @@ -836,12 +810,12 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE class UnserializableClass val a1 = new UnserializableClass intercept[java.io.NotSerializableException] { - store.putSingle("a1", a1, StorageLevel.DISK_ONLY) + store.putSingleAndReleaseLock("a1", a1, StorageLevel.DISK_ONLY) } // Make sure get a1 doesn't hang and returns None. failAfter(1 second) { - assert(store.getSingle("a1").isEmpty, "a1 should not be in store") + assert(store.getSingleAndReleaseLock("a1").isEmpty, "a1 should not be in store") } } @@ -907,7 +881,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // 1 updated block (i.e. list1) val updatedBlocks1 = getUpdatedBlocks { - store.putIterator("list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIteratorAndReleaseLock( + "list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) } assert(updatedBlocks1.size === 1) assert(updatedBlocks1.head._1 === TestBlockId("list1")) @@ -915,7 +890,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // 1 updated block (i.e. list2) val updatedBlocks2 = getUpdatedBlocks { - store.putIterator("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIteratorAndReleaseLock( + "list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) } assert(updatedBlocks2.size === 1) assert(updatedBlocks2.head._1 === TestBlockId("list2")) @@ -923,7 +899,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // 2 updated blocks - list1 is kicked out of memory while list3 is added val updatedBlocks3 = getUpdatedBlocks { - store.putIterator("list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIteratorAndReleaseLock( + "list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) } assert(updatedBlocks3.size === 2) updatedBlocks3.foreach { case (id, status) => @@ -937,7 +914,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // 2 updated blocks - list2 is kicked out of memory (but put on disk) while list4 is added val updatedBlocks4 = getUpdatedBlocks { - store.putIterator("list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIteratorAndReleaseLock( + "list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) } assert(updatedBlocks4.size === 2) updatedBlocks4.foreach { case (id, status) => @@ -952,7 +930,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // No updated blocks - list5 is too big to fit in store and nothing is kicked out val updatedBlocks5 = getUpdatedBlocks { - store.putIterator("list5", bigList.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIteratorAndReleaseLock( + "list5", bigList.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) } assert(updatedBlocks5.size === 0) @@ -976,9 +955,12 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val list = List.fill(2)(new Array[Byte](2000)) // Tell master. By LRU, only list2 and list3 remains. - 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) + store.putIteratorAndReleaseLock( + "list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIteratorAndReleaseLock( + "list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIteratorAndReleaseLock( + "list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) // getLocations and getBlockStatus should yield the same locations assert(store.master.getLocations("list1").size === 0) @@ -992,9 +974,12 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE 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.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) + store.putIteratorAndReleaseLock( + "list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) + store.putIteratorAndReleaseLock( + "list5", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + store.putIteratorAndReleaseLock( + "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 @@ -1015,9 +1000,12 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val list = List.fill(2)(new Array[Byte](100)) // insert some blocks - 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) + store.putIteratorAndReleaseLock( + "list1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIteratorAndReleaseLock( + "list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIteratorAndReleaseLock( + "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 @@ -1026,9 +1014,12 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE === 1) // insert some more blocks - 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) + store.putIteratorAndReleaseLock( + "newlist1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIteratorAndReleaseLock( + "newlist2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + store.putIteratorAndReleaseLock( + "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 @@ -1038,7 +1029,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val blockIds = Seq(RDDBlockId(1, 0), RDDBlockId(1, 1), RDDBlockId(2, 0)) blockIds.foreach { blockId => - store.putIterator(blockId, list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIteratorAndReleaseLock( + blockId, list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) } val matchedBlockIds = store.master.getMatchingBlockIds(_ match { case RDDBlockId(1, _) => true @@ -1049,13 +1041,12 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE test("SPARK-1194 regression: fix the same-RDD rule for cache replacement") { 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) + store.putSingleAndReleaseLock(rdd(0, 0), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) + store.putSingleAndReleaseLock(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") - store.releaseLock(rdd(1, 0)) + assert(store.getSingleAndReleaseLock(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](4000), StorageLevel.MEMORY_ONLY) + store.putSingleAndReleaseLock(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") @@ -1134,8 +1125,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE memoryStore.releasePendingUnrollMemoryForThisTask() // 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) + store.putIteratorAndReleaseLock("someBlock1", smallList.iterator, StorageLevel.MEMORY_ONLY) + store.putIteratorAndReleaseLock("someBlock2", smallList.iterator, StorageLevel.MEMORY_ONLY) unrollResult = memoryStore.unrollSafely("unroll", smallList.iterator) verifyUnroll(smallList.iterator, unrollResult, shouldBeArray = true) assert(memoryStore.currentUnrollMemoryForThisTask === 0) @@ -1146,7 +1137,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // 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) + store.putIteratorAndReleaseLock("someBlock3", smallList.iterator, StorageLevel.MEMORY_ONLY) unrollResult = memoryStore.unrollSafely("unroll", bigList.iterator) verifyUnroll(bigList.iterator, unrollResult, shouldBeArray = false) assert(memoryStore.currentUnrollMemoryForThisTask > 0) // we returned an iterator @@ -1178,8 +1169,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // 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) + store.putIteratorAndReleaseLock("b1", smallIterator, memOnly) + store.putIteratorAndReleaseLock("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) @@ -1190,7 +1181,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(memoryStore.contains("b3")) assert(memoryStore.currentUnrollMemoryForThisTask === 0) memoryStore.remove("b3") - store.putIterator("b3", smallIterator, memOnly) + store.putIteratorAndReleaseLock("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) @@ -1217,8 +1208,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE def bigIterator: Iterator[Any] = bigList.iterator.asInstanceOf[Iterator[Any]] assert(memoryStore.currentUnrollMemoryForThisTask === 0) - store.putIterator("b1", smallIterator, memAndDisk) - store.putIterator("b2", smallIterator, memAndDisk) + store.putIteratorAndReleaseLock("b1", smallIterator, memAndDisk) + store.putIteratorAndReleaseLock("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 @@ -1231,7 +1222,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(!diskStore.contains("b2")) assert(!diskStore.contains("b3")) memoryStore.remove("b3") - store.putIterator("b3", smallIterator, StorageLevel.MEMORY_ONLY) + store.putIteratorAndReleaseLock("b3", smallIterator, StorageLevel.MEMORY_ONLY) assert(memoryStore.currentUnrollMemoryForThisTask === 0) // Unroll huge block with not enough space. This should fail and drop the new block to disk @@ -1312,28 +1303,86 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(result.data === Right(bytes)) } - test("blocks with non-zero pin counts cannot be evicted from the MemoryStore") { + test("read-locked blocks cannot be evicted from the MemoryStore") { store = makeBlockManager(12000) val arr = new Array[Byte](4000) // First store a1 and a2, both in memory, and a3, on disk only - store.putSingle("a1", arr, StorageLevel.MEMORY_ONLY_SER) - store.putSingle("a2", arr, StorageLevel.MEMORY_ONLY_SER) + store.putSingleAndReleaseLock("a1", arr, StorageLevel.MEMORY_ONLY_SER) + store.putSingleAndReleaseLock("a2", arr, StorageLevel.MEMORY_ONLY_SER) assert(store.getSingle("a1").isDefined, "a1 was not in store") assert(store.getSingle("a2").isDefined, "a2 was not in store") - // This put should fail because both a1 and a2 have non-zero pin counts: - store.putSingle("a3", arr, StorageLevel.MEMORY_ONLY_SER) + // This put should fail because both a1 and a2 should be read-locked: + store.putSingleAndReleaseLock("a3", arr, StorageLevel.MEMORY_ONLY_SER) assert(store.getSingle("a3").isEmpty, "a3 was in store") assert(store.getSingle("a1").isDefined, "a1 was not in store") assert(store.getSingle("a2").isDefined, "a2 was not in store") // Release both pins of block a2: store.releaseLock("a2") store.releaseLock("a2") - // TODO(josh): assert(store.getPinCount("a2") === 0) // Block a1 is the least-recently accessed, so an LRU eviction policy would evict it before // block a2. However, a1 is still pinned so this put of a3 should evict a2 instead: - store.putSingle("a3", arr, StorageLevel.MEMORY_ONLY_SER) + store.putSingleAndReleaseLock("a3", arr, StorageLevel.MEMORY_ONLY_SER) assert(store.getSingle("a2").isEmpty, "a2 was in store") assert(store.getSingle("a1").isDefined, "a1 was not in store") assert(store.getSingle("a3").isDefined, "a3 was not in store") } } + +private object BlockManagerSuite { + + private implicit class BlockManagerTestUtils(store: BlockManager) { + + def putSingleAndReleaseLock( + block: BlockId, + value: Any, + storageLevel: StorageLevel, + tellMaster: Boolean): Unit = { + if (store.putSingle(block, value, storageLevel, tellMaster)) { + store.releaseLock(block) + } + } + + def putSingleAndReleaseLock(block: BlockId, value: Any, storageLevel: StorageLevel): Unit = { + if (store.putSingle(block, value, storageLevel)) { + store.releaseLock(block) + } + } + + def putIteratorAndReleaseLock( + blockId: BlockId, + values: Iterator[Any], + level: StorageLevel): Unit = { + if (store.putIterator(blockId, values, level)) { + store.releaseLock(blockId) + } + } + + def putIteratorAndReleaseLock( + blockId: BlockId, + values: Iterator[Any], + level: StorageLevel, + tellMaster: Boolean): Unit = { + if (store.putIterator(blockId, values, level, tellMaster)) { + store.releaseLock(blockId) + } + } + + private def wrapGet[T](f: BlockId => Option[T]): BlockId => Option[T] = (blockId: BlockId) => { + val result = f(blockId) + if (result.isDefined) { + store.releaseLock(blockId) + } + result + } + + def hasLocalBlock(blockId: BlockId): Boolean = { + getLocalAndReleaseLock(blockId).isDefined + } + + val getLocalAndReleaseLock: (BlockId) => Option[BlockResult] = wrapGet(store.getLocal) + val getAndReleaseLock: (BlockId) => Option[BlockResult] = wrapGet(store.get) + val getSingleAndReleaseLock: (BlockId) => Option[Any] = wrapGet(store.getSingle) + val getLocalBytesAndReleaseLock: (BlockId) => Option[ByteBuffer] = wrapGet(store.getLocalBytes) + } + +} \ No newline at end of file From a0c5bb336c0dc06ec9ffdf2ff12cb4f7aae3bc1d Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 11 Feb 2016 14:10:54 -0800 Subject: [PATCH 24/81] Fix scalastyle. --- .../scala/org/apache/spark/storage/BlockInfoManagerSuite.scala | 2 +- .../test/scala/org/apache/spark/storage/BlockManagerSuite.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala index 5c5fc6f937a3d..0ddfb13955619 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala @@ -223,4 +223,4 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { blockInfoManager.releaseAllLocksForTask(0) assert(blockInfoManager.getNumberOfMapEntries === 1) } -} \ No newline at end of file +} 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 6c40541101169..80f7ac8ed1f43 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -1385,4 +1385,4 @@ private object BlockManagerSuite { val getLocalBytesAndReleaseLock: (BlockId) => Option[ByteBuffer] = wrapGet(store.getLocalBytes) } -} \ No newline at end of file +} From d40e010cd40ab62d7961c4ee2484c1b2588d694c Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 11 Feb 2016 14:58:59 -0800 Subject: [PATCH 25/81] Add a bunch of comments. --- .../spark/storage/BlockInfoManager.scala | 63 +++++++++++++++++-- 1 file changed, 59 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index 04a8c61aa4ec0..610b36f51d74e 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -29,28 +29,78 @@ import com.google.common.collect.ConcurrentHashMultiset import org.apache.spark.{Logging, TaskContext} -private[storage] class BlockInfo( - val level: StorageLevel, - val tellMaster: Boolean) { +/** + * Tracks metadata for an individual block. + * + * @param level the block's storage level. This is the requested persistence level, not the + * effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this + * does not imply that the block is actually resident in memory). + * @param tellMaster whether state changes for this block should be reported to the master. This + * is true for most blocks, but is false for broadcast blocks. + */ +private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) { + + /** + * The size of the block (in bytes) + */ var size: Long = 0 + + /** + * The number of times that this block has been locked for reading. + */ var readerCount: Int = 0 + + /** + * The task attempt id of the task which currently holds the write lock for this block, or -1 + * if this block is not locked for writing. + */ var writerTask: Long = -1 + + // Invariants: + // (writerTask != -1) implies (readerCount == 0) + // (readerCount != 0) implies (writerTask == -1) + + /** + * True if this block has been removed from the BlockManager and false otherwise. + * This field is used to communicate block deletion to blocked readers / writers (see its usage + * in [[BlockInfoManager]]). + */ var removed: Boolean = false } - +/** + * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking. + * + * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is + * automatically associated with a running task and locks are automatically released upon task + * completion or failure. + * + * This class is thread-safe. + */ private[storage] class BlockInfoManager extends Logging { private type TaskAttemptId = Long + /** + * Used to look up metadata for individual blocks. Entries are added to this map via an atomic + * set-if-not-exists operation ([[putAndLockForWritingIfAbsent()]]) and are removed + * by [[remove()]]. + */ @GuardedBy("this") private[this] val infos = new mutable.HashMap[BlockId, BlockInfo] + /** + * Tracks the set of blocks that each task has locked for writing. + */ @GuardedBy("this") private[this] val writeLocksByTask = new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]] with mutable.MultiMap[TaskAttemptId, BlockId] + /** + * Tracks the set of blocks that each task has locked for reading, along with the number of times + * that a block has been locked (since our read locks are re-entrant). This is thread-safe. + */ private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = { // We need to explicitly box as java.lang.Long to avoid a type mismatch error: val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] { @@ -61,6 +111,11 @@ private[storage] class BlockInfoManager extends Logging { // ---------------------------------------------------------------------------------------------- + /** + * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024 + * if called outside of a task (-1024 was chosen because it's different than the -1 which is used + * in [[BlockInfo.writerTask]] to denote the absence of a write lock). + */ private def currentTaskAttemptId: TaskAttemptId = { Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L) } From 3f295952d0df806691f2d1035ba4e72566697ca6 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 11 Feb 2016 15:16:48 -0800 Subject: [PATCH 26/81] Add even more comments. --- .../spark/storage/BlockInfoManager.scala | 68 ++++++++++++++++--- 1 file changed, 58 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index 610b36f51d74e..f4fce4970d251 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -68,6 +68,7 @@ private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolea var removed: Boolean = false } + /** * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking. * @@ -121,10 +122,16 @@ private[storage] class BlockInfoManager extends Logging { } /** - * Todo: document blocking / locking semantics. + * Lock a block for reading and return its metadata. + * + * A single task can lock a block multiple times for reading, in which case each lock will need + * to be released separately. * - * @param blockId - * @return + * @param blockId the block to lock. + * @param blocking if true (default), this call will block until the lock is acquired. If false, + * this call will return immediately if the lock acquisition fails. + * @return None if the block did not exist or was removed (in which case no lock is held), or + * Some(BlockInfo) (in which case the block is locked for reading). */ def getAndLockForReading( blockId: BlockId, @@ -146,6 +153,19 @@ private[storage] class BlockInfoManager extends Logging { } } + /** + * Lock a block for writing and return its metadata. + * + * If this is called by a task which already holds the block's exclusive write lock, then this + * will return success but will not further increment any lock counts (so both write-lock + * acquisitions will be freed by the same [[releaseLock()]] or [[downgradeLock()]] call. + * + * @param blockId the block to lock. + * @param blocking if true (default), this call will block until the lock is acquired. If false, + * this call will return immediately if the lock acquisition fails. + * @return None if the block did not exist or was removed (in which case no lock is held), or + * Some(BlockInfo) (in which case the block is locked for writing). + */ def getAndLockForWriting( blockId: BlockId, blocking: Boolean = true): Option[BlockInfo] = synchronized { @@ -167,10 +187,16 @@ private[storage] class BlockInfoManager extends Logging { } } + /** + * Get a block's metadata without acquiring any locks. + */ def get(blockId: BlockId): Option[BlockInfo] = synchronized { infos.get(blockId) } + /** + * Downgrades an exclusive write lock to a shared read lock. + */ def downgradeLock(blockId: BlockId): Unit = synchronized { logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId") // TODO: refactor this code so that log messages aren't confusing. @@ -182,6 +208,9 @@ private[storage] class BlockInfoManager extends Logging { notifyAll() } + /** + * Release a lock on the given block. + */ def releaseLock(blockId: BlockId): Unit = synchronized { logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId") val info = get(blockId).getOrElse { @@ -201,6 +230,13 @@ private[storage] class BlockInfoManager extends Logging { notifyAll() } + /** + * Atomically create metadata for a non-existent block. + * + * @param blockId the block id. + * @param newBlockInfo the block info for the new block. + * @return true if the block did not already exist, false otherwise. + */ def putAndLockForWritingIfAbsent( blockId: BlockId, newBlockInfo: BlockInfo): Boolean = synchronized { @@ -253,7 +289,9 @@ private[storage] class BlockInfoManager extends Logging { totalPinCountForTask } - + /** + * Returns the number of blocks tracked. + */ def size: Int = synchronized { infos.size } @@ -262,7 +300,7 @@ private[storage] class BlockInfoManager extends Logging { * Return the number of map entries in this pin counter's internal data structures. * This is used in unit tests in order to detect memory leaks. */ - private[storage] def getNumberOfMapEntries: Long = synchronized { + private[storage] def getNumberOfMapEntries: Long = synchronized { size + readLocksByTask.size() + readLocksByTask.asMap().asScala.map(_._2.size()).sum + @@ -270,7 +308,18 @@ private[storage] class BlockInfoManager extends Logging { writeLocksByTask.map(_._2.size).sum } - // This implicitly drops all locks. + /** + * Returns an iterator over a snapshot of all blocks' metadata. Note that the individual entries + * is this iterator are mutable and thus may reflect blocks that are deleted while the iterator + * is being traversed. + */ + def entries: Iterator[(BlockId, BlockInfo)] = synchronized { + infos.iterator.toArray.toIterator + } + + /** + * Removes the given block and automatically drops all locks on it. + */ def remove(blockId: BlockId): Unit = synchronized { logTrace(s"Task $currentTaskAttemptId trying to remove block $blockId") // TODO: Should probably have safety checks here @@ -280,14 +329,13 @@ private[storage] class BlockInfoManager extends Logging { notifyAll() } + /** + * Delete all state. Called during shutdown. + */ def clear(): Unit = synchronized { infos.clear() readLocksByTask.invalidateAll() writeLocksByTask.clear() } - def entries: Iterator[(BlockId, BlockInfo)] = synchronized { - infos.iterator.toArray.toIterator - } - } From ef7d88508af04b81d6671fd7ccf55111ca3e7856 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 11 Feb 2016 15:27:32 -0800 Subject: [PATCH 27/81] Update to reflect new semantics for get() of removed block. --- .../apache/spark/storage/BlockInfoManager.scala | 16 ++++------------ 1 file changed, 4 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index f4fce4970d251..05845e708387a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -142,12 +142,8 @@ private[storage] class BlockInfoManager extends Logging { if (info.removed) return None if (blocking) wait() else return None } - // TODO: try to remember why you need actualInfo / the extra get() here. - val actualInfo = infos.get(blockId) - actualInfo.foreach { i => - i.readerCount += 1 - readLocksByTask(currentTaskAttemptId).add(blockId) - } + info.readerCount += 1 + readLocksByTask(currentTaskAttemptId).add(blockId) logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId") info } @@ -177,11 +173,8 @@ private[storage] class BlockInfoManager extends Logging { if (blocking) wait() else return None } } - val actualInfo = infos.get(blockId) - actualInfo.foreach { i => - i.writerTask = currentTaskAttemptId - writeLocksByTask.addBinding(currentTaskAttemptId, blockId) - } + info.writerTask = currentTaskAttemptId + writeLocksByTask.addBinding(currentTaskAttemptId, blockId) logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId") info } @@ -199,7 +192,6 @@ private[storage] class BlockInfoManager extends Logging { */ def downgradeLock(blockId: BlockId): Unit = synchronized { logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId") - // TODO: refactor this code so that log messages aren't confusing. val info = get(blockId).get require(info.writerTask == currentTaskAttemptId, s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold") From e8d6ec87d37d9c262f3fe29bd7a719d350d4610d Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 12 Feb 2016 12:30:22 -0800 Subject: [PATCH 28/81] Fixes to torrent broadcast block removal. A subset of these changes will be spun off into a separate smaller patch. --- .../spark/broadcast/TorrentBroadcast.scala | 61 +++++++++++-------- .../spark/network/BlockDataManager.scala | 5 ++ .../spark/storage/BlockInfoManager.scala | 18 +++--- .../apache/spark/storage/BlockManager.scala | 9 ++- .../storage/BlockManagerManagedBuffer.scala | 43 +++++++++++++ .../network/buffer/NioManagedBuffer.java | 2 +- .../network/protocol/AbstractMessage.java | 1 + .../network/protocol/MessageEncoder.java | 3 +- .../network/protocol/MessageWithHeader.java | 7 ++- .../protocol/MessageWithHeaderSuite.java | 4 +- 10 files changed, 115 insertions(+), 38 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/storage/BlockManagerManagedBuffer.scala diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index a2872207bde9e..d5083fa15022b 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -90,22 +90,22 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) /** * Divide the object into multiple blocks and put those blocks in the block manager. + * * @param value the object to divide * @return number of blocks this broadcast variable is divided into */ private def writeBlocks(value: T): Int = { // Store a copy of the broadcast variable in the driver so that tasks run on the driver // do not create a duplicate copy of the broadcast variable's value. - SparkEnv.get.blockManager.putSingle(broadcastId, value, StorageLevel.MEMORY_AND_DISK, - tellMaster = false) + val blockManager = SparkEnv.get.blockManager + blockManager.putSingle(broadcastId, value, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + blockManager.releaseLock(broadcastId) val blocks = TorrentBroadcast.blockifyObject(value, blockSize, SparkEnv.get.serializer, compressionCodec) blocks.zipWithIndex.foreach { case (block, i) => - SparkEnv.get.blockManager.putBytes( - BroadcastBlockId(id, "piece" + i), - block, - StorageLevel.MEMORY_AND_DISK_SER, - tellMaster = true) + val pieceId = BroadcastBlockId(id, "piece" + i) + blockManager.putBytes(pieceId, block, StorageLevel.MEMORY_AND_DISK_SER, tellMaster = true) + blockManager.releaseLock(pieceId) } blocks.length } @@ -127,20 +127,22 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) def getRemote: Option[ByteBuffer] = bm.getRemoteBytes(pieceId).map { block => // If we found the block from remote executors/driver's BlockManager, put the block // in this executor's BlockManager. - SparkEnv.get.blockManager.putBytes( - pieceId, - block, - StorageLevel.MEMORY_AND_DISK_SER, - tellMaster = true) + bm.putBytes(pieceId, block, StorageLevel.MEMORY_AND_DISK_SER, tellMaster = true) block } val block: ByteBuffer = getLocal.orElse(getRemote).getOrElse( throw new SparkException(s"Failed to get $pieceId of $broadcastId")) blocks(pid) = block - // On the driver broadcast variables may be loaded when computing rdd.partitions(), which - // takes place outside of the context of a task, so we need to use an option here: - Option(TaskContext.get()).foreach { taskContext => - taskContext.addTaskCompletionListener(_ => SparkEnv.get.blockManager.releaseLock(pieceId)) + Option(TaskContext.get()) match { + case Some(taskContext) => + taskContext.addTaskCompletionListener(_ => bm.releaseLock(pieceId)) + case None => + // This should only happen on the driver, where broadcast variables may be accessed + // outside of running tasks (e.g. when computing rdd.partitions()). In order to allow + // broadcast variables to be garbage collected we need to free the reference here, which + // is slightly unsafe but is technically okay because broadcast variables aren't stored + // off-heap. + bm.releaseLock(pieceId) } } blocks @@ -170,13 +172,19 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) private def readBroadcastBlock(): T = Utils.tryOrIOException { TorrentBroadcast.synchronized { setConf(SparkEnv.get.conf) - SparkEnv.get.blockManager.getLocal(broadcastId).map(_.data.next()) match { + val blockManager = SparkEnv.get.blockManager + blockManager.getLocal(broadcastId).map(_.data.next()) match { case Some(x) => - // On the driver broadcast variables may be loaded when computing rdd.partitions(), which - // takes place outside of the context of a task, so we need to use an option here: - Option(TaskContext.get()).foreach { taskContext => - taskContext.addTaskCompletionListener(_ => - SparkEnv.get.blockManager.releaseLock(broadcastId)) + Option(TaskContext.get()) match { + case Some(taskContext) => + taskContext.addTaskCompletionListener(_ => blockManager.releaseLock(broadcastId)) + case None => + // This should only happen on the driver, where broadcast variables may be accessed + // outside of running tasks (e.g. when computing rdd.partitions()). In order to allow + // broadcast variables to be garbage collected we need to free the reference here + // which is slightly unsafe but is technically okay because broadcast variables aren't + // stored off-heap. + blockManager.releaseLock(broadcastId) } x.asInstanceOf[T] @@ -190,8 +198,13 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) blocks, SparkEnv.get.serializer, compressionCodec) // Store the merged copy in BlockManager so other tasks on this executor don't // need to re-fetch it. - SparkEnv.get.blockManager.putSingle( - broadcastId, obj, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + blockManager.putSingle(broadcastId, obj, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + Option(TaskContext.get()) match { + case Some(taskContext) => + taskContext.addTaskCompletionListener(_ => blockManager.releaseLock(broadcastId)) + case None => + blockManager.releaseLock(broadcastId) + } obj } } diff --git a/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala b/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala index 1745d52c81923..e05cfd376a42f 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala @@ -33,4 +33,9 @@ trait BlockDataManager { * Put the block locally, using the given storage level. */ def putBlockData(blockId: BlockId, data: ManagedBuffer, level: StorageLevel): Unit + + /** + * Release locks acquired by [[putBlockData()]] and [[getBlockData()]]. + */ + def releaseLock(blockId: BlockId): Unit } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index 05845e708387a..2f81bfb077772 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -118,6 +118,8 @@ private[storage] class BlockInfoManager extends Logging { * in [[BlockInfo.writerTask]] to denote the absence of a write lock). */ private def currentTaskAttemptId: TaskAttemptId = { + // TODO(josh): assert that this only happens on the driver? + // What about block transfer / getRemote()? Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L) } @@ -136,7 +138,7 @@ private[storage] class BlockInfoManager extends Logging { def getAndLockForReading( blockId: BlockId, blocking: Boolean = true): Option[BlockInfo] = synchronized { - logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId") + println(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId") infos.get(blockId).map { info => while (info.writerTask != -1) { if (info.removed) return None @@ -144,7 +146,7 @@ private[storage] class BlockInfoManager extends Logging { } info.readerCount += 1 readLocksByTask(currentTaskAttemptId).add(blockId) - logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId") + println(s"Task $currentTaskAttemptId acquired read lock for $blockId") info } } @@ -165,7 +167,7 @@ private[storage] class BlockInfoManager extends Logging { def getAndLockForWriting( blockId: BlockId, blocking: Boolean = true): Option[BlockInfo] = synchronized { - logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId") + println(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId") infos.get(blockId).map { info => if (info.writerTask != currentTaskAttemptId) { while (info.writerTask != -1 || info.readerCount != 0) { @@ -175,7 +177,7 @@ private[storage] class BlockInfoManager extends Logging { } info.writerTask = currentTaskAttemptId writeLocksByTask.addBinding(currentTaskAttemptId, blockId) - logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId") + println(s"Task $currentTaskAttemptId acquired write lock for $blockId") info } } @@ -191,7 +193,7 @@ private[storage] class BlockInfoManager extends Logging { * Downgrades an exclusive write lock to a shared read lock. */ def downgradeLock(blockId: BlockId): Unit = synchronized { - logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId") + println(s"Task $currentTaskAttemptId downgrading write lock for $blockId") val info = get(blockId).get require(info.writerTask == currentTaskAttemptId, s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold") @@ -204,7 +206,7 @@ private[storage] class BlockInfoManager extends Logging { * Release a lock on the given block. */ def releaseLock(blockId: BlockId): Unit = synchronized { - logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId") + println(s"Task $currentTaskAttemptId releasing lock for $blockId") val info = get(blockId).getOrElse { throw new IllegalStateException(s"Block $blockId not found") } @@ -232,7 +234,7 @@ private[storage] class BlockInfoManager extends Logging { def putAndLockForWritingIfAbsent( blockId: BlockId, newBlockInfo: BlockInfo): Boolean = synchronized { - logTrace(s"Task $currentTaskAttemptId trying to put $blockId") + println(s"Task $currentTaskAttemptId trying to put $blockId") val actualInfo = infos.getOrElseUpdate(blockId, newBlockInfo) if (actualInfo eq newBlockInfo) { actualInfo.writerTask = currentTaskAttemptId @@ -313,7 +315,7 @@ private[storage] class BlockInfoManager extends Logging { * Removes the given block and automatically drops all locks on it. */ def remove(blockId: BlockId): Unit = synchronized { - logTrace(s"Task $currentTaskAttemptId trying to remove block $blockId") + println(s"Task $currentTaskAttemptId trying to remove block $blockId") // TODO: Should probably have safety checks here infos.remove(blockId).foreach { info => info.removed = true 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 4efd5e4ce7399..e47ab89ac08c8 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -284,7 +284,7 @@ private[spark] class BlockManager( .asInstanceOf[Option[ByteBuffer]] if (blockBytesOpt.isDefined) { val buffer = blockBytesOpt.get - new NioManagedBuffer(buffer) + new BlockManagerManagedBuffer(this, blockId, buffer) } else { throw new BlockNotFoundException(blockId.toString) } @@ -595,6 +595,13 @@ private[spark] class BlockManager( None } + /** + * Downgrades an exclusive write lock to a shared read lock. + */ + def downgradeLock(blockId: BlockId): Unit = { + blockInfoManager.downgradeLock(blockId) + } + /** * Release a lock on the given block. */ diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerManagedBuffer.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerManagedBuffer.scala new file mode 100644 index 0000000000000..24f38d84da0f4 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerManagedBuffer.scala @@ -0,0 +1,43 @@ +/* + * 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.storage + +import java.nio.ByteBuffer + +import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} + + +private[storage] class BlockManagerManagedBuffer( + blockManager: BlockManager, + blockId: BlockId, + buf: ByteBuffer) extends NioManagedBuffer(buf) { + + override def retain(): ManagedBuffer = { + println("RETAIN") + super.retain() + val locked = blockManager.blockInfoManager.getAndLockForReading(blockId, blocking = false) + assert(locked.isDefined) + this + } + + override def release(): ManagedBuffer = { + println("RELEASE") + blockManager.releaseLock(blockId) + super.release() + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java index f55b884bc45ce..631d767715256 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java @@ -28,7 +28,7 @@ /** * A {@link ManagedBuffer} backed by {@link ByteBuffer}. */ -public final class NioManagedBuffer extends ManagedBuffer { +public class NioManagedBuffer extends ManagedBuffer { private final ByteBuffer buf; public NioManagedBuffer(ByteBuffer buf) { diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/AbstractMessage.java b/network/common/src/main/java/org/apache/spark/network/protocol/AbstractMessage.java index 2924218c2f08b..3f098ce495016 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/AbstractMessage.java +++ b/network/common/src/main/java/org/apache/spark/network/protocol/AbstractMessage.java @@ -19,6 +19,7 @@ import com.google.common.base.Objects; +import io.netty.util.AbstractReferenceCounted; import org.apache.spark.network.buffer.ManagedBuffer; /** diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java b/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java index abca22347b783..1343be1f9e76d 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java +++ b/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java @@ -54,6 +54,7 @@ public void encode(ChannelHandlerContext ctx, Message in, List out) thro body = in.body().convertToNetty(); isBodyInFrame = in.isBodyInFrame(); } catch (Exception e) { + in.body().release(); if (in instanceof AbstractResponseMessage) { AbstractResponseMessage resp = (AbstractResponseMessage) in; // Re-encode this message as a failure response. @@ -81,7 +82,7 @@ public void encode(ChannelHandlerContext ctx, Message in, List out) thro assert header.writableBytes() == 0; if (body != null && bodyLength > 0) { - out.add(new MessageWithHeader(header, body, bodyLength)); + out.add(new MessageWithHeader(header, in.body(), body, bodyLength)); } else { out.add(header); } diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java b/network/common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java index d686a951467cf..3d1f3126fb222 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java +++ b/network/common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java @@ -26,6 +26,8 @@ import io.netty.util.AbstractReferenceCounted; import io.netty.util.ReferenceCountUtil; +import org.apache.spark.network.buffer.ManagedBuffer; + /** * A wrapper message that holds two separate pieces (a header and a body). * @@ -34,16 +36,18 @@ class MessageWithHeader extends AbstractReferenceCounted implements FileRegion { private final ByteBuf header; + private final ManagedBuffer managedBuffer; private final int headerLength; private final Object body; private final long bodyLength; private long totalBytesTransferred; - MessageWithHeader(ByteBuf header, Object body, long bodyLength) { + MessageWithHeader(ByteBuf header, ManagedBuffer managedBuffer, Object body, long bodyLength) { Preconditions.checkArgument(body instanceof ByteBuf || body instanceof FileRegion, "Body must be a ByteBuf or a FileRegion."); this.header = header; this.headerLength = header.readableBytes(); + this.managedBuffer = managedBuffer; this.body = body; this.bodyLength = bodyLength; } @@ -99,6 +103,7 @@ public long transferTo(final WritableByteChannel target, final long position) th protected void deallocate() { header.release(); ReferenceCountUtil.release(body); + managedBuffer.release(); } private int copyByteBuf(ByteBuf buf, WritableByteChannel target) throws IOException { diff --git a/network/common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java b/network/common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java index 6c98e733b462f..d400e1186ef75 100644 --- a/network/common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java +++ b/network/common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java @@ -47,7 +47,7 @@ public void testShortWrite() throws Exception { public void testByteBufBody() throws Exception { ByteBuf header = Unpooled.copyLong(42); ByteBuf body = Unpooled.copyLong(84); - MessageWithHeader msg = new MessageWithHeader(header, body, body.readableBytes()); + MessageWithHeader msg = new MessageWithHeader(header, null, body, body.readableBytes()); ByteBuf result = doWrite(msg, 1); assertEquals(msg.count(), result.readableBytes()); @@ -59,7 +59,7 @@ private void testFileRegionBody(int totalWrites, int writesPerCall) throws Excep ByteBuf header = Unpooled.copyLong(42); int headerLength = header.readableBytes(); TestFileRegion region = new TestFileRegion(totalWrites, writesPerCall); - MessageWithHeader msg = new MessageWithHeader(header, region, region.count()); + MessageWithHeader msg = new MessageWithHeader(header, null, region, region.count()); ByteBuf result = doWrite(msg, totalWrites / writesPerCall); assertEquals(headerLength + region.count(), result.readableBytes()); From 9c8d5308ae4482c86a381e9a863fa91d3ed63899 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 12 Feb 2016 12:31:11 -0800 Subject: [PATCH 29/81] Roll back logging change. --- .../apache/spark/storage/BlockInfoManager.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index 2f81bfb077772..f1eca9140ebf3 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -138,7 +138,7 @@ private[storage] class BlockInfoManager extends Logging { def getAndLockForReading( blockId: BlockId, blocking: Boolean = true): Option[BlockInfo] = synchronized { - println(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId") + logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId") infos.get(blockId).map { info => while (info.writerTask != -1) { if (info.removed) return None @@ -146,7 +146,7 @@ private[storage] class BlockInfoManager extends Logging { } info.readerCount += 1 readLocksByTask(currentTaskAttemptId).add(blockId) - println(s"Task $currentTaskAttemptId acquired read lock for $blockId") + logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId") info } } @@ -167,7 +167,7 @@ private[storage] class BlockInfoManager extends Logging { def getAndLockForWriting( blockId: BlockId, blocking: Boolean = true): Option[BlockInfo] = synchronized { - println(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId") + logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId") infos.get(blockId).map { info => if (info.writerTask != currentTaskAttemptId) { while (info.writerTask != -1 || info.readerCount != 0) { @@ -177,7 +177,7 @@ private[storage] class BlockInfoManager extends Logging { } info.writerTask = currentTaskAttemptId writeLocksByTask.addBinding(currentTaskAttemptId, blockId) - println(s"Task $currentTaskAttemptId acquired write lock for $blockId") + logTrace(s"Task $currentTaskAttemptId acquired write lock for $blockId") info } } @@ -193,7 +193,7 @@ private[storage] class BlockInfoManager extends Logging { * Downgrades an exclusive write lock to a shared read lock. */ def downgradeLock(blockId: BlockId): Unit = synchronized { - println(s"Task $currentTaskAttemptId downgrading write lock for $blockId") + logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId") val info = get(blockId).get require(info.writerTask == currentTaskAttemptId, s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold") @@ -206,7 +206,7 @@ private[storage] class BlockInfoManager extends Logging { * Release a lock on the given block. */ def releaseLock(blockId: BlockId): Unit = synchronized { - println(s"Task $currentTaskAttemptId releasing lock for $blockId") + logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId") val info = get(blockId).getOrElse { throw new IllegalStateException(s"Block $blockId not found") } @@ -234,7 +234,7 @@ private[storage] class BlockInfoManager extends Logging { def putAndLockForWritingIfAbsent( blockId: BlockId, newBlockInfo: BlockInfo): Boolean = synchronized { - println(s"Task $currentTaskAttemptId trying to put $blockId") + logTrace(s"Task $currentTaskAttemptId trying to put $blockId") val actualInfo = infos.getOrElseUpdate(blockId, newBlockInfo) if (actualInfo eq newBlockInfo) { actualInfo.writerTask = currentTaskAttemptId @@ -315,7 +315,7 @@ private[storage] class BlockInfoManager extends Logging { * Removes the given block and automatically drops all locks on it. */ def remove(blockId: BlockId): Unit = synchronized { - println(s"Task $currentTaskAttemptId trying to remove block $blockId") + logTrace(s"Task $currentTaskAttemptId trying to remove block $blockId") // TODO: Should probably have safety checks here infos.remove(blockId).foreach { info => info.removed = true From f3fc2987585352d9e5a2dd3083ed65c1f5f30507 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 12 Feb 2016 13:02:09 -0800 Subject: [PATCH 30/81] Remove more printlns. --- .../org/apache/spark/storage/BlockManagerManagedBuffer.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerManagedBuffer.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerManagedBuffer.scala index 24f38d84da0f4..1ab44316a8210 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerManagedBuffer.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerManagedBuffer.scala @@ -28,7 +28,6 @@ private[storage] class BlockManagerManagedBuffer( buf: ByteBuffer) extends NioManagedBuffer(buf) { override def retain(): ManagedBuffer = { - println("RETAIN") super.retain() val locked = blockManager.blockInfoManager.getAndLockForReading(blockId, blocking = false) assert(locked.isDefined) @@ -36,7 +35,6 @@ private[storage] class BlockManagerManagedBuffer( } override def release(): ManagedBuffer = { - println("RELEASE") blockManager.releaseLock(blockId) super.release() } From dd6358c7f32145327b2ac2b19416894df30123bb Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 12 Feb 2016 13:55:24 -0800 Subject: [PATCH 31/81] Add todos. --- .../org/apache/spark/storage/BlockInfoManager.scala | 12 ++++++++++-- .../org/apache/spark/storage/BlockManager.scala | 2 ++ .../scala/org/apache/spark/storage/MemoryStore.scala | 1 + 3 files changed, 13 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index f1eca9140ebf3..b0b5e067f275c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -59,6 +59,7 @@ private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolea // Invariants: // (writerTask != -1) implies (readerCount == 0) // (readerCount != 0) implies (writerTask == -1) + // TODO: add assertions around every method /** * True if this block has been removed from the BlockManager and false otherwise. @@ -66,8 +67,11 @@ private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolea * in [[BlockInfoManager]]). */ var removed: Boolean = false -} + // TODO: Add timestamps on lock acquisitions +} +// In debugging mode, check that locks haven't been held for too long. +// Every few minutes, dump debug info. /** * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking. @@ -227,6 +231,8 @@ private[storage] class BlockInfoManager extends Logging { /** * Atomically create metadata for a non-existent block. * + * TODO: clarify locking semantics when this returns false. + * * @param blockId the block id. * @param newBlockInfo the block info for the new block. * @return true if the block did not already exist, false otherwise. @@ -246,7 +252,7 @@ private[storage] class BlockInfoManager extends Logging { } /** - * Release all pins held by the given task, clearing that task's pin bookkeeping + * Release all lock held by the given task, clearing that task's pin bookkeeping * structures and updating the global pin counts. This method should be called at the * end of a task (either by a task completion handler or in `TaskRunner.run()`). * @@ -313,6 +319,8 @@ private[storage] class BlockInfoManager extends Logging { /** * Removes the given block and automatically drops all locks on it. + * + * TODO: document validity conditions. */ def remove(blockId: BlockId): Unit = synchronized { logTrace(s"Task $currentTaskAttemptId trying to remove block $blockId") 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 e47ab89ac08c8..4305ecc909cc0 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -719,6 +719,8 @@ private[spark] class BlockManager( } } + // TODO: release write lock and split up this code. + val startTimeMs = System.currentTimeMillis /* If we're storing values and we need to replicate the data, we'll want access to the values, 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 c7d0b71fed230..40a19d4e1a5a2 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -471,6 +471,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo logInfo(s"Will not store $id as it would require dropping another block " + "from the same RDD") } + // TODO: free write locks here. 0L } } From ec8cc24ddd2b35a074674b06503ca710a17c1af2 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 12 Feb 2016 16:09:44 -0800 Subject: [PATCH 32/81] Add missing ManagedBuffer.release() call. --- .../network/protocol/MessageEncoder.java | 12 ++++++-- .../network/protocol/MessageWithHeader.java | 28 ++++++++++++++++++- .../protocol/MessageWithHeaderSuite.java | 7 +++-- 3 files changed, 41 insertions(+), 6 deletions(-) diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java b/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java index abca22347b783..442b60bdb9b5e 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java +++ b/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java @@ -49,11 +49,15 @@ public void encode(ChannelHandlerContext ctx, Message in, List out) thro // If the message has a body, take it out to enable zero-copy transfer for the payload. if (in.body() != null) { + bodyLength = in.body().size(); + if (bodyLength > 0) { + in.body().retain(); + } try { - bodyLength = in.body().size(); body = in.body().convertToNetty(); isBodyInFrame = in.isBodyInFrame(); } catch (Exception e) { + in.body().release(); if (in instanceof AbstractResponseMessage) { AbstractResponseMessage resp = (AbstractResponseMessage) in; // Re-encode this message as a failure response. @@ -80,8 +84,10 @@ public void encode(ChannelHandlerContext ctx, Message in, List out) thro in.encode(header); assert header.writableBytes() == 0; - if (body != null && bodyLength > 0) { - out.add(new MessageWithHeader(header, body, bodyLength)); + if (body != null) { + // We transfer ownership of the reference on in.body() to MessageWithHeader. + // This reference will be freed when MessageWithHeader.deallocate() is called. + out.add(new MessageWithHeader(in.body(), header, body, bodyLength)); } else { out.add(header); } diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java b/network/common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java index d686a951467cf..66227f96a1a21 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java +++ b/network/common/src/main/java/org/apache/spark/network/protocol/MessageWithHeader.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.nio.channels.WritableByteChannel; +import javax.annotation.Nullable; import com.google.common.base.Preconditions; import io.netty.buffer.ByteBuf; @@ -26,6 +27,8 @@ import io.netty.util.AbstractReferenceCounted; import io.netty.util.ReferenceCountUtil; +import org.apache.spark.network.buffer.ManagedBuffer; + /** * A wrapper message that holds two separate pieces (a header and a body). * @@ -33,15 +36,35 @@ */ class MessageWithHeader extends AbstractReferenceCounted implements FileRegion { + @Nullable private final ManagedBuffer managedBuffer; private final ByteBuf header; private final int headerLength; private final Object body; private final long bodyLength; private long totalBytesTransferred; - MessageWithHeader(ByteBuf header, Object body, long bodyLength) { + /** + * Construct a new MessageWithHeader. + * + * @param managedBuffer the {@link ManagedBuffer} that the message body came from. This needs to + * be passed in so that the buffer can be freed when this message is + * deallocated. Ownership of the caller's reference to this buffer is + * transferred to this class, so if the caller wants to continue to use the + * ManagedBuffer in other messages then they will need to call retain() on + * it before passing it to this constructor. This may be null if and only if + * `body` is a {@link FileRegion}. + * @param header the message header. + * @param body the message body. Must be either a {@link ByteBuf} or a {@link FileRegion}. + * @param bodyLength the length of the message body, in bytes. + */ + MessageWithHeader( + @Nullable ManagedBuffer managedBuffer, + ByteBuf header, + Object body, + long bodyLength) { Preconditions.checkArgument(body instanceof ByteBuf || body instanceof FileRegion, "Body must be a ByteBuf or a FileRegion."); + this.managedBuffer = managedBuffer; this.header = header; this.headerLength = header.readableBytes(); this.body = body; @@ -99,6 +122,9 @@ public long transferTo(final WritableByteChannel target, final long position) th protected void deallocate() { header.release(); ReferenceCountUtil.release(body); + if (managedBuffer != null) { + managedBuffer.release(); + } } private int copyByteBuf(ByteBuf buf, WritableByteChannel target) throws IOException { diff --git a/network/common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java b/network/common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java index 6c98e733b462f..50035fba5702c 100644 --- a/network/common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java +++ b/network/common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java @@ -29,6 +29,8 @@ import static org.junit.Assert.*; +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.buffer.NettyManagedBuffer; import org.apache.spark.network.util.ByteArrayWritableChannel; public class MessageWithHeaderSuite { @@ -47,7 +49,8 @@ public void testShortWrite() throws Exception { public void testByteBufBody() throws Exception { ByteBuf header = Unpooled.copyLong(42); ByteBuf body = Unpooled.copyLong(84); - MessageWithHeader msg = new MessageWithHeader(header, body, body.readableBytes()); + ManagedBuffer managedBuf = new NettyManagedBuffer(body); + MessageWithHeader msg = new MessageWithHeader(managedBuf, header, body, body.readableBytes()); ByteBuf result = doWrite(msg, 1); assertEquals(msg.count(), result.readableBytes()); @@ -59,7 +62,7 @@ private void testFileRegionBody(int totalWrites, int writesPerCall) throws Excep ByteBuf header = Unpooled.copyLong(42); int headerLength = header.readableBytes(); TestFileRegion region = new TestFileRegion(totalWrites, writesPerCall); - MessageWithHeader msg = new MessageWithHeader(header, region, region.count()); + MessageWithHeader msg = new MessageWithHeader(null, header, region, region.count()); ByteBuf result = doWrite(msg, totalWrites / writesPerCall); assertEquals(headerLength + region.count(), result.readableBytes()); From 613498911f3867cddb72d0a3b235260e4e7d1433 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 12 Feb 2016 16:51:29 -0800 Subject: [PATCH 33/81] Add a test for OneForOneStreamManager.connectionTerminated --- .../server/OneForOneStreamManager.java | 1 - .../server/OneForOneStreamManagerSuite.java | 51 +++++++++++++++++++ 2 files changed, 51 insertions(+), 1 deletion(-) create mode 100644 network/common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java diff --git a/network/common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java b/network/common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java index e671854da1cae..ea9e735e0a173 100644 --- a/network/common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java +++ b/network/common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java @@ -20,7 +20,6 @@ import java.util.Iterator; import java.util.Map; import java.util.Random; -import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; diff --git a/network/common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java b/network/common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java new file mode 100644 index 0000000000000..6356ac6c24f80 --- /dev/null +++ b/network/common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java @@ -0,0 +1,51 @@ +/* + * 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.network.server; + +import java.util.ArrayList; +import java.util.List; + +import io.netty.channel.Channel; +import org.junit.Test; +import org.mockito.Mockito; +import static org.mockito.Mockito.*; + +import org.apache.spark.network.TestManagedBuffer; +import org.apache.spark.network.buffer.ManagedBuffer; + +public class OneForOneStreamManagerSuite { + + @Test + public void managedBuffersAreFeedWhenConnectionIsClosed() throws Exception { + OneForOneStreamManager manager = new OneForOneStreamManager(); + List buffers = new ArrayList<>(); + TestManagedBuffer buffer1 = Mockito.spy(new TestManagedBuffer(10)); + TestManagedBuffer buffer2 = Mockito.spy(new TestManagedBuffer(20)); + buffers.add(buffer1); + buffers.add(buffer2); + long streamId = manager.registerStream("appId", buffers.iterator()); + + Channel dummyChannel = Mockito.mock(Channel.class); + manager.registerChannel(dummyChannel, streamId); + + manager.connectionTerminated(dummyChannel); + + Mockito.verify(buffer1, times(1)).release(); + Mockito.verify(buffer2, times(1)).release(); + } +} From c9726c26fe5f92d84265346c3998d7ab40e75cff Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 12 Feb 2016 17:06:35 -0800 Subject: [PATCH 34/81] Add tests covering new release() call. --- .../network/protocol/MessageWithHeaderSuite.java | 16 +++++++++++++++- .../server/OneForOneStreamManagerSuite.java | 7 +++---- 2 files changed, 18 insertions(+), 5 deletions(-) diff --git a/network/common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java b/network/common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java index 50035fba5702c..2f9bd7e81fab1 100644 --- a/network/common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java +++ b/network/common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java @@ -26,9 +26,11 @@ import io.netty.channel.FileRegion; import io.netty.util.AbstractReferenceCounted; import org.junit.Test; +import org.mockito.Mockito; import static org.junit.Assert.*; +import org.apache.spark.network.TestManagedBuffer; import org.apache.spark.network.buffer.ManagedBuffer; import org.apache.spark.network.buffer.NettyManagedBuffer; import org.apache.spark.network.util.ByteArrayWritableChannel; @@ -48,7 +50,7 @@ public void testShortWrite() throws Exception { @Test public void testByteBufBody() throws Exception { ByteBuf header = Unpooled.copyLong(42); - ByteBuf body = Unpooled.copyLong(84); + ByteBuf body = Unpooled.copyLong(84).retain(); ManagedBuffer managedBuf = new NettyManagedBuffer(body); MessageWithHeader msg = new MessageWithHeader(managedBuf, header, body, body.readableBytes()); @@ -56,6 +58,17 @@ public void testByteBufBody() throws Exception { assertEquals(msg.count(), result.readableBytes()); assertEquals(42, result.readLong()); assertEquals(84, result.readLong()); + msg.deallocate(); + } + + @Test + public void testDeallocateReleasesManagedBuffer() throws Exception { + ByteBuf header = Unpooled.copyLong(42); + ManagedBuffer managedBuf = Mockito.spy(new TestManagedBuffer(84)); + ByteBuf body = ((ByteBuf) managedBuf.convertToNetty()).retain(); + MessageWithHeader msg = new MessageWithHeader(managedBuf, header, body, body.readableBytes()); + msg.deallocate(); + Mockito.verify(managedBuf, Mockito.times(1)).release(); } private void testFileRegionBody(int totalWrites, int writesPerCall) throws Exception { @@ -70,6 +83,7 @@ private void testFileRegionBody(int totalWrites, int writesPerCall) throws Excep for (long i = 0; i < 8; i++) { assertEquals(i, result.readLong()); } + msg.deallocate(); } private ByteBuf doWrite(MessageWithHeader msg, int minExpectedWrites) throws Exception { diff --git a/network/common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java b/network/common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java index 6356ac6c24f80..c647525d8f1bd 100644 --- a/network/common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java +++ b/network/common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java @@ -23,7 +23,6 @@ import io.netty.channel.Channel; import org.junit.Test; import org.mockito.Mockito; -import static org.mockito.Mockito.*; import org.apache.spark.network.TestManagedBuffer; import org.apache.spark.network.buffer.ManagedBuffer; @@ -40,12 +39,12 @@ public void managedBuffersAreFeedWhenConnectionIsClosed() throws Exception { buffers.add(buffer2); long streamId = manager.registerStream("appId", buffers.iterator()); - Channel dummyChannel = Mockito.mock(Channel.class); + Channel dummyChannel = Mockito.mock(Channel.class, Mockito.RETURNS_SMART_NULLS); manager.registerChannel(dummyChannel, streamId); manager.connectionTerminated(dummyChannel); - Mockito.verify(buffer1, times(1)).release(); - Mockito.verify(buffer2, times(1)).release(); + Mockito.verify(buffer1, Mockito.times(1)).release(); + Mockito.verify(buffer2, Mockito.times(1)).release(); } } From c629f261b23530b706eaf65baec37638505e4192 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 13 Feb 2016 11:43:49 -0800 Subject: [PATCH 35/81] Add defensive check to guard against exiting while loop when info.removed == true. --- .../main/scala/org/apache/spark/storage/BlockInfoManager.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index b0b5e067f275c..b8d2b6c0b0ba7 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -148,6 +148,7 @@ private[storage] class BlockInfoManager extends Logging { if (info.removed) return None if (blocking) wait() else return None } + if (info.removed) return None info.readerCount += 1 readLocksByTask(currentTaskAttemptId).add(blockId) logTrace(s"Task $currentTaskAttemptId acquired read lock for $blockId") @@ -178,6 +179,7 @@ private[storage] class BlockInfoManager extends Logging { if (info.removed) return None if (blocking) wait() else return None } + if (info.removed) return None } info.writerTask = currentTaskAttemptId writeLocksByTask.addBinding(currentTaskAttemptId, blockId) From 0aa23929e10d7a3bf5eae8bfb823c3e305113078 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 13 Feb 2016 12:03:18 -0800 Subject: [PATCH 36/81] Fix serialization problems in getMatchingBlockIds(). --- .../main/scala/org/apache/spark/storage/BlockManager.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) 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 4305ecc909cc0..c42dd47222f5a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -316,7 +316,11 @@ private[spark] class BlockManager( * may not know of). */ def getMatchingBlockIds(filter: BlockId => Boolean): Seq[BlockId] = { - (blockInfoManager.entries.map(_._1) ++ diskBlockManager.getAllBlocks()).filter(filter).toSeq + val matches = + (blockInfoManager.entries.map(_._1) ++ diskBlockManager.getAllBlocks()).filter(filter) + // The `toArray` is necessary here in order to force the list to be materialized so that we + // don't try to serialize a lazy iterator when responding to client requests. + matches.toArray.toSeq } /** From b273422d95e4a3688ba556b2dfe125961a43c2f2 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 13 Feb 2016 14:19:50 -0800 Subject: [PATCH 37/81] Remove bad retain. --- .../org/apache/spark/network/protocol/MessageEncoder.java | 5 +---- .../java/org/apache/spark/network/TestManagedBuffer.java | 2 +- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java b/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java index 442b60bdb9b5e..664df57feca4f 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java +++ b/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java @@ -49,11 +49,8 @@ public void encode(ChannelHandlerContext ctx, Message in, List out) thro // If the message has a body, take it out to enable zero-copy transfer for the payload. if (in.body() != null) { - bodyLength = in.body().size(); - if (bodyLength > 0) { - in.body().retain(); - } try { + bodyLength = in.body().size(); body = in.body().convertToNetty(); isBodyInFrame = in.isBodyInFrame(); } catch (Exception e) { diff --git a/network/common/src/test/java/org/apache/spark/network/TestManagedBuffer.java b/network/common/src/test/java/org/apache/spark/network/TestManagedBuffer.java index 83c90f9eff2b1..e15b1309efe50 100644 --- a/network/common/src/test/java/org/apache/spark/network/TestManagedBuffer.java +++ b/network/common/src/test/java/org/apache/spark/network/TestManagedBuffer.java @@ -44,7 +44,7 @@ public TestManagedBuffer(int len) { for (int i = 0; i < len; i ++) { byteArray[i] = (byte) i; } - this.underlying = new NettyManagedBuffer(Unpooled.wrappedBuffer(byteArray)); + this.underlying = new NettyManagedBuffer(Unpooled.wrappedBuffer(byteArray).retain()); } From 7639e03eeea8665808c0a9b6cc81c3298d4753e0 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 13 Feb 2016 14:22:35 -0800 Subject: [PATCH 38/81] Logging improvements that were helpful when debugging tests. --- .../scala/org/apache/spark/executor/Executor.scala | 8 +++++--- .../org/apache/spark/storage/BlockInfoManager.scala | 10 ++++++---- .../scala/org/apache/spark/storage/BlockManager.scala | 6 +++--- .../scala/org/apache/spark/ContextCleanerSuite.scala | 3 ++- 4 files changed, 16 insertions(+), 11 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 769237a8a5e43..6b9523bce5beb 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -218,7 +218,7 @@ private[spark] class Executor( threwException = false res } finally { - val releasedPins = env.blockManager.releaseAllLocksForTask(taskId) + val releasedLocks = env.blockManager.releaseAllLocksForTask(taskId) val freedMemory = taskMemoryManager.cleanUpAllAllocatedMemory() if (freedMemory > 0) { @@ -230,8 +230,10 @@ private[spark] class Executor( } } - if (releasedPins > 0) { - val errMsg = s"$releasedPins block pins were not released, TID = $taskId" + if (releasedLocks.nonEmpty) { + val errMsg = + s"${releasedLocks.size} block locks were not released by TID = $taskId:\n" + + releasedLocks.mkString("[", ", ", "]") if (conf.getBoolean("spark.storage.exceptionOnPinLeak", false) && !threwException) { throw new SparkException(errMsg) } else { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index b8d2b6c0b0ba7..ef4e803076874 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -258,9 +258,10 @@ private[storage] class BlockInfoManager extends Logging { * structures and updating the global pin counts. This method should be called at the * end of a task (either by a task completion handler or in `TaskRunner.run()`). * - * @return the number of pins released + * @return the ids of blocks whose pins were released */ - def releaseAllLocksForTask(taskAttemptId: TaskAttemptId): Int = { + def releaseAllLocksForTask(taskAttemptId: TaskAttemptId): Seq[BlockId] = { + val blocksWithReleasedLocks = mutable.ArrayBuffer[BlockId]() synchronized { writeLocksByTask.remove(taskAttemptId).foreach { locks => for (blockId <- locks) { @@ -268,16 +269,17 @@ private[storage] class BlockInfoManager extends Logging { assert(info.writerTask == taskAttemptId) info.writerTask = -1 } + blocksWithReleasedLocks += blockId } } notifyAll() } val readLocks = readLocksByTask.get(taskAttemptId) readLocksByTask.invalidate(taskAttemptId) - val totalPinCountForTask = readLocks.size() readLocks.entrySet().iterator().asScala.foreach { entry => val blockId = entry.getElement val lockCount = entry.getCount + blocksWithReleasedLocks += blockId synchronized { get(blockId).foreach { info => info.readerCount -= lockCount @@ -288,7 +290,7 @@ private[storage] class BlockInfoManager extends Logging { synchronized { notifyAll() } - totalPinCountForTask + blocksWithReleasedLocks } /** 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 c42dd47222f5a..9037e19dcb583 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -614,11 +614,11 @@ private[spark] class BlockManager( } /** - * Release all pins for the given task. + * Release all locks for the given task. * - * @return the total number of pins released. + * @return the blocks whose locks were released. */ - def releaseAllLocksForTask(taskAttemptId: Long): Int = { + def releaseAllLocksForTask(taskAttemptId: Long): Seq[BlockId] = { blockInfoManager.releaseAllLocksForTask(taskAttemptId) } diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index 7b0238091730d..d1e806b2eb80a 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -485,7 +485,8 @@ class CleanerTester( def assertCleanup()(implicit waitTimeout: PatienceConfiguration.Timeout) { try { eventually(waitTimeout, interval(100 millis)) { - assert(isAllCleanedUp) + assert(isAllCleanedUp, + "The following resources were not cleaned up:\n" + uncleanedResourcesToString) } postCleanupValidate() } finally { From 27e98a36bc343d3bbd1715a5f613f7e7a35b85c5 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 13 Feb 2016 14:33:39 -0800 Subject: [PATCH 39/81] Fix SparkContext leak in KryoSerializerDistributedSuite --- .../spark/serializer/KryoSerializerDistributedSuite.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala index a0483f6483889..c1484b0afa85f 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala @@ -23,7 +23,7 @@ import org.apache.spark._ import org.apache.spark.serializer.KryoDistributedTest._ import org.apache.spark.util.Utils -class KryoSerializerDistributedSuite extends SparkFunSuite { +class KryoSerializerDistributedSuite extends SparkFunSuite with LocalSparkContext { test("kryo objects are serialised consistently in different processes") { val conf = new SparkConf(false) @@ -34,7 +34,7 @@ class KryoSerializerDistributedSuite extends SparkFunSuite { val jar = TestUtils.createJarWithClasses(List(AppJarRegistrator.customClassName)) conf.setJars(List(jar.getPath)) - val sc = new SparkContext("local-cluster[2,1,1024]", "test", conf) + sc = new SparkContext("local-cluster[2,1,1024]", "test", conf) val original = Thread.currentThread.getContextClassLoader val loader = new java.net.URLClassLoader(Array(jar), Utils.getContextOrSparkClassLoader) SparkEnv.get.serializer.setDefaultClassLoader(loader) @@ -47,8 +47,6 @@ class KryoSerializerDistributedSuite extends SparkFunSuite { // Join the two RDDs, and force evaluation assert(shuffledRDD.join(cachedRDD).collect().size == 1) - - LocalSparkContext.stop(sc) } } From b72cd7b7f765d578ad4e44deccb3d96afd76fbbc Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 14 Feb 2016 16:20:44 -0800 Subject: [PATCH 40/81] Fix block replication bugs. --- .../scala/org/apache/spark/network/BlockDataManager.scala | 5 ++++- .../apache/spark/network/netty/NettyBlockRpcServer.scala | 6 +++++- .../main/scala/org/apache/spark/storage/BlockManager.scala | 2 +- .../apache/spark/storage/BlockManagerReplicationSuite.scala | 5 +++++ 4 files changed, 15 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala b/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala index e05cfd376a42f..9aba9544266eb 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala @@ -31,8 +31,11 @@ trait BlockDataManager { /** * Put the block locally, using the given storage level. + * + * Returns true if the block was stored and false if the put operation failed or the block + * aready existed. */ - def putBlockData(blockId: BlockId, data: ManagedBuffer, level: StorageLevel): Unit + def putBlockData(blockId: BlockId, data: ManagedBuffer, level: StorageLevel): Boolean /** * Release locks acquired by [[putBlockData()]] and [[getBlockData()]]. diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala index df8c21fb837ed..e4246df83a6ec 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala @@ -65,7 +65,11 @@ class NettyBlockRpcServer( val level: StorageLevel = serializer.newInstance().deserialize(ByteBuffer.wrap(uploadBlock.metadata)) val data = new NioManagedBuffer(ByteBuffer.wrap(uploadBlock.blockData)) - blockManager.putBlockData(BlockId(uploadBlock.blockId), data, level) + val blockId = BlockId(uploadBlock.blockId) + val putSucceeded = blockManager.putBlockData(blockId, data, level) + if (putSucceeded) { + blockManager.releaseLock(blockId) + } responseContext.onSuccess(ByteBuffer.allocate(0)) } } 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 9037e19dcb583..10d2c32c26a67 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -294,7 +294,7 @@ private[spark] class BlockManager( /** * Put the block locally, using the given storage level. */ - override def putBlockData(blockId: BlockId, data: ManagedBuffer, level: StorageLevel): Unit = { + override def putBlockData(blockId: BlockId, data: ManagedBuffer, level: StorageLevel): Boolean = { putBytes(blockId, data.nioByteBuffer(), level) } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala index ae1faf5d98f38..a94d8b424d956 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -190,6 +190,7 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo def putBlockAndGetLocations(blockId: String, level: StorageLevel): Set[BlockManagerId] = { stores.head.putSingle(blockId, new Array[Byte](blockSize), level) + stores.head.releaseLock(blockId) val locations = master.getLocations(blockId).sortBy { _.executorId }.toSet stores.foreach { _.removeBlock(blockId) } master.removeBlock(blockId) @@ -251,6 +252,7 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo // Insert a block with 2x replication and return the number of copies of the block def replicateAndGetNumCopies(blockId: String): Int = { store.putSingle(blockId, new Array[Byte](1000), StorageLevel.MEMORY_AND_DISK_2) + store.releaseLock(blockId) val numLocations = master.getLocations(blockId).size allStores.foreach { _.removeBlock(blockId) } numLocations @@ -288,6 +290,7 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo def replicateAndGetNumCopies(blockId: String, replicationFactor: Int): Int = { val storageLevel = StorageLevel(true, true, false, true, replicationFactor) initialStores.head.putSingle(blockId, new Array[Byte](blockSize), storageLevel) + initialStores.head.releaseLock(blockId) val numLocations = master.getLocations(blockId).size allStores.foreach { _.removeBlock(blockId) } numLocations @@ -355,6 +358,7 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo val blockId = new TestBlockId( "block-with-" + storageLevel.description.replace(" ", "-").toLowerCase) stores(0).putSingle(blockId, new Array[Byte](blockSize), storageLevel) + stores(0).releaseLock(blockId) // Assert that master know two locations for the block val blockLocations = master.getLocations(blockId).map(_.executorId).toSet @@ -393,6 +397,7 @@ class BlockManagerReplicationSuite extends SparkFunSuite with Matchers with Befo (1 to 10).foreach { i => testStore.putSingle(s"dummy-block-$i", new Array[Byte](1000), MEMORY_ONLY_SER) + testStore.releaseLock(s"dummy-block-$i") } (1 to 10).foreach { i => testStore.removeBlock(s"dummy-block-$i") From 5e23177ad80205054b2d3e33833bd107fdfbae39 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 14 Feb 2016 17:37:22 -0800 Subject: [PATCH 41/81] Fix locking in indirect task result code path. --- core/src/main/scala/org/apache/spark/executor/Executor.scala | 5 ++++- .../scala/org/apache/spark/scheduler/TaskResultGetter.scala | 1 - 2 files changed, 4 insertions(+), 2 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 6b9523bce5beb..a602fcac68a6b 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -279,8 +279,11 @@ private[spark] class Executor( ser.serialize(new IndirectTaskResult[Any](TaskResultBlockId(taskId), resultSize)) } else if (resultSize >= maxRpcMessageSize) { val blockId = TaskResultBlockId(taskId) - env.blockManager.putBytes( + val putSucceeded = env.blockManager.putBytes( blockId, serializedDirectResult, StorageLevel.MEMORY_AND_DISK_SER) + if (putSucceeded) { + env.blockManager.releaseLock(blockId) + } logInfo( s"Finished $taskName (TID $taskId). $resultSize bytes result sent via BlockManager)") ser.serialize(new IndirectTaskResult[Any](blockId, resultSize)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 541bc256f983b..c94c4f55e9ced 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -83,7 +83,6 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul } val deserializedResult = serializer.get().deserialize[DirectTaskResult[_]]( serializedTaskResult.get) - sparkEnv.blockManager.releaseLock(blockId) sparkEnv.blockManager.master.removeBlock(blockId) (deserializedResult, size) } From f0b6d711b29e7e7b71a954860a66df27c26380b1 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 14 Feb 2016 17:44:23 -0800 Subject: [PATCH 42/81] Add a missing release() in ReceivedBlockHandler. --- .../apache/spark/streaming/receiver/ReceivedBlockHandler.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala index e22e320b17126..514009d71d8ae 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala @@ -91,6 +91,8 @@ private[streaming] class BlockManagerBasedBlockHandler( if (!putSucceeded) { throw new SparkException( s"Could not store $blockId to block manager with storage level $storageLevel") + } else { + blockManager.releaseLock(blockId) } BlockManagerBasedStoreResult(blockId, numRecords) } From e549f2f82ea6517c6b9fe39a47f3fb19bc950b27 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 14 Feb 2016 17:59:58 -0800 Subject: [PATCH 43/81] Add another missing release in WriteAheadLogBasedBlockHandler --- .../apache/spark/streaming/receiver/ReceivedBlockHandler.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala index 514009d71d8ae..3d9c085013ff5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala @@ -191,6 +191,8 @@ private[streaming] class WriteAheadLogBasedBlockHandler( if (!putSucceeded) { throw new SparkException( s"Could not store $blockId to block manager with storage level $storageLevel") + } else { + blockManager.releaseLock(blockId) } } From 6d0940028ecd7f3bec5d965554339dc3d44fd69e Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 14 Feb 2016 18:16:27 -0800 Subject: [PATCH 44/81] Fix SQL test compilation. --- .../src/test/scala/org/apache/spark/sql/CachedTableSuite.scala | 2 +- .../test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 5d62460dd0b1f..98744e40072fb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -47,7 +47,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext def isMaterialized(rddId: Int): Boolean = { val maybeBlock = sparkContext.env.blockManager.get(RDDBlockId(rddId, 0)) - maybeBlock.foreach(_ => sparkContext.env.blockManager.unpin(RDDBlockId(rddId, 0))) + maybeBlock.foreach(_ => sparkContext.env.blockManager.releaseLock(RDDBlockId(rddId, 0))) maybeBlock.nonEmpty } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index 694df185f19e1..86f02e68e5b0e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -41,7 +41,7 @@ class CachedTableSuite extends QueryTest with TestHiveSingleton { def isMaterialized(rddId: Int): Boolean = { val maybeBlock = sparkContext.env.blockManager.get(RDDBlockId(rddId, 0)) - maybeBlock.foreach(_ => sparkContext.env.blockManager.unpin(RDDBlockId(rddId, 0))) + maybeBlock.foreach(_ => sparkContext.env.blockManager.releaseLock(RDDBlockId(rddId, 0))) maybeBlock.nonEmpty } From 717c476668bad9a1516bd462461714108163c496 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 14 Feb 2016 18:30:05 -0800 Subject: [PATCH 45/81] Add missing lock release in CacheManager. --- core/src/main/scala/org/apache/spark/CacheManager.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 36b18530c0fe7..0cb3a8b80834a 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -164,7 +164,9 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { case Left(arr) => // We have successfully unrolled the entire partition, so cache it in memory blockManager.putArray(key, arr, level, tellMaster = true, effectiveStorageLevel) - arr.iterator.asInstanceOf[Iterator[T]] + CompletionIterator[T, Iterator[T]]( + arr.iterator.asInstanceOf[Iterator[T]], + blockManager.releaseLock(key)) case Right(it) => // There is not enough space to cache this partition in memory val returnValues = it.asInstanceOf[Iterator[T]] From 0c08731890f1a34606b95ffd31d063fc931b4e21 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 16 Feb 2016 13:25:27 -0800 Subject: [PATCH 46/81] Revert change in network/common/src/test/java/org/apache/spark/network/TestManagedBuffer.java --- .../test/java/org/apache/spark/network/TestManagedBuffer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/network/common/src/test/java/org/apache/spark/network/TestManagedBuffer.java b/network/common/src/test/java/org/apache/spark/network/TestManagedBuffer.java index e15b1309efe50..83c90f9eff2b1 100644 --- a/network/common/src/test/java/org/apache/spark/network/TestManagedBuffer.java +++ b/network/common/src/test/java/org/apache/spark/network/TestManagedBuffer.java @@ -44,7 +44,7 @@ public TestManagedBuffer(int len) { for (int i = 0; i < len; i ++) { byteArray[i] = (byte) i; } - this.underlying = new NettyManagedBuffer(Unpooled.wrappedBuffer(byteArray).retain()); + this.underlying = new NettyManagedBuffer(Unpooled.wrappedBuffer(byteArray)); } From 55b5b194544ff2a056fe7eafb73ab2d014d4a122 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 16 Feb 2016 19:20:27 -0800 Subject: [PATCH 47/81] Check preconditions in remove(). --- .../spark/storage/BlockInfoManager.scala | 17 ++++++++--- .../spark/storage/BlockInfoManagerSuite.scala | 29 +++++++++++++++++++ 2 files changed, 42 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index ef4e803076874..c0eeeaf78d6f2 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -324,13 +324,22 @@ private[storage] class BlockInfoManager extends Logging { /** * Removes the given block and automatically drops all locks on it. * - * TODO: document validity conditions. + * This can only be called while holding a write lock on the given block. */ def remove(blockId: BlockId): Unit = synchronized { logTrace(s"Task $currentTaskAttemptId trying to remove block $blockId") - // TODO: Should probably have safety checks here - infos.remove(blockId).foreach { info => - info.removed = true + infos.get(blockId) match { + case Some(blockInfo) => + if (blockInfo.writerTask != currentTaskAttemptId) { + throw new IllegalStateException( + s"Task $currentTaskAttemptId called remove() on block $blockId without a write lock") + } else { + infos.remove(blockId) + blockInfo.removed = true + } + case None => + throw new IllegalArgumentException( + s"Task $currentTaskAttemptId called remove() on non-existent block $blockId") } notifyAll() } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala index 0ddfb13955619..6fb15d62e7724 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala @@ -193,6 +193,35 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { assert(Await.result(write2Future, 1.seconds).isDefined) } + test("removing a non-existent block throws IllegalArgumentException") { + withTaskId(0) { + intercept[IllegalArgumentException] { + blockInfoManager.remove("non-existent-block") + } + } + } + + test("removing a block without holding any locks throws IllegalStateException") { + withTaskId(0) { + assert(blockInfoManager.putAndLockForWritingIfAbsent("block", newBlockInfo())) + blockInfoManager.releaseLock("block") + intercept[IllegalStateException] { + blockInfoManager.remove("block") + } + } + } + + test("removing a block while holding only a read lock throws IllegalStateException") { + withTaskId(0) { + assert(blockInfoManager.putAndLockForWritingIfAbsent("block", newBlockInfo())) + blockInfoManager.releaseLock("block") + assert(blockInfoManager.getAndLockForReading("block").isDefined) + intercept[IllegalStateException] { + blockInfoManager.remove("block") + } + } + } + test("removing a block causes blocked callers to receive None") { withTaskId(0) { assert(blockInfoManager.putAndLockForWritingIfAbsent("block", newBlockInfo())) From 3a124806a8c99ad4a84285a24476ab01117ca907 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 17 Feb 2016 13:49:36 -0800 Subject: [PATCH 48/81] Free locks in dropFromMemory(). --- .../src/main/scala/org/apache/spark/storage/MemoryStore.scala | 4 +++- 1 file changed, 3 insertions(+), 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 e721c60c621d5..0fdcd2bae0771 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -475,7 +475,9 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo logInfo(s"Will not store $id as it would require dropping another block " + "from the same RDD") } - // TODO: free write locks here. + selectedBlocks.foreach { id => + blockManager.releaseLock(id) + } 0L } } From 25b09d762273e5b743055e2944734671b3a9197c Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 17 Feb 2016 14:15:29 -0800 Subject: [PATCH 49/81] Guard against MemoryStore removing the block first. --- .../scala/org/apache/spark/storage/BlockManager.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 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 10d2c32c26a67..9e0b27c1abd16 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -812,12 +812,12 @@ private[spark] class BlockManager( } } } finally { - // If we failed in putting the block to memory/disk, notify other possible readers - // that it has failed, and then remove it from the block info map. if (!blockWasSuccessfullyStored) { - // Note that the remove must happen before markFailure otherwise another thread - // could've inserted a new BlockInfo before we remove it. - blockInfoManager.remove(blockId) + // Guard against the fact that MemoryStore might have already removed the block if the + // put() failed and the block could not be dropped to disk. + if (blockInfoManager.getAndLockForWriting(blockId, blocking = false).isDefined) { + blockInfoManager.remove(blockId) + } logWarning(s"Putting block $blockId failed") } } From bcb8318b558cb9add9b9ddf260c929ec4e268542 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 17 Feb 2016 18:28:57 -0800 Subject: [PATCH 50/81] Fix bug in release of locks after network fetch of block data. --- .../scala/org/apache/spark/CacheManager.scala | 21 +++++++++++++------ .../apache/spark/storage/BlockManager.scala | 5 +++++ 2 files changed, 20 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 0cb3a8b80834a..bd0174ff04b18 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -19,6 +19,7 @@ package org.apache.spark import scala.collection.mutable +import org.apache.spark.executor.DataReadMethod import org.apache.spark.rdd.RDD import org.apache.spark.storage._ import org.apache.spark.util.CompletionIterator @@ -47,9 +48,14 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { val existingMetrics = context.taskMetrics().registerInputMetrics(blockResult.readMethod) existingMetrics.incBytesReadInternal(blockResult.bytes) - val iter = CompletionIterator[T, Iterator[T]]( - blockResult.data.asInstanceOf[Iterator[T]], - blockManager.releaseLock(key)) + val iter = { + val dataIter = blockResult.data.asInstanceOf[Iterator[T]] + if (blockResult.readMethod != DataReadMethod.Network) { + CompletionIterator[T, Iterator[T]](dataIter, blockManager.releaseLock(key)) + } else { + dataIter + } + } new InterruptibleIterator[T](context, iter) { override def next(): T = { existingMetrics.incRecordsReadInternal(1) @@ -142,9 +148,12 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { blockManager.putIterator(key, values, level, tellMaster = true, effectiveStorageLevel) blockManager.get(key) match { case Some(v) => - CompletionIterator[T, Iterator[T]]( - v.data.asInstanceOf[Iterator[T]], - blockManager.releaseLock(key)) + val iter = v.data.asInstanceOf[Iterator[T]] + if (v.readMethod != DataReadMethod.Network) { + CompletionIterator[T, Iterator[T]](iter, blockManager.releaseLock(key)) + } else { + iter + } case None => logInfo(s"Failure to store $key") throw new BlockException(key, s"Block manager failed to return cached value for $key!") 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 9e0b27c1abd16..d961df91d774e 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -518,6 +518,8 @@ private[spark] class BlockManager( /** * Get block from remote block managers. + * + * This does not acquire a local lock on this block. */ def getRemote(blockId: BlockId): Option[BlockResult] = { logDebug(s"Getting remote block $blockId") @@ -584,6 +586,9 @@ private[spark] class BlockManager( /** * Get a block from the block manager (either local or remote). + * + * This acquires a read lock on the block if the block was stored locally and does not acquire + * any locks if the block was fetched from a remote block manager. */ def get(blockId: BlockId): Option[BlockResult] = { val local = getLocal(blockId) From 4e11d00398862e20b5effcf679dda1b86661a6c6 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 18 Feb 2016 14:14:35 -0800 Subject: [PATCH 51/81] Rename methods. --- .../spark/storage/BlockInfoManager.scala | 28 +++--- .../apache/spark/storage/BlockManager.scala | 25 +++-- .../storage/BlockManagerManagedBuffer.scala | 2 +- .../apache/spark/storage/MemoryStore.scala | 2 +- .../spark/storage/BlockInfoManagerSuite.scala | 96 +++++++++---------- 5 files changed, 75 insertions(+), 78 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index c0eeeaf78d6f2..b51e79860ae8d 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -88,8 +88,8 @@ private[storage] class BlockInfoManager extends Logging { /** * Used to look up metadata for individual blocks. Entries are added to this map via an atomic - * set-if-not-exists operation ([[putAndLockForWritingIfAbsent()]]) and are removed - * by [[remove()]]. + * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are removed + * by [[removeBlock()]]. */ @GuardedBy("this") private[this] val infos = new mutable.HashMap[BlockId, BlockInfo] @@ -139,7 +139,7 @@ private[storage] class BlockInfoManager extends Logging { * @return None if the block did not exist or was removed (in which case no lock is held), or * Some(BlockInfo) (in which case the block is locked for reading). */ - def getAndLockForReading( + def lockForReading( blockId: BlockId, blocking: Boolean = true): Option[BlockInfo] = synchronized { logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId") @@ -161,7 +161,7 @@ private[storage] class BlockInfoManager extends Logging { * * If this is called by a task which already holds the block's exclusive write lock, then this * will return success but will not further increment any lock counts (so both write-lock - * acquisitions will be freed by the same [[releaseLock()]] or [[downgradeLock()]] call. + * acquisitions will be freed by the same [[unlock()]] or [[downgradeLock()]] call. * * @param blockId the block to lock. * @param blocking if true (default), this call will block until the lock is acquired. If false, @@ -169,7 +169,7 @@ private[storage] class BlockInfoManager extends Logging { * @return None if the block did not exist or was removed (in which case no lock is held), or * Some(BlockInfo) (in which case the block is locked for writing). */ - def getAndLockForWriting( + def lockForWriting( blockId: BlockId, blocking: Boolean = true): Option[BlockInfo] = synchronized { logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId") @@ -203,15 +203,15 @@ private[storage] class BlockInfoManager extends Logging { val info = get(blockId).get require(info.writerTask == currentTaskAttemptId, s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold") - releaseLock(blockId) - getAndLockForReading(blockId, blocking = false) + unlock(blockId) + lockForReading(blockId, blocking = false) notifyAll() } /** * Release a lock on the given block. */ - def releaseLock(blockId: BlockId): Unit = synchronized { + def unlock(blockId: BlockId): Unit = synchronized { logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId") val info = get(blockId).getOrElse { throw new IllegalStateException(s"Block $blockId not found") @@ -233,13 +233,13 @@ private[storage] class BlockInfoManager extends Logging { /** * Atomically create metadata for a non-existent block. * - * TODO: clarify locking semantics when this returns false. - * * @param blockId the block id. * @param newBlockInfo the block info for the new block. - * @return true if the block did not already exist, false otherwise. + * @return true if the block did not already exist, false otherwise. If this returns false, then + * no new locks are acquired. If this returns true, a write lock on the new block will + * be held. */ - def putAndLockForWritingIfAbsent( + def lockNewBlockForWriting( blockId: BlockId, newBlockInfo: BlockInfo): Boolean = synchronized { logTrace(s"Task $currentTaskAttemptId trying to put $blockId") @@ -260,7 +260,7 @@ private[storage] class BlockInfoManager extends Logging { * * @return the ids of blocks whose pins were released */ - def releaseAllLocksForTask(taskAttemptId: TaskAttemptId): Seq[BlockId] = { + def unlockAllLocksForTask(taskAttemptId: TaskAttemptId): Seq[BlockId] = { val blocksWithReleasedLocks = mutable.ArrayBuffer[BlockId]() synchronized { writeLocksByTask.remove(taskAttemptId).foreach { locks => @@ -326,7 +326,7 @@ private[storage] class BlockInfoManager extends Logging { * * This can only be called while holding a write lock on the given block. */ - def remove(blockId: BlockId): Unit = synchronized { + def removeBlock(blockId: BlockId): Unit = synchronized { logTrace(s"Task $currentTaskAttemptId trying to remove block $blockId") infos.get(blockId) match { case Some(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 d961df91d774e..65fc21e528567 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -427,7 +427,7 @@ private[spark] class BlockManager( } private def doGetLocal(blockId: BlockId, asBlockResult: Boolean): Option[Any] = { - blockInfoManager.getAndLockForReading(blockId) match { + blockInfoManager.lockForReading(blockId) match { case None => logDebug(s"Block $blockId was not found") None @@ -615,7 +615,7 @@ private[spark] class BlockManager( * Release a lock on the given block. */ def releaseLock(blockId: BlockId): Unit = { - blockInfoManager.releaseLock(blockId) + blockInfoManager.unlock(blockId) } /** @@ -624,7 +624,7 @@ private[spark] class BlockManager( * @return the blocks whose locks were released. */ def releaseAllLocksForTask(taskAttemptId: Long): Seq[BlockId] = { - blockInfoManager.releaseAllLocksForTask(taskAttemptId) + blockInfoManager.unlockAllLocksForTask(taskAttemptId) } /** @@ -718,9 +718,8 @@ private[spark] class BlockManager( * to be dropped right after it got put into memory. Note, however, that other threads will * not be able to get() this block until we call markReady on its BlockInfo. */ val putBlockInfo = { - // TODO(josh): if an existing put is in progress, do we block to see if it's done / succeeds? val newInfo = new BlockInfo(level, tellMaster) - if (blockInfoManager.putAndLockForWritingIfAbsent(blockId, newInfo)) { + if (blockInfoManager.lockNewBlockForWriting(blockId, newInfo)) { newInfo } else { logWarning(s"Block $blockId already exists on this machine; not re-adding it") @@ -728,8 +727,6 @@ private[spark] class BlockManager( } } - // TODO: release write lock and split up this code. - val startTimeMs = System.currentTimeMillis /* If we're storing values and we need to replicate the data, we'll want access to the values, @@ -820,8 +817,8 @@ private[spark] class BlockManager( if (!blockWasSuccessfullyStored) { // Guard against the fact that MemoryStore might have already removed the block if the // put() failed and the block could not be dropped to disk. - if (blockInfoManager.getAndLockForWriting(blockId, blocking = false).isDefined) { - blockInfoManager.remove(blockId) + if (blockInfoManager.lockForWriting(blockId, blocking = false).isDefined) { + blockInfoManager.removeBlock(blockId) } logWarning(s"Putting block $blockId failed") } @@ -1006,7 +1003,7 @@ private[spark] class BlockManager( data: () => Either[Array[Any], ByteBuffer]): Unit = { logInfo(s"Dropping block $blockId from memory") - blockInfoManager.getAndLockForWriting(blockId) match { + blockInfoManager.lockForWriting(blockId) match { case None => logDebug(s"Block $blockId has already been dropped") case Some(info) => @@ -1041,9 +1038,9 @@ private[spark] class BlockManager( } if (!level.useDisk) { // The block is completely gone from this node; forget it so we can put() it again later. - blockInfoManager.remove(blockId) + blockInfoManager.removeBlock(blockId) } else { - blockInfoManager.releaseLock(blockId) + blockInfoManager.unlock(blockId) } if (blockIsUpdated) { Option(TaskContext.get()).foreach { c => @@ -1083,7 +1080,7 @@ private[spark] class BlockManager( */ def removeBlock(blockId: BlockId, tellMaster: Boolean = true): Unit = { logDebug(s"Removing block $blockId") - blockInfoManager.getAndLockForWriting(blockId) match { + blockInfoManager.lockForWriting(blockId) match { case None => // The block has already been removed; do nothing. logWarning(s"Asked to remove block $blockId, which does not exist") @@ -1095,7 +1092,7 @@ private[spark] class BlockManager( logWarning(s"Block $blockId could not be removed as it was not found in either " + "the disk, memory, or external block store") } - blockInfoManager.remove(blockId) + blockInfoManager.removeBlock(blockId) if (tellMaster && info.tellMaster) { val status = getCurrentBlockStatus(blockId, info) reportBlockStatus(blockId, info, status) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerManagedBuffer.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerManagedBuffer.scala index 1ab44316a8210..38d593962171e 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerManagedBuffer.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerManagedBuffer.scala @@ -29,7 +29,7 @@ private[storage] class BlockManagerManagedBuffer( override def retain(): ManagedBuffer = { super.retain() - val locked = blockManager.blockInfoManager.getAndLockForReading(blockId, blocking = false) + val locked = blockManager.blockInfoManager.lockForReading(blockId, blocking = false) assert(locked.isDefined) this } 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 0fdcd2bae0771..7a5d316467fd8 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -445,7 +445,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo val pair = iterator.next() val blockId = pair.getKey if (blockIsEvictable(blockId)) { - if (blockManager.blockInfoManager.getAndLockForWriting(blockId, false).isDefined) { + if (blockManager.blockInfoManager.lockForWriting(blockId, false).isDefined) { selectedBlocks += blockId freedMemory += pair.getValue.size } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala index 6fb15d62e7724..5f88500919877 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala @@ -65,20 +65,20 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { test("get non-existent block") { assert(blockInfoManager.get("non-existent-block").isEmpty) - assert(blockInfoManager.getAndLockForReading("non-existent-block").isEmpty) - assert(blockInfoManager.getAndLockForWriting("non-existent-block").isEmpty) + assert(blockInfoManager.lockForReading("non-existent-block").isEmpty) + assert(blockInfoManager.lockForWriting("non-existent-block").isEmpty) } test("basic putAndLockForWritingIfAbsent") { val blockInfo = newBlockInfo() withTaskId(1) { - assert(blockInfoManager.putAndLockForWritingIfAbsent("block", blockInfo)) + assert(blockInfoManager.lockNewBlockForWriting("block", blockInfo)) assert(blockInfoManager.get("block").get eq blockInfo) - assert(!blockInfoManager.putAndLockForWritingIfAbsent("block", newBlockInfo())) + assert(!blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) assert(blockInfoManager.get("block").get eq blockInfo) assert(blockInfo.readerCount === 0) assert(blockInfo.writerTask === 1) - blockInfoManager.releaseLock("block") + blockInfoManager.unlock("block") assert(blockInfo.readerCount === 0) assert(blockInfo.writerTask === -1) } @@ -88,53 +88,53 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { test("read locks are reentrant") { withTaskId(1) { - assert(blockInfoManager.putAndLockForWritingIfAbsent("block", newBlockInfo())) - blockInfoManager.releaseLock("block") - assert(blockInfoManager.getAndLockForReading("block").isDefined) - assert(blockInfoManager.getAndLockForReading("block").isDefined) + assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) + blockInfoManager.unlock("block") + assert(blockInfoManager.lockForReading("block").isDefined) + assert(blockInfoManager.lockForReading("block").isDefined) assert(blockInfoManager.get("block").get.readerCount === 2) assert(blockInfoManager.get("block").get.writerTask === -1) - blockInfoManager.releaseLock("block") + blockInfoManager.unlock("block") assert(blockInfoManager.get("block").get.readerCount === 1) - blockInfoManager.releaseLock("block") + blockInfoManager.unlock("block") assert(blockInfoManager.get("block").get.readerCount === 0) } } test("multiple tasks can hold read locks") { withTaskId(0) { - assert(blockInfoManager.putAndLockForWritingIfAbsent("block", newBlockInfo())) - blockInfoManager.releaseLock("block") + assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) + blockInfoManager.unlock("block") } - withTaskId(1) { assert(blockInfoManager.getAndLockForReading("block").isDefined) } - withTaskId(2) { assert(blockInfoManager.getAndLockForReading("block").isDefined) } - withTaskId(3) { assert(blockInfoManager.getAndLockForReading("block").isDefined) } - withTaskId(4) { assert(blockInfoManager.getAndLockForReading("block").isDefined) } + withTaskId(1) { assert(blockInfoManager.lockForReading("block").isDefined) } + withTaskId(2) { assert(blockInfoManager.lockForReading("block").isDefined) } + withTaskId(3) { assert(blockInfoManager.lockForReading("block").isDefined) } + withTaskId(4) { assert(blockInfoManager.lockForReading("block").isDefined) } assert(blockInfoManager.get("block").get.readerCount === 4) } test("single task can hold write lock") { withTaskId(0) { - assert(blockInfoManager.putAndLockForWritingIfAbsent("block", newBlockInfo())) - blockInfoManager.releaseLock("block") + assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) + blockInfoManager.unlock("block") } withTaskId(1) { - assert(blockInfoManager.getAndLockForWriting("block").isDefined) + assert(blockInfoManager.lockForWriting("block").isDefined) assert(blockInfoManager.get("block").get.writerTask === 1) } withTaskId(2) { - assert(blockInfoManager.getAndLockForWriting("block", blocking = false).isEmpty) + assert(blockInfoManager.lockForWriting("block", blocking = false).isEmpty) assert(blockInfoManager.get("block").get.writerTask === 1) } } test("downgrade lock") { withTaskId(0) { - assert(blockInfoManager.putAndLockForWritingIfAbsent("block", newBlockInfo())) + assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) blockInfoManager.downgradeLock("block") } withTaskId(1) { - assert(blockInfoManager.getAndLockForReading("block").isDefined) + assert(blockInfoManager.lockForReading("block").isDefined) } assert(blockInfoManager.get("block").get.readerCount === 2) assert(blockInfoManager.get("block").get.writerTask === -1) @@ -142,21 +142,21 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { test("write lock will block readers") { withTaskId(0) { - assert(blockInfoManager.putAndLockForWritingIfAbsent("block", newBlockInfo())) + assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) } val get1Future = Future { withTaskId(1) { - blockInfoManager.getAndLockForReading("block") + blockInfoManager.lockForReading("block") } } val get2Future = Future { withTaskId(2) { - blockInfoManager.getAndLockForReading("block") + blockInfoManager.lockForReading("block") } } Thread.sleep(300) // Hack to try to ensure that both future tasks are waiting withTaskId(0) { - blockInfoManager.releaseLock("block") + blockInfoManager.unlock("block") } assert(Await.result(get1Future, 1.seconds).isDefined) assert(Await.result(get2Future, 1.seconds).isDefined) @@ -165,29 +165,29 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { test("read locks will block writer") { withTaskId(0) { - assert(blockInfoManager.putAndLockForWritingIfAbsent("block", newBlockInfo())) - blockInfoManager.releaseLock("block") - blockInfoManager.getAndLockForReading("block") + assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) + blockInfoManager.unlock("block") + blockInfoManager.lockForReading("block") } val write1Future = Future { withTaskId(1) { - blockInfoManager.getAndLockForWriting("block") + blockInfoManager.lockForWriting("block") } } val write2Future = Future { withTaskId(2) { - blockInfoManager.getAndLockForWriting("block") + blockInfoManager.lockForWriting("block") } } Thread.sleep(300) // Hack to try to ensure that both future tasks are waiting withTaskId(0) { - blockInfoManager.releaseLock("block") + blockInfoManager.unlock("block") } assert( Await.result(Future.firstCompletedOf(Seq(write1Future, write2Future)), 1.seconds).isDefined) val firstWriteWinner = if (write1Future.isCompleted) 1 else 2 withTaskId(firstWriteWinner) { - blockInfoManager.releaseLock("block") + blockInfoManager.unlock("block") } assert(Await.result(write1Future, 1.seconds).isDefined) assert(Await.result(write2Future, 1.seconds).isDefined) @@ -196,49 +196,49 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { test("removing a non-existent block throws IllegalArgumentException") { withTaskId(0) { intercept[IllegalArgumentException] { - blockInfoManager.remove("non-existent-block") + blockInfoManager.removeBlock("non-existent-block") } } } test("removing a block without holding any locks throws IllegalStateException") { withTaskId(0) { - assert(blockInfoManager.putAndLockForWritingIfAbsent("block", newBlockInfo())) - blockInfoManager.releaseLock("block") + assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) + blockInfoManager.unlock("block") intercept[IllegalStateException] { - blockInfoManager.remove("block") + blockInfoManager.removeBlock("block") } } } test("removing a block while holding only a read lock throws IllegalStateException") { withTaskId(0) { - assert(blockInfoManager.putAndLockForWritingIfAbsent("block", newBlockInfo())) - blockInfoManager.releaseLock("block") - assert(blockInfoManager.getAndLockForReading("block").isDefined) + assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) + blockInfoManager.unlock("block") + assert(blockInfoManager.lockForReading("block").isDefined) intercept[IllegalStateException] { - blockInfoManager.remove("block") + blockInfoManager.removeBlock("block") } } } test("removing a block causes blocked callers to receive None") { withTaskId(0) { - assert(blockInfoManager.putAndLockForWritingIfAbsent("block", newBlockInfo())) + assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) } val getFuture = Future { withTaskId(1) { - blockInfoManager.getAndLockForReading("block") + blockInfoManager.lockForReading("block") } } val writeFuture = Future { withTaskId(2) { - blockInfoManager.getAndLockForWriting("block") + blockInfoManager.lockForWriting("block") } } Thread.sleep(300) // Hack to try to ensure that both future tasks are waiting withTaskId(0) { - blockInfoManager.remove("block") + blockInfoManager.removeBlock("block") } assert(Await.result(getFuture, 1.seconds).isEmpty) assert(Await.result(writeFuture, 1.seconds).isEmpty) @@ -246,10 +246,10 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { test("releaseAllLocksForTask releases write locks") { withTaskId(0) { - assert(blockInfoManager.putAndLockForWritingIfAbsent("block", newBlockInfo())) + assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) } assert(blockInfoManager.getNumberOfMapEntries === 3) - blockInfoManager.releaseAllLocksForTask(0) + blockInfoManager.unlockAllLocksForTask(0) assert(blockInfoManager.getNumberOfMapEntries === 1) } } From ed44f45fe793503e11773429efdd9c2dd26c4391 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 18 Feb 2016 14:15:41 -0800 Subject: [PATCH 52/81] Remove outdated block comment. --- .../apache/spark/storage/MemoryStore.scala | 21 ------------------- 1 file changed, 21 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 7a5d316467fd8..201b4f25251d6 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -208,27 +208,6 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo } override def remove(blockId: BlockId): Boolean = memoryManager.synchronized { - // This method is called from two different places: - // - // 1. When removing a block in BlockManager.removeBlock(). This is called by ContextCleaner - // cleanup code (e.g. when removing blocks from RDDs which are have fallen out of scope on - // the driver) or when a user explicitly unpersists an RDD or deletes a broadcast variable. - // - // 2. When dropping a block memory in BlockManager.dropFromMemory(), which is called by the - // MemoryStore when dropping blocks to free up space. The MemoryStore will never evict a - // pinned block. - // - // As a result, the only situation where `pinCount != 0` in this block is if the user performed - // an unsafe manual block eviction (which currently has undefined semantics), so in that case - // we choose to fail the user-initiated eviction rather than possibly crash running tasks by - // deleting data that they're using. - // - // Regarding thread-safety: - // - // - We want to avoid a race where we see that the pin count is zero, begin removing a block, - // then have a new read which re-pins the block right as we start removing it here. - // - In the code below, we synchronize on `entries` before checking the pin count. - // - In order for a val entry = entries.synchronized { entries.remove(blockId) } From a401adcd40704c891386c94d4ec071e9b2ba62b8 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 22 Feb 2016 16:49:01 -0800 Subject: [PATCH 53/81] Use try-finally in afterEach(). --- .../org/apache/spark/storage/BlockInfoManagerSuite.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala index 5f88500919877..c9300971cbb0a 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala @@ -37,8 +37,11 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { } override protected def afterEach(): Unit = { - blockInfoManager = null - super.afterEach() + try { + blockInfoManager = null + } finally { + super.afterEach() + } } private implicit def stringToBlockId(str: String): BlockId = { From 504986f548e09e3097dbb84d4e50c760c538aaee Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 22 Feb 2016 17:58:49 -0800 Subject: [PATCH 54/81] Try to clean up confusing release logic related to dropFromMemory(). --- .../apache/spark/storage/BlockManager.scala | 32 ++++++++----------- .../apache/spark/storage/MemoryStore.scala | 29 ++++++++++------- .../spark/storage/BlockManagerSuite.scala | 28 +++++++++++++--- 3 files changed, 55 insertions(+), 34 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 65fc21e528567..eddc6e342d792 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -800,12 +800,11 @@ private[spark] class BlockManager( } val putBlockStatus = getCurrentBlockStatus(blockId, putBlockInfo) - if (putBlockStatus.storageLevel != StorageLevel.NONE) { + blockWasSuccessfullyStored = putBlockStatus.storageLevel.isValid + if (blockWasSuccessfullyStored) { // Now that the block is in either the memory, externalBlockStore, or disk store, // let other threads read it, and tell the master about it. - blockWasSuccessfullyStored = true putBlockInfo.size = size - blockInfoManager.downgradeLock(blockId) if (tellMaster) { reportBlockStatus(blockId, putBlockInfo, putBlockStatus) } @@ -814,12 +813,10 @@ private[spark] class BlockManager( } } } finally { - if (!blockWasSuccessfullyStored) { - // Guard against the fact that MemoryStore might have already removed the block if the - // put() failed and the block could not be dropped to disk. - if (blockInfoManager.lockForWriting(blockId, blocking = false).isDefined) { - blockInfoManager.removeBlock(blockId) - } + if (blockWasSuccessfullyStored) { + blockInfoManager.downgradeLock(blockId) + } else { + blockInfoManager.removeBlock(blockId) logWarning(s"Putting block $blockId failed") } } @@ -828,7 +825,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 (putLevel.replication > 1) { + if (blockWasSuccessfullyStored && putLevel.replication > 1) { data match { case ByteBufferValues(bytes) => if (replicationFuture != null) { @@ -852,7 +849,7 @@ private[spark] class BlockManager( BlockManager.dispose(bytesAfterPut) - if (putLevel.replication > 1) { + if (blockWasSuccessfullyStored && putLevel.replication > 1) { logDebug("Putting block %s with replication took %s" .format(blockId, Utils.getUsedTimeMs(startTimeMs))) } else { @@ -997,15 +994,19 @@ private[spark] class BlockManager( * store reaches its limit and needs to free up space. * * If `data` is not put on disk, it won't be created. + * + * @return the block's new effective StorageLevel if the block existed, or None otherwise. */ def dropFromMemory( blockId: BlockId, - data: () => Either[Array[Any], ByteBuffer]): Unit = { + data: () => Either[Array[Any], ByteBuffer]): Option[StorageLevel] = { logInfo(s"Dropping block $blockId from memory") + // TODO: make lock holding a precondition of calling this method. blockInfoManager.lockForWriting(blockId) match { case None => logDebug(s"Block $blockId has already been dropped") + None case Some(info) => var blockIsUpdated = false val level = info.level @@ -1036,17 +1037,12 @@ private[spark] class BlockManager( if (info.tellMaster) { reportBlockStatus(blockId, info, status, droppedMemorySize) } - if (!level.useDisk) { - // The block is completely gone from this node; forget it so we can put() it again later. - blockInfoManager.removeBlock(blockId) - } else { - blockInfoManager.unlock(blockId) - } if (blockIsUpdated) { Option(TaskContext.get()).foreach { c => c.taskMetrics().incUpdatedBlockStatuses(Seq((blockId, status))) } } + Some(status.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 201b4f25251d6..13875204e9424 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -95,7 +95,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo val values = blockManager.dataDeserialize(blockId, bytes) putIterator(blockId, values, level, returnValues = true) } else { - tryToPut(blockId, bytes, bytes.limit, deserialized = false) + tryToPut(blockId, () => bytes, bytes.limit, deserialized = false) PutResult(bytes.limit(), Right(bytes.duplicate())) } } @@ -127,11 +127,11 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo returnValues: Boolean): PutResult = { if (level.deserialized) { val sizeEstimate = SizeEstimator.estimate(values.asInstanceOf[AnyRef]) - tryToPut(blockId, values, sizeEstimate, deserialized = true) + tryToPut(blockId, () => values, sizeEstimate, deserialized = true) PutResult(sizeEstimate, Left(values.iterator)) } else { val bytes = blockManager.dataSerialize(blockId, values.iterator) - tryToPut(blockId, bytes, bytes.limit, deserialized = false) + tryToPut(blockId, () => bytes, bytes.limit, deserialized = false) PutResult(bytes.limit(), Right(bytes.duplicate())) } } @@ -329,14 +329,6 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo blockId.asRDDId.map(_.rddId) } - private def tryToPut( - blockId: BlockId, - value: Any, - size: Long, - deserialized: Boolean): Boolean = { - tryToPut(blockId, () => value, size, deserialized) - } - /** * Try to put in a set of values, if we can free up enough space. The value should either be * an Array if deserialized is true or a ByteBuffer otherwise. Its (possibly estimated) size @@ -445,7 +437,20 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo } else { Right(entry.value.asInstanceOf[ByteBuffer].duplicate()) } - blockManager.dropFromMemory(blockId, () => data) + blockManager.dropFromMemory(blockId, () => data) match { + case Some(newEffectiveStorageLevel) => + if (newEffectiveStorageLevel.isValid) { + // The block is still present in at least one store, so release the lock + // but don't delete the block info + blockManager.releaseLock(blockId) + } else { + // The block isn't present in any store, so delete the block info so that the + // block can be stored again + blockManager.blockInfoManager.removeBlock(blockId) + } + case None => + throw new IllegalStateException("block should have existed prior to dropFromMemory") + } } } freedMemory 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 80f7ac8ed1f43..681a969ca1102 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -187,8 +187,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(master.getLocations("a3").size === 0, "master was told about a3") // Drop a1 and a2 from memory; this should be reported back to the master - store.dropFromMemory("a1", () => null: Either[Array[Any], ByteBuffer]) - store.dropFromMemory("a2", () => null: Either[Array[Any], ByteBuffer]) + store.dropFromMemoryAndReleaseLocks("a1", () => null: Either[Array[Any], ByteBuffer]) + store.dropFromMemoryAndReleaseLocks("a2", () => null: Either[Array[Any], ByteBuffer]) assert(store.getSingleAndReleaseLock("a1") === None, "a1 not removed from store") assert(store.getSingleAndReleaseLock("a2") === None, "a2 not removed from store") assert(master.getLocations("a1").size === 0, "master did not remove a1") @@ -429,8 +429,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE t2.join() t3.join() - store.dropFromMemory("a1", () => null: Either[Array[Any], ByteBuffer]) - store.dropFromMemory("a2", () => null: Either[Array[Any], ByteBuffer]) + store.dropFromMemoryAndReleaseLocks("a1", () => null: Either[Array[Any], ByteBuffer]) + store.dropFromMemoryAndReleaseLocks("a2", () => null: Either[Array[Any], ByteBuffer]) store.waitForAsyncReregister() } } @@ -1367,6 +1367,26 @@ private object BlockManagerSuite { } } + def dropFromMemoryAndReleaseLocks( + blockId: BlockId, + data: () => Either[Array[Any], ByteBuffer]): Unit = { + store.dropFromMemory(blockId, data) match { + case Some(newEffectiveStorageLevel) => + if (newEffectiveStorageLevel.isValid) { + // The block is still present in at least one store, so release the lock + // but don't delete the block info + store.releaseLock(blockId) + } else { + // The block isn't present in any store, so delete the block info so that the + // block can be stored again + store.blockInfoManager.removeBlock(blockId) + } + case None => + // TODO: it's confusing why we have to do nothing here. I think this is only needed + // for the "re-registration doesn't deadlock" test, which is slightly confusing. + } + } + private def wrapGet[T](f: BlockId => Option[T]): BlockId => Option[T] = (blockId: BlockId) => { val result = f(blockId) if (result.isDefined) { From 66202f2d7eda4abcf1b315a85f91b43d4424d93e Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 22 Feb 2016 18:31:24 -0800 Subject: [PATCH 55/81] Push CompletionIterator logic into BlockResult. --- .../scala/org/apache/spark/CacheManager.scala | 23 +++---------- .../spark/storage/BlockInfoManager.scala | 3 +- .../apache/spark/storage/BlockManager.scala | 19 +++++++---- .../spark/storage/BlockManagerSuite.scala | 34 +++++++++---------- 4 files changed, 36 insertions(+), 43 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index bbc05b7dc202c..2b456facd9439 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -19,7 +19,6 @@ package org.apache.spark import scala.collection.mutable -import org.apache.spark.executor.DataReadMethod import org.apache.spark.rdd.RDD import org.apache.spark.storage._ import org.apache.spark.util.CompletionIterator @@ -48,14 +47,8 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { val existingMetrics = context.taskMetrics().registerInputMetrics(blockResult.readMethod) existingMetrics.incBytesReadInternal(blockResult.bytes) - val iter = { - val dataIter = blockResult.data.asInstanceOf[Iterator[T]] - if (blockResult.readMethod != DataReadMethod.Network) { - CompletionIterator[T, Iterator[T]](dataIter, blockManager.releaseLock(key)) - } else { - dataIter - } - } + val iter = blockResult.data.asInstanceOf[Iterator[T]] + new InterruptibleIterator[T](context, iter) { override def next(): T = { existingMetrics.incRecordsReadInternal(1) @@ -67,9 +60,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { // If another thread already holds the lock, wait for it to finish return its results val storedValues = acquireLockForPartition[T](key) if (storedValues.isDefined) { - val iter = - CompletionIterator[T, Iterator[T]](storedValues.get, blockManager.releaseLock(key)) - return new InterruptibleIterator[T](context, iter) + return new InterruptibleIterator[T](context, storedValues.get) } // Otherwise, we have to load the partition ourselves @@ -147,13 +138,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { */ blockManager.putIterator(key, values, level, tellMaster = true, effectiveStorageLevel) blockManager.get(key) match { - case Some(v) => - val iter = v.data.asInstanceOf[Iterator[T]] - if (v.readMethod != DataReadMethod.Network) { - CompletionIterator[T, Iterator[T]](iter, blockManager.releaseLock(key)) - } else { - iter - } + case Some(v) => v.data.asInstanceOf[Iterator[T]] case None => logInfo(s"Failure to store $key") throw new BlockException(key, s"Block manager failed to return cached value for $key!") diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index b51e79860ae8d..d793dd48890ec 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -202,7 +202,8 @@ private[storage] class BlockInfoManager extends Logging { logTrace(s"Task $currentTaskAttemptId downgrading write lock for $blockId") val info = get(blockId).get require(info.writerTask == currentTaskAttemptId, - s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold") + s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold on" + + s" block $blockId") unlock(blockId) lockForReading(blockId, blocking = false) notifyAll() 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 eddc6e342d792..d385347409469 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -439,7 +439,10 @@ private[spark] class BlockManager( if (level.useMemory) { logDebug(s"Getting block $blockId from memory") val result = if (asBlockResult) { - memoryStore.getValues(blockId).map(new BlockResult(_, DataReadMethod.Memory, info.size)) + memoryStore.getValues(blockId).map { iter => + val ci = CompletionIterator[Any, Iterator[Any]](iter, releaseLock(blockId)) + new BlockResult(ci, DataReadMethod.Memory, info.size) + } } else { memoryStore.getBytes(blockId) } @@ -466,8 +469,9 @@ private[spark] class BlockManager( if (!level.useMemory) { // If the block shouldn't be stored in memory, we can just return it if (asBlockResult) { - return Some(new BlockResult(dataDeserialize(blockId, bytes), DataReadMethod.Disk, - info.size)) + val iter = CompletionIterator[Any, Iterator[Any]]( + dataDeserialize(blockId, bytes), releaseLock(blockId)) + return Some(new BlockResult(iter, DataReadMethod.Disk, info.size)) } else { return Some(bytes) } @@ -499,13 +503,15 @@ private[spark] class BlockManager( // 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)) + val ci = CompletionIterator[Any, Iterator[Any]](it, releaseLock(blockId)) + return Some(new BlockResult(ci, 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 { - return Some(new BlockResult(values, DataReadMethod.Disk, info.size)) + val ci = CompletionIterator[Any, Iterator[Any]](values, releaseLock(blockId)) + return Some(new BlockResult(ci, DataReadMethod.Disk, info.size)) } } } @@ -588,7 +594,8 @@ private[spark] class BlockManager( * Get a block from the block manager (either local or remote). * * This acquires a read lock on the block if the block was stored locally and does not acquire - * any locks if the block was fetched from a remote block manager. + * any locks if the block was fetched from a remote block manager. The read lock will + * automatically be freed once the result's `data` iterator is fully consumed. */ def get(blockId: BlockId): Option[BlockResult] = { val local = getLocal(blockId) 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 681a969ca1102..3364f863d33c8 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -447,17 +447,17 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE "list2memory", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) store.putIteratorAndReleaseLock( "list2disk", list2.iterator, StorageLevel.DISK_ONLY, tellMaster = true) - val list1Get = store.getAndReleaseLock("list1") + val list1Get = store.get("list1") assert(list1Get.isDefined, "list1 expected to be in store") assert(list1Get.get.data.size === 2) assert(list1Get.get.bytes === list1SizeEstimate) assert(list1Get.get.readMethod === DataReadMethod.Memory) - val list2MemoryGet = store.getAndReleaseLock("list2memory") + val list2MemoryGet = store.get("list2memory") assert(list2MemoryGet.isDefined, "list2memory expected to be in store") assert(list2MemoryGet.get.data.size === 3) assert(list2MemoryGet.get.bytes === list2SizeEstimate) assert(list2MemoryGet.get.readMethod === DataReadMethod.Memory) - val list2DiskGet = store.getAndReleaseLock("list2disk") + val list2DiskGet = store.get("list2disk") assert(list2DiskGet.isDefined, "list2memory expected to be in store") assert(list2DiskGet.get.data.size === 3) // We don't know the exact size of the data on disk, but it should certainly be > 0. @@ -651,19 +651,19 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE store.putIteratorAndReleaseLock( "list3", list3.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(store.getAndReleaseLock("list2").isDefined, "list2 was not in store") - assert(store.getAndReleaseLock("list2").get.data.size === 2) + assert(store.get("list2").get.data.size === 2) assert(store.getAndReleaseLock("list3").isDefined, "list3 was not in store") - assert(store.getAndReleaseLock("list3").get.data.size === 2) + assert(store.get("list3").get.data.size === 2) assert(store.getAndReleaseLock("list1") === None, "list1 was in store") assert(store.getAndReleaseLock("list2").isDefined, "list2 was not in store") - assert(store.getAndReleaseLock("list2").get.data.size === 2) + assert(store.get("list2").get.data.size === 2) // At this point list2 was gotten last, so LRU will getSingle rid of list3 store.putIteratorAndReleaseLock( "list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(store.getAndReleaseLock("list1").isDefined, "list1 was not in store") - assert(store.getAndReleaseLock("list1").get.data.size === 2) + assert(store.get("list1").get.data.size === 2) assert(store.getAndReleaseLock("list2").isDefined, "list2 was not in store") - assert(store.getAndReleaseLock("list2").get.data.size === 2) + assert(store.get("list2").get.data.size === 2) assert(store.getAndReleaseLock("list3") === None, "list1 was in store") } @@ -685,27 +685,27 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val listSize = SizeEstimator.estimate(listForSizeEstimate) // At this point LRU should not kick in because list3 is only on disk assert(store.getAndReleaseLock("list1").isDefined, "list1 was not in store") - assert(store.getAndReleaseLock("list1").get.data.size === 2) + assert(store.get("list1").get.data.size === 2) assert(store.getAndReleaseLock("list2").isDefined, "list2 was not in store") - assert(store.getAndReleaseLock("list2").get.data.size === 2) + assert(store.get("list2").get.data.size === 2) assert(store.getAndReleaseLock("list3").isDefined, "list3 was not in store") - assert(store.getAndReleaseLock("list3").get.data.size === 2) + assert(store.get("list3").get.data.size === 2) assert(store.getAndReleaseLock("list1").isDefined, "list1 was not in store") - assert(store.getAndReleaseLock("list1").get.data.size === 2) + assert(store.get("list1").get.data.size === 2) assert(store.getAndReleaseLock("list2").isDefined, "list2 was not in store") - assert(store.getAndReleaseLock("list2").get.data.size === 2) + assert(store.get("list2").get.data.size === 2) assert(store.getAndReleaseLock("list3").isDefined, "list3 was not in store") - assert(store.getAndReleaseLock("list3").get.data.size === 2) + 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.putIteratorAndReleaseLock( "list4", list4.iterator, StorageLevel.MEMORY_AND_DISK_SER, tellMaster = true) assert(store.getAndReleaseLock("list1") === None, "list1 was in store") assert(store.getAndReleaseLock("list2").isDefined, "list2 was not in store") - assert(store.getAndReleaseLock("list2").get.data.size === 2) + assert(store.get("list2").get.data.size === 2) assert(store.getAndReleaseLock("list3").isDefined, "list3 was not in store") - assert(store.getAndReleaseLock("list3").get.data.size === 2) + assert(store.get("list3").get.data.size === 2) assert(store.getAndReleaseLock("list4").isDefined, "list4 was not in store") - assert(store.getAndReleaseLock("list4").get.data.size === 2) + assert(store.get("list4").get.data.size === 2) } test("negative byte values in ByteBufferInputStream") { From 4f620a46d5bbd9d9a3817669cc4a258a46d91835 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 22 Feb 2016 22:35:55 -0800 Subject: [PATCH 56/81] Add scaladoc to BlockManagerManagedBuffer. --- .../apache/spark/storage/BlockManagerManagedBuffer.scala | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerManagedBuffer.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerManagedBuffer.scala index 38d593962171e..5886b9c00b557 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerManagedBuffer.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerManagedBuffer.scala @@ -21,7 +21,14 @@ import java.nio.ByteBuffer import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} - +/** + * This [[ManagedBuffer]] wraps a [[ByteBuffer]] which was retrieved from the [[BlockManager]] + * so that the corresponding block's read lock can be released once this buffer's references + * are released. + * + * This is effectively a wrapper / bridge to connect the BlockManager's notion of read locks + * to the network layer's notion of retain / release counts. + */ private[storage] class BlockManagerManagedBuffer( blockManager: BlockManager, blockId: BlockId, From 854784139891dc71a63399f252247b5b03ebba64 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 22 Feb 2016 22:42:14 -0800 Subject: [PATCH 57/81] Document non-blocking tryLock in MemoryStore. --- .../main/scala/org/apache/spark/storage/MemoryStore.scala | 5 ++++- 1 file changed, 4 insertions(+), 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 13875204e9424..2d4de2ffdf3a2 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -416,7 +416,10 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo val pair = iterator.next() val blockId = pair.getKey if (blockIsEvictable(blockId)) { - if (blockManager.blockInfoManager.lockForWriting(blockId, false).isDefined) { + // We don't want to evict blocks which are currently being read, so we need to obtain + // an exclusive write lock on blocks which are candidates for eviction. We perform a + // non-blocking "tryLock" here in order to ignore blocks which are locked for reading: + if (blockManager.blockInfoManager.lockForWriting(blockId, blocking = false).isDefined) { selectedBlocks += blockId freedMemory += pair.getValue.size } From 99c460c6ba326a832e0509b9e6b05b2da447ed3b Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 22 Feb 2016 22:43:17 -0800 Subject: [PATCH 58/81] Fix comment typo in BlockDataManager. --- .../main/scala/org/apache/spark/network/BlockDataManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala b/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala index 9aba9544266eb..cc5e851c29b32 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala @@ -33,7 +33,7 @@ trait BlockDataManager { * Put the block locally, using the given storage level. * * Returns true if the block was stored and false if the put operation failed or the block - * aready existed. + * already existed. */ def putBlockData(blockId: BlockId, data: ManagedBuffer, level: StorageLevel): Boolean From c94984e21f7b6d3f8c823bc5a1d58b42a030fb90 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 22 Feb 2016 23:03:14 -0800 Subject: [PATCH 59/81] Check invariants whenever BlockInfo is mutated. --- .../spark/storage/BlockInfoManager.scala | 50 ++++++++++++++----- 1 file changed, 37 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index d793dd48890ec..bb6ce8a1b3d91 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -43,40 +43,62 @@ private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolea /** * The size of the block (in bytes) */ - var size: Long = 0 + def size: Long = _size + def size_=(s: Long): Unit = { + _size = s + checkInvariants() + } + private[this] var _size: Long = 0 /** * The number of times that this block has been locked for reading. */ - var readerCount: Int = 0 + def readerCount: Int = _readerCount + def readerCount_=(c: Int): Unit = { + _readerCount = c + checkInvariants() + } + private[this] var _readerCount: Int = 0 /** * The task attempt id of the task which currently holds the write lock for this block, or -1 * if this block is not locked for writing. */ - var writerTask: Long = -1 - - // Invariants: - // (writerTask != -1) implies (readerCount == 0) - // (readerCount != 0) implies (writerTask == -1) - // TODO: add assertions around every method + def writerTask: Long = _writerTask + def writerTask_=(t: Long): Unit = { + _writerTask = t + checkInvariants() + } + private[this] var _writerTask: Long = 0 /** * True if this block has been removed from the BlockManager and false otherwise. * This field is used to communicate block deletion to blocked readers / writers (see its usage * in [[BlockInfoManager]]). */ - var removed: Boolean = false + def removed: Boolean = _removed + def removed_=(r: Boolean): Unit = { + _removed = r + checkInvariants() + } + private[this] var _removed: Boolean = false + + private def checkInvariants(): Unit = { + // A block's reader count must be non-negative: + assert(_readerCount >= 0) + // A block is either locked for reading or for writing, but not for both at the same time: + assert(!(_readerCount != 0 && _writerTask != -1)) + // If a block is removed then it is not locked: + assert(!_removed || (_readerCount == 0 && _writerTask == -1)) + } - // TODO: Add timestamps on lock acquisitions + checkInvariants() } -// In debugging mode, check that locks haven't been held for too long. -// Every few minutes, dump debug info. /** * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking. * - * The locking interface exposed by this class is readers-writers lock. Every lock acquisition is + * The locking interface exposed by this class is readers-writer lock. Every lock acquisition is * automatically associated with a running task and locks are automatically released upon task * completion or failure. * @@ -336,6 +358,8 @@ private[storage] class BlockInfoManager extends Logging { s"Task $currentTaskAttemptId called remove() on block $blockId without a write lock") } else { infos.remove(blockId) + blockInfo.readerCount = 0 + blockInfo.writerTask = -1 blockInfo.removed = true } case None => From ac2b73f7a4d72bc110d44e2554ca95a52977d69b Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 22 Feb 2016 23:11:32 -0800 Subject: [PATCH 60/81] Extract magic writerTask values into constants. --- .../spark/storage/BlockInfoManager.scala | 44 ++++++++++++------- .../spark/storage/BlockInfoManagerSuite.scala | 6 +-- 2 files changed, 31 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index bb6ce8a1b3d91..df5eb32a83984 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -61,8 +61,9 @@ private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolea private[this] var _readerCount: Int = 0 /** - * The task attempt id of the task which currently holds the write lock for this block, or -1 - * if this block is not locked for writing. + * The task attempt id of the task which currently holds the write lock for this block, or + * [[BlockInfo.NON_TASK_WRITER]] if the write lock is held by non-task code, or + * [[BlockInfo.NO_WRITER]] if this block is not locked for writing. */ def writerTask: Long = _writerTask def writerTask_=(t: Long): Unit = { @@ -87,14 +88,28 @@ private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolea // A block's reader count must be non-negative: assert(_readerCount >= 0) // A block is either locked for reading or for writing, but not for both at the same time: - assert(!(_readerCount != 0 && _writerTask != -1)) + assert(!(_readerCount != 0 && _writerTask != BlockInfo.NO_WRITER)) // If a block is removed then it is not locked: - assert(!_removed || (_readerCount == 0 && _writerTask == -1)) + assert(!_removed || (_readerCount == 0 && _writerTask == BlockInfo.NO_WRITER)) } checkInvariants() } +private[storage] object BlockInfo { + + /** + * Special task attempt id constant used to mark a block's write lock as being unlocked. + */ + val NO_WRITER: Long = -1 + + /** + * Special task attempt id constant used to mark a block's write lock as being held by + * a non-task thread (e.g. by a driver thread or by unit test code). + */ + val NON_TASK_WRITER: Long = -1024 +} + /** * Component of the [[BlockManager]] which tracks metadata for blocks and manages block locking. * @@ -139,14 +154,11 @@ private[storage] class BlockInfoManager extends Logging { // ---------------------------------------------------------------------------------------------- /** - * Returns the current tasks's task attempt id (which uniquely identifies the task), or -1024 - * if called outside of a task (-1024 was chosen because it's different than the -1 which is used - * in [[BlockInfo.writerTask]] to denote the absence of a write lock). + * Returns the current task's task attempt id (which uniquely identifies the task), or + * [[BlockInfo.NON_TASK_WRITER]] if called by a non-task thread. */ private def currentTaskAttemptId: TaskAttemptId = { - // TODO(josh): assert that this only happens on the driver? - // What about block transfer / getRemote()? - Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1024L) + Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(BlockInfo.NON_TASK_WRITER) } /** @@ -166,7 +178,7 @@ private[storage] class BlockInfoManager extends Logging { blocking: Boolean = true): Option[BlockInfo] = synchronized { logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId") infos.get(blockId).map { info => - while (info.writerTask != -1) { + while (info.writerTask != BlockInfo.NO_WRITER) { if (info.removed) return None if (blocking) wait() else return None } @@ -197,7 +209,7 @@ private[storage] class BlockInfoManager extends Logging { logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId") infos.get(blockId).map { info => if (info.writerTask != currentTaskAttemptId) { - while (info.writerTask != -1 || info.readerCount != 0) { + while (info.writerTask != BlockInfo.NO_WRITER || info.readerCount != 0) { if (info.removed) return None if (blocking) wait() else return None } @@ -239,8 +251,8 @@ private[storage] class BlockInfoManager extends Logging { val info = get(blockId).getOrElse { throw new IllegalStateException(s"Block $blockId not found") } - if (info.writerTask != -1) { - info.writerTask = -1 + if (info.writerTask != BlockInfo.NO_WRITER) { + info.writerTask = BlockInfo.NO_WRITER writeLocksByTask.removeBinding(currentTaskAttemptId, blockId) } else { assert(info.readerCount > 0, s"Block $blockId is not locked for reading") @@ -290,7 +302,7 @@ private[storage] class BlockInfoManager extends Logging { for (blockId <- locks) { infos.get(blockId).foreach { info => assert(info.writerTask == taskAttemptId) - info.writerTask = -1 + info.writerTask = BlockInfo.NO_WRITER } blocksWithReleasedLocks += blockId } @@ -359,7 +371,7 @@ private[storage] class BlockInfoManager extends Logging { } else { infos.remove(blockId) blockInfo.readerCount = 0 - blockInfo.writerTask = -1 + blockInfo.writerTask = BlockInfo.NO_WRITER blockInfo.removed = true } case None => diff --git a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala index c9300971cbb0a..d16ba63988fc1 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala @@ -83,7 +83,7 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { assert(blockInfo.writerTask === 1) blockInfoManager.unlock("block") assert(blockInfo.readerCount === 0) - assert(blockInfo.writerTask === -1) + assert(blockInfo.writerTask === BlockInfo.NO_WRITER) } assert(blockInfoManager.size === 1) assert(blockInfoManager.getNumberOfMapEntries === 1) @@ -96,7 +96,7 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { assert(blockInfoManager.lockForReading("block").isDefined) assert(blockInfoManager.lockForReading("block").isDefined) assert(blockInfoManager.get("block").get.readerCount === 2) - assert(blockInfoManager.get("block").get.writerTask === -1) + assert(blockInfoManager.get("block").get.writerTask === BlockInfo.NO_WRITER) blockInfoManager.unlock("block") assert(blockInfoManager.get("block").get.readerCount === 1) blockInfoManager.unlock("block") @@ -140,7 +140,7 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { assert(blockInfoManager.lockForReading("block").isDefined) } assert(blockInfoManager.get("block").get.readerCount === 2) - assert(blockInfoManager.get("block").get.writerTask === -1) + assert(blockInfoManager.get("block").get.writerTask === BlockInfo.NO_WRITER) } test("write lock will block readers") { From 39b118542ee27aa11dc892a12008d14ee4065116 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 22 Feb 2016 23:24:28 -0800 Subject: [PATCH 61/81] Numerous documentation updates in BlockInfoManager. --- .../spark/storage/BlockInfoManager.scala | 19 +++++++++++++++---- 1 file changed, 15 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index df5eb32a83984..2261a1326418c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -164,6 +164,12 @@ private[storage] class BlockInfoManager extends Logging { /** * Lock a block for reading and return its metadata. * + * If another task has already locked this block for reading, then the read lock will be + * immediately granted to the calling task and its lock count will be incremented. + * + * If another task has locked this block for reading, then this call will block until the write + * lock is released or will return immediately if `blocking = false`. + * * A single task can lock a block multiple times for reading, in which case each lock will need * to be released separately. * @@ -193,6 +199,9 @@ private[storage] class BlockInfoManager extends Logging { /** * Lock a block for writing and return its metadata. * + * If another task has already locked this block for either reading or writing, then this call + * will block until the other locks are released or will return immediately if `blocking = false`. + * * If this is called by a task which already holds the block's exclusive write lock, then this * will return success but will not further increment any lock counts (so both write-lock * acquisitions will be freed by the same [[unlock()]] or [[downgradeLock()]] call. @@ -223,9 +232,10 @@ private[storage] class BlockInfoManager extends Logging { } /** - * Get a block's metadata without acquiring any locks. + * Get a block's metadata without acquiring any locks. This method is only exposed for use by + * [[BlockManager.getStatus()]] and should not be called by other code outside of this class. */ - def get(blockId: BlockId): Option[BlockInfo] = synchronized { + private[storage] def get(blockId: BlockId): Option[BlockInfo] = synchronized { infos.get(blockId) } @@ -266,7 +276,8 @@ private[storage] class BlockInfoManager extends Logging { } /** - * Atomically create metadata for a non-existent block. + * Atomically create metadata for a block and acquire a write lock for it, if it doesn't already + * exist. * * @param blockId the block id. * @param newBlockInfo the block info for the new block. @@ -357,7 +368,7 @@ private[storage] class BlockInfoManager extends Logging { } /** - * Removes the given block and automatically drops all locks on it. + * Removes the given block and releases the write lock on it. * * This can only be called while holding a write lock on the given block. */ From 65020473c8abce75f984e2a13541e35ee1b11069 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 22 Feb 2016 23:25:35 -0800 Subject: [PATCH 62/81] Add defensive notifyAll() to BlockInfoManager.clear(). --- .../scala/org/apache/spark/storage/BlockInfoManager.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index 2261a1326418c..d760c7685ac68 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -396,9 +396,15 @@ private[storage] class BlockInfoManager extends Logging { * Delete all state. Called during shutdown. */ def clear(): Unit = synchronized { + infos.valuesIterator.foreach { blockInfo => + blockInfo.readerCount = 0 + blockInfo.writerTask = BlockInfo.NO_WRITER + blockInfo.removed = true + } infos.clear() readLocksByTask.invalidateAll() writeLocksByTask.clear() + notifyAll() } } From 1d903fffddfda5103c1548894c6aa6d6c08d1456 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 22 Feb 2016 23:26:33 -0800 Subject: [PATCH 63/81] Remove now-redundant info.removed checks in loop body. --- .../main/scala/org/apache/spark/storage/BlockInfoManager.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index d760c7685ac68..4f24217d0a3d2 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -185,7 +185,6 @@ private[storage] class BlockInfoManager extends Logging { logTrace(s"Task $currentTaskAttemptId trying to acquire read lock for $blockId") infos.get(blockId).map { info => while (info.writerTask != BlockInfo.NO_WRITER) { - if (info.removed) return None if (blocking) wait() else return None } if (info.removed) return None @@ -219,7 +218,6 @@ private[storage] class BlockInfoManager extends Logging { infos.get(blockId).map { info => if (info.writerTask != currentTaskAttemptId) { while (info.writerTask != BlockInfo.NO_WRITER || info.readerCount != 0) { - if (info.removed) return None if (blocking) wait() else return None } if (info.removed) return None From 24dbc3d7f7b5b9b71728c1c8f6ea00b2505ab41e Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 22 Feb 2016 23:27:22 -0800 Subject: [PATCH 64/81] Clean up BlockInfoManager.entries() typos. --- .../scala/org/apache/spark/storage/BlockInfoManager.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index 4f24217d0a3d2..c8f7a5bdb1f0a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -358,11 +358,11 @@ private[storage] class BlockInfoManager extends Logging { /** * Returns an iterator over a snapshot of all blocks' metadata. Note that the individual entries - * is this iterator are mutable and thus may reflect blocks that are deleted while the iterator + * in this iterator are mutable and thus may reflect blocks that are deleted while the iterator * is being traversed. */ def entries: Iterator[(BlockId, BlockInfo)] = synchronized { - infos.iterator.toArray.toIterator + infos.toArray.toIterator } /** From 745c1f9f2fb8a7f0e1af9f63a9013cc16289450e Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 22 Feb 2016 23:28:14 -0800 Subject: [PATCH 65/81] unlockAllLocksForTask => releaseAllLocksForTask --- .../main/scala/org/apache/spark/storage/BlockInfoManager.scala | 2 +- core/src/main/scala/org/apache/spark/storage/BlockManager.scala | 2 +- .../scala/org/apache/spark/storage/BlockInfoManagerSuite.scala | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index c8f7a5bdb1f0a..369c6479dfc1f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -304,7 +304,7 @@ private[storage] class BlockInfoManager extends Logging { * * @return the ids of blocks whose pins were released */ - def unlockAllLocksForTask(taskAttemptId: TaskAttemptId): Seq[BlockId] = { + def releaseAllLocksForTask(taskAttemptId: TaskAttemptId): Seq[BlockId] = { val blocksWithReleasedLocks = mutable.ArrayBuffer[BlockId]() synchronized { writeLocksByTask.remove(taskAttemptId).foreach { locks => 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 d385347409469..e6e090a32eca2 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -631,7 +631,7 @@ private[spark] class BlockManager( * @return the blocks whose locks were released. */ def releaseAllLocksForTask(taskAttemptId: Long): Seq[BlockId] = { - blockInfoManager.unlockAllLocksForTask(taskAttemptId) + blockInfoManager.releaseAllLocksForTask(taskAttemptId) } /** diff --git a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala index d16ba63988fc1..d42f1a3db60e7 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala @@ -252,7 +252,7 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) } assert(blockInfoManager.getNumberOfMapEntries === 3) - blockInfoManager.unlockAllLocksForTask(0) + blockInfoManager.releaseAllLocksForTask(0) assert(blockInfoManager.getNumberOfMapEntries === 1) } } From 5cfbbdb0b35e101469bdc02de34b828e155d385c Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 22 Feb 2016 23:32:40 -0800 Subject: [PATCH 66/81] Address style nit in BlockManager.getMatchingBlockIds(). --- .../main/scala/org/apache/spark/storage/BlockManager.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 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 e6e090a32eca2..bdb02ffe872ff 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -316,11 +316,12 @@ private[spark] class BlockManager( * may not know of). */ def getMatchingBlockIds(filter: BlockId => Boolean): Seq[BlockId] = { - val matches = - (blockInfoManager.entries.map(_._1) ++ diskBlockManager.getAllBlocks()).filter(filter) // The `toArray` is necessary here in order to force the list to be materialized so that we // don't try to serialize a lazy iterator when responding to client requests. - matches.toArray.toSeq + (blockInfoManager.entries.map(_._1) ++ diskBlockManager.getAllBlocks()) + .filter(filter) + .toArray + .toSeq } /** From 9427576e240645dd20f3f65af335eb25a00cff3e Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 22 Feb 2016 23:33:19 -0800 Subject: [PATCH 67/81] Address confusing "local lock" comment. --- core/src/main/scala/org/apache/spark/storage/BlockManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 bdb02ffe872ff..713718d65d828 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -526,7 +526,7 @@ private[spark] class BlockManager( /** * Get block from remote block managers. * - * This does not acquire a local lock on this block. + * This does not acquire a lock on this block in this JVM. */ def getRemote(blockId: BlockId): Option[BlockResult] = { logDebug(s"Getting remote block $blockId") From 3d377b5c1278beba129a9a89604c997cf2933886 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 23 Feb 2016 00:15:36 -0800 Subject: [PATCH 68/81] Remove unnecessary notifyAll() in downgradeLock(). --- .../main/scala/org/apache/spark/storage/BlockInfoManager.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index 369c6479dfc1f..188c914fca3d0 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -248,7 +248,6 @@ private[storage] class BlockInfoManager extends Logging { s" block $blockId") unlock(blockId) lockForReading(blockId, blocking = false) - notifyAll() } /** From 07e0e377d87762cffde110dc93c51b95c0bc7c3c Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 23 Feb 2016 00:50:25 -0800 Subject: [PATCH 69/81] Deduplicate code in TorrentBroadcast and check put() return values. --- .../spark/broadcast/TorrentBroadcast.scala | 78 +++++++++++-------- 1 file changed, 44 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index d5083fa15022b..267ca1fb3c9a4 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -27,7 +27,7 @@ import scala.util.Random import org.apache.spark._ import org.apache.spark.io.CompressionCodec import org.apache.spark.serializer.Serializer -import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} +import org.apache.spark.storage.{BlockId, BroadcastBlockId, StorageLevel} import org.apache.spark.util.{ByteBufferInputStream, Utils} import org.apache.spark.util.io.ByteArrayChunkOutputStream @@ -97,15 +97,22 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) private def writeBlocks(value: T): Int = { // Store a copy of the broadcast variable in the driver so that tasks run on the driver // do not create a duplicate copy of the broadcast variable's value. + val storageLevel = StorageLevel.MEMORY_AND_DISK val blockManager = SparkEnv.get.blockManager - blockManager.putSingle(broadcastId, value, StorageLevel.MEMORY_AND_DISK, tellMaster = false) - blockManager.releaseLock(broadcastId) + if (blockManager.putSingle(broadcastId, value, storageLevel, tellMaster = false)) { + blockManager.releaseLock(broadcastId) + } else { + throw new SparkException(s"Failed to store $broadcastId in BlockManager") + } val blocks = TorrentBroadcast.blockifyObject(value, blockSize, SparkEnv.get.serializer, compressionCodec) blocks.zipWithIndex.foreach { case (block, i) => val pieceId = BroadcastBlockId(id, "piece" + i) - blockManager.putBytes(pieceId, block, StorageLevel.MEMORY_AND_DISK_SER, tellMaster = true) - blockManager.releaseLock(pieceId) + if (blockManager.putBytes(pieceId, block, storageLevel, tellMaster = true)) { + blockManager.releaseLock(pieceId) + } else { + throw new SparkException(s"Failed to store $pieceId of $broadcastId in local BlockManager") + } } blocks.length } @@ -127,23 +134,18 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) def getRemote: Option[ByteBuffer] = bm.getRemoteBytes(pieceId).map { block => // If we found the block from remote executors/driver's BlockManager, put the block // in this executor's BlockManager. - bm.putBytes(pieceId, block, StorageLevel.MEMORY_AND_DISK_SER, tellMaster = true) + if (!bm.putBytes(pieceId, block, StorageLevel.MEMORY_AND_DISK_SER, tellMaster = true)) { + throw new SparkException( + s"Failed to store $pieceId of $broadcastId in local BlockManager") + } block } val block: ByteBuffer = getLocal.orElse(getRemote).getOrElse( throw new SparkException(s"Failed to get $pieceId of $broadcastId")) + // At this point we are guaranteed to hold a read lock, since we either got the block locally + // or stored the remotely-fetched block and automatically downgraded the write lock. blocks(pid) = block - Option(TaskContext.get()) match { - case Some(taskContext) => - taskContext.addTaskCompletionListener(_ => bm.releaseLock(pieceId)) - case None => - // This should only happen on the driver, where broadcast variables may be accessed - // outside of running tasks (e.g. when computing rdd.partitions()). In order to allow - // broadcast variables to be garbage collected we need to free the reference here, which - // is slightly unsafe but is technically okay because broadcast variables aren't stored - // off-heap. - bm.releaseLock(pieceId) - } + releaseLock(pieceId) } blocks } @@ -175,17 +177,7 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) val blockManager = SparkEnv.get.blockManager blockManager.getLocal(broadcastId).map(_.data.next()) match { case Some(x) => - Option(TaskContext.get()) match { - case Some(taskContext) => - taskContext.addTaskCompletionListener(_ => blockManager.releaseLock(broadcastId)) - case None => - // This should only happen on the driver, where broadcast variables may be accessed - // outside of running tasks (e.g. when computing rdd.partitions()). In order to allow - // broadcast variables to be garbage collected we need to free the reference here - // which is slightly unsafe but is technically okay because broadcast variables aren't - // stored off-heap. - blockManager.releaseLock(broadcastId) - } + releaseLock(broadcastId) x.asInstanceOf[T] case None => @@ -198,18 +190,36 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) blocks, SparkEnv.get.serializer, compressionCodec) // Store the merged copy in BlockManager so other tasks on this executor don't // need to re-fetch it. - blockManager.putSingle(broadcastId, obj, StorageLevel.MEMORY_AND_DISK, tellMaster = false) - Option(TaskContext.get()) match { - case Some(taskContext) => - taskContext.addTaskCompletionListener(_ => blockManager.releaseLock(broadcastId)) - case None => - blockManager.releaseLock(broadcastId) + val storageLevel = StorageLevel.MEMORY_AND_DISK + if (blockManager.putSingle(broadcastId, obj, storageLevel, tellMaster = false)) { + releaseLock(broadcastId) + } else { + throw new SparkException(s"Failed to store $broadcastId in BlockManager") } obj } } } + /** + * If running in a task, register the given block's locks for release upon task completion. + * Otherwise, if not running in a task then immediately release the lock. + */ + private def releaseLock(blockId: BlockId): Unit = { + val blockManager = SparkEnv.get.blockManager + Option(TaskContext.get()) match { + case Some(taskContext) => + taskContext.addTaskCompletionListener(_ => blockManager.releaseLock(blockId)) + case None => + // This should only happen on the driver, where broadcast variables may be accessed + // outside of running tasks (e.g. when computing rdd.partitions()). In order to allow + // broadcast variables to be garbage collected we need to free the reference here + // which is slightly unsafe but is technically okay because broadcast variables aren't + // stored off-heap. + blockManager.releaseLock(blockId) + } + } + } From 697eba213c11bc92724cfd891f5b86acd60dfd4f Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 23 Feb 2016 13:52:37 -0800 Subject: [PATCH 70/81] Torrent broadcast pieces need to be stored in serialized form. --- .../scala/org/apache/spark/broadcast/TorrentBroadcast.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 267ca1fb3c9a4..c08f87a8b45c1 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -95,11 +95,11 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) * @return number of blocks this broadcast variable is divided into */ private def writeBlocks(value: T): Int = { + import StorageLevel._ // Store a copy of the broadcast variable in the driver so that tasks run on the driver // do not create a duplicate copy of the broadcast variable's value. - val storageLevel = StorageLevel.MEMORY_AND_DISK val blockManager = SparkEnv.get.blockManager - if (blockManager.putSingle(broadcastId, value, storageLevel, tellMaster = false)) { + if (blockManager.putSingle(broadcastId, value, MEMORY_AND_DISK, tellMaster = false)) { blockManager.releaseLock(broadcastId) } else { throw new SparkException(s"Failed to store $broadcastId in BlockManager") @@ -108,7 +108,7 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) TorrentBroadcast.blockifyObject(value, blockSize, SparkEnv.get.serializer, compressionCodec) blocks.zipWithIndex.foreach { case (block, i) => val pieceId = BroadcastBlockId(id, "piece" + i) - if (blockManager.putBytes(pieceId, block, storageLevel, tellMaster = true)) { + if (blockManager.putBytes(pieceId, block, MEMORY_AND_DISK_SER, tellMaster = true)) { blockManager.releaseLock(pieceId) } else { throw new SparkException(s"Failed to store $pieceId of $broadcastId in local BlockManager") From f5f089dcf2b12ef25944541614ca2b4bfe52e18d Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 23 Feb 2016 15:22:36 -0800 Subject: [PATCH 71/81] Don't acquire lock in dropFromMemory(). --- .../spark/storage/BlockInfoManager.scala | 28 ++++++- .../apache/spark/storage/BlockManager.scala | 79 +++++++++---------- .../apache/spark/storage/MemoryStore.scala | 22 +++--- .../spark/storage/BlockInfoManagerSuite.scala | 27 ++++++- .../spark/storage/BlockManagerSuite.scala | 37 +++++---- 5 files changed, 114 insertions(+), 79 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index 188c914fca3d0..deeecbe953cb3 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -26,7 +26,7 @@ import scala.collection.mutable import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} import com.google.common.collect.ConcurrentHashMultiset -import org.apache.spark.{Logging, TaskContext} +import org.apache.spark.{Logging, SparkException, TaskContext} /** @@ -202,8 +202,7 @@ private[storage] class BlockInfoManager extends Logging { * will block until the other locks are released or will return immediately if `blocking = false`. * * If this is called by a task which already holds the block's exclusive write lock, then this - * will return success but will not further increment any lock counts (so both write-lock - * acquisitions will be freed by the same [[unlock()]] or [[downgradeLock()]] call. + * method will throw an exception. * * @param blockId the block to lock. * @param blocking if true (default), this call will block until the lock is acquired. If false, @@ -216,7 +215,10 @@ private[storage] class BlockInfoManager extends Logging { blocking: Boolean = true): Option[BlockInfo] = synchronized { logTrace(s"Task $currentTaskAttemptId trying to acquire write lock for $blockId") infos.get(blockId).map { info => - if (info.writerTask != currentTaskAttemptId) { + if (info.writerTask == currentTaskAttemptId) { + throw new IllegalStateException( + s"Task $currentTaskAttemptId has already locked $blockId for writing") + } else { while (info.writerTask != BlockInfo.NO_WRITER || info.readerCount != 0) { if (blocking) wait() else return None } @@ -229,6 +231,24 @@ private[storage] class BlockInfoManager extends Logging { } } + /** + * Throws an exception if the current task does not hold a write lock on the given block. + * Otherwise, returns the block's BlockInfo. + */ + def assertBlockIsLockedForWriting(blockId: BlockId): BlockInfo = synchronized { + infos.get(blockId) match { + case Some(info) => + if (info.writerTask != currentTaskAttemptId) { + throw new SparkException( + s"Task $currentTaskAttemptId has not locked block $blockId for writing") + } else { + info + } + case None => + throw new SparkException(s"Block $blockId does not exist") + } + } + /** * Get a block's metadata without acquiring any locks. This method is only exposed for use by * [[BlockManager.getStatus()]] and should not be called by other code outside of this class. 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 713718d65d828..c2400907b0687 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -1003,55 +1003,50 @@ private[spark] class BlockManager( * * If `data` is not put on disk, it won't be created. * - * @return the block's new effective StorageLevel if the block existed, or None otherwise. + * The caller of this method must hold a write lock on the block before calling this method. + * + * @return the block's new effective StorageLevel. */ def dropFromMemory( blockId: BlockId, - data: () => Either[Array[Any], ByteBuffer]): Option[StorageLevel] = { - + data: () => Either[Array[Any], ByteBuffer]): StorageLevel = { logInfo(s"Dropping block $blockId from memory") - // TODO: make lock holding a precondition of calling this method. - blockInfoManager.lockForWriting(blockId) match { - case None => - logDebug(s"Block $blockId has already been dropped") - None - case Some(info) => - var blockIsUpdated = false - val level = info.level - - // Drop to disk, if storage level requires - if (level.useDisk && !diskStore.contains(blockId)) { - logInfo(s"Writing block $blockId to disk") - data() match { - case Left(elements) => - diskStore.putArray(blockId, elements, level, returnValues = false) - case Right(bytes) => - diskStore.putBytes(blockId, bytes, level) - } - blockIsUpdated = true - } + val info = blockInfoManager.assertBlockIsLockedForWriting(blockId) + var blockIsUpdated = false + val level = info.level + + // Drop to disk, if storage level requires + if (level.useDisk && !diskStore.contains(blockId)) { + logInfo(s"Writing block $blockId to disk") + data() match { + case Left(elements) => + diskStore.putArray(blockId, elements, level, returnValues = false) + case Right(bytes) => + diskStore.putBytes(blockId, bytes, level) + } + blockIsUpdated = true + } - // Actually drop from memory store - val droppedMemorySize = - if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L - val blockIsRemoved = memoryStore.remove(blockId) - if (blockIsRemoved) { - blockIsUpdated = true - } else { - logWarning(s"Block $blockId could not be dropped from memory as it does not exist") - } + // Actually drop from memory store + val droppedMemorySize = + if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L + val blockIsRemoved = memoryStore.remove(blockId) + if (blockIsRemoved) { + blockIsUpdated = true + } else { + logWarning(s"Block $blockId could not be dropped from memory as it does not exist") + } - val status = getCurrentBlockStatus(blockId, info) - if (info.tellMaster) { - reportBlockStatus(blockId, info, status, droppedMemorySize) - } - if (blockIsUpdated) { - Option(TaskContext.get()).foreach { c => - c.taskMetrics().incUpdatedBlockStatuses(Seq((blockId, status))) - } - } - Some(status.storageLevel) + val status = getCurrentBlockStatus(blockId, info) + if (info.tellMaster) { + reportBlockStatus(blockId, info, status, droppedMemorySize) + } + if (blockIsUpdated) { + Option(TaskContext.get()).foreach { c => + c.taskMetrics().incUpdatedBlockStatuses(Seq((blockId, status))) + } } + status.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 2d4de2ffdf3a2..2f16c8f3d8bad 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -440,19 +440,15 @@ private[spark] class MemoryStore(blockManager: BlockManager, memoryManager: Memo } else { Right(entry.value.asInstanceOf[ByteBuffer].duplicate()) } - blockManager.dropFromMemory(blockId, () => data) match { - case Some(newEffectiveStorageLevel) => - if (newEffectiveStorageLevel.isValid) { - // The block is still present in at least one store, so release the lock - // but don't delete the block info - blockManager.releaseLock(blockId) - } else { - // The block isn't present in any store, so delete the block info so that the - // block can be stored again - blockManager.blockInfoManager.removeBlock(blockId) - } - case None => - throw new IllegalStateException("block should have existed prior to dropFromMemory") + val newEffectiveStorageLevel = blockManager.dropFromMemory(blockId, () => data) + if (newEffectiveStorageLevel.isValid) { + // The block is still present in at least one store, so release the lock + // but don't delete the block info + blockManager.releaseLock(blockId) + } else { + // The block isn't present in any store, so delete the block info so that the + // block can be stored again + blockManager.blockInfoManager.removeBlock(blockId) } } } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala index d42f1a3db60e7..71a6568f3d5bb 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala @@ -23,7 +23,7 @@ import scala.language.implicitConversions import org.scalatest.BeforeAndAfterEach import org.scalatest.time.SpanSugar._ -import org.apache.spark.{SparkFunSuite, TaskContext, TaskContextImpl} +import org.apache.spark.{SparkException, SparkFunSuite, TaskContext, TaskContextImpl} class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { @@ -131,6 +131,31 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { } } + test("cannot call lockForWriting while already holding a write lock") { + withTaskId(0) { + assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) + blockInfoManager.unlock("block") + } + withTaskId(1) { + assert(blockInfoManager.lockForWriting("block").isDefined) + intercept[IllegalStateException] { + blockInfoManager.lockForWriting("block") + } + blockInfoManager.assertBlockIsLockedForWriting("block") + } + } + + test("assertBlockIsLockedForWriting throws exception if block is not locked") { + intercept[SparkException] { + blockInfoManager.assertBlockIsLockedForWriting("block") + } + withTaskId(BlockInfo.NON_TASK_WRITER) { + intercept[SparkException] { + blockInfoManager.assertBlockIsLockedForWriting("block") + } + } + } + test("downgrade lock") { withTaskId(0) { assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) 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 3364f863d33c8..071dcbd321031 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -187,8 +187,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(master.getLocations("a3").size === 0, "master was told about a3") // Drop a1 and a2 from memory; this should be reported back to the master - store.dropFromMemoryAndReleaseLocks("a1", () => null: Either[Array[Any], ByteBuffer]) - store.dropFromMemoryAndReleaseLocks("a2", () => null: Either[Array[Any], ByteBuffer]) + store.dropFromMemoryIfExists("a1", () => null: Either[Array[Any], ByteBuffer]) + store.dropFromMemoryIfExists("a2", () => null: Either[Array[Any], ByteBuffer]) assert(store.getSingleAndReleaseLock("a1") === None, "a1 not removed from store") assert(store.getSingleAndReleaseLock("a2") === None, "a2 not removed from store") assert(master.getLocations("a1").size === 0, "master did not remove a1") @@ -429,8 +429,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE t2.join() t3.join() - store.dropFromMemoryAndReleaseLocks("a1", () => null: Either[Array[Any], ByteBuffer]) - store.dropFromMemoryAndReleaseLocks("a2", () => null: Either[Array[Any], ByteBuffer]) + store.dropFromMemoryIfExists("a1", () => null: Either[Array[Any], ByteBuffer]) + store.dropFromMemoryIfExists("a2", () => null: Either[Array[Any], ByteBuffer]) store.waitForAsyncReregister() } } @@ -1283,6 +1283,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE store = makeBlockManager(12000) val memoryStore = store.memoryStore val blockId = BlockId("rdd_3_10") + store.blockInfoManager.lockNewBlockForWriting( + blockId, new BlockInfo(StorageLevel.MEMORY_ONLY, tellMaster = false)) val result = memoryStore.putBytes(blockId, 13000, () => { fail("A big ByteBuffer that cannot be put into MemoryStore should not be created") }) @@ -1367,23 +1369,20 @@ private object BlockManagerSuite { } } - def dropFromMemoryAndReleaseLocks( + def dropFromMemoryIfExists( blockId: BlockId, data: () => Either[Array[Any], ByteBuffer]): Unit = { - store.dropFromMemory(blockId, data) match { - case Some(newEffectiveStorageLevel) => - if (newEffectiveStorageLevel.isValid) { - // The block is still present in at least one store, so release the lock - // but don't delete the block info - store.releaseLock(blockId) - } else { - // The block isn't present in any store, so delete the block info so that the - // block can be stored again - store.blockInfoManager.removeBlock(blockId) - } - case None => - // TODO: it's confusing why we have to do nothing here. I think this is only needed - // for the "re-registration doesn't deadlock" test, which is slightly confusing. + store.blockInfoManager.lockForWriting(blockId).foreach { info => + val newEffectiveStorageLevel = store.dropFromMemory(blockId, data) + if (newEffectiveStorageLevel.isValid) { + // The block is still present in at least one store, so release the lock + // but don't delete the block info + store.releaseLock(blockId) + } else { + // The block isn't present in any store, so delete the block info so that the + // block can be stored again + store.blockInfoManager.removeBlock(blockId) + } } } From 0b7281b680dbd00bfe70b0ea0d01669d9f62d128 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 23 Feb 2016 15:25:47 -0800 Subject: [PATCH 72/81] Simplify confusing getOrElseUpdate in lockNewBlockForWriting(). --- .../scala/org/apache/spark/storage/BlockInfoManager.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index deeecbe953cb3..aa11074e6bcd1 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -306,9 +306,9 @@ private[storage] class BlockInfoManager extends Logging { blockId: BlockId, newBlockInfo: BlockInfo): Boolean = synchronized { logTrace(s"Task $currentTaskAttemptId trying to put $blockId") - val actualInfo = infos.getOrElseUpdate(blockId, newBlockInfo) - if (actualInfo eq newBlockInfo) { - actualInfo.writerTask = currentTaskAttemptId + if (!infos.contains(blockId)) { + infos(blockId) = newBlockInfo + newBlockInfo.writerTask = currentTaskAttemptId writeLocksByTask.addBinding(currentTaskAttemptId, blockId) true } else { From 68b9e83f10438345773c92043055955e9bc260c2 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 23 Feb 2016 15:27:03 -0800 Subject: [PATCH 73/81] Roll back checking of blockWasSuccessfullyStored in replication code. --- .../main/scala/org/apache/spark/storage/BlockManager.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 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 c2400907b0687..d4ba68bb8196b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -833,7 +833,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 (blockWasSuccessfullyStored && putLevel.replication > 1) { + if (putLevel.replication > 1) { data match { case ByteBufferValues(bytes) => if (replicationFuture != null) { @@ -857,7 +857,7 @@ private[spark] class BlockManager( BlockManager.dispose(bytesAfterPut) - if (blockWasSuccessfullyStored && putLevel.replication > 1) { + if (putLevel.replication > 1) { logDebug("Putting block %s with replication took %s" .format(blockId, Utils.getUsedTimeMs(startTimeMs))) } else { From a5ef11b5ed6d8b1328a7642cc501b9a0480139ae Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 23 Feb 2016 15:32:40 -0800 Subject: [PATCH 74/81] Explain seemingly-unreachable error handling code. --- .../main/scala/org/apache/spark/storage/BlockManager.scala | 5 +++++ 1 file changed, 5 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 d4ba68bb8196b..e4c8a9f38cf24 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -517,6 +517,11 @@ private[spark] class BlockManager( } } } else { + // This branch represents a case where the BlockInfoManager contained an entry for + // the block but the block could not be found in any of the block stores. This case + // should never occur, but for completeness's sake we address it here. + logError( + s"Block $blockId is supposedly stored locally but was not found in any block store") releaseLock(blockId) None } From b9d6e181db0c7293b0b00b3a679716f652df936f Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 23 Feb 2016 16:03:29 -0800 Subject: [PATCH 75/81] Require tasks to explicitly register themselves with the BlockManager. --- .../org/apache/spark/executor/Executor.scala | 2 + .../spark/storage/BlockInfoManager.scala | 63 +++++++++++-------- .../apache/spark/storage/BlockManager.scala | 7 +++ .../spark/storage/BlockInfoManagerSuite.scala | 14 +++-- .../spark/storage/BlockManagerSuite.scala | 1 + 5 files changed, 56 insertions(+), 31 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 a602fcac68a6b..4abb9fb425c45 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -207,6 +207,8 @@ private[spark] class Executor( logDebug("Task " + taskId + "'s epoch is " + task.epoch) env.mapOutputTracker.updateEpoch(task.epoch) + env.blockManager.registerTask(taskId) + // Run the actual task and measure its runtime. taskStart = System.currentTimeMillis() var threwException = true diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index aa11074e6bcd1..5d80dd7802355 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -17,13 +17,11 @@ package org.apache.spark.storage -import java.lang import javax.annotation.concurrent.GuardedBy import scala.collection.JavaConverters._ import scala.collection.mutable -import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} import com.google.common.collect.ConcurrentHashMultiset import org.apache.spark.{Logging, SparkException, TaskContext} @@ -141,18 +139,29 @@ private[storage] class BlockInfoManager extends Logging { /** * Tracks the set of blocks that each task has locked for reading, along with the number of times - * that a block has been locked (since our read locks are re-entrant). This is thread-safe. + * that a block has been locked (since our read locks are re-entrant). */ - private[this] val readLocksByTask: LoadingCache[lang.Long, ConcurrentHashMultiset[BlockId]] = { - // We need to explicitly box as java.lang.Long to avoid a type mismatch error: - val loader = new CacheLoader[java.lang.Long, ConcurrentHashMultiset[BlockId]] { - override def load(t: java.lang.Long) = ConcurrentHashMultiset.create[BlockId]() - } - CacheBuilder.newBuilder().build(loader) - } + @GuardedBy("this") + private[this] val readLocksByTask = + new mutable.HashMap[TaskAttemptId, ConcurrentHashMultiset[BlockId]] + + // ---------------------------------------------------------------------------------------------- + + // Initialization for special task attempt ids: + registerTask(BlockInfo.NON_TASK_WRITER) // ---------------------------------------------------------------------------------------------- + /** + * Called at the start of a task in order to register that task with this [[BlockInfoManager]]. + * This must be called prior to calling any other BlockInfoManager methods from that task. + */ + def registerTask(taskAttemptId: TaskAttemptId): Unit = { + require(!readLocksByTask.contains(taskAttemptId), + s"Task attempt $taskAttemptId is already registered") + readLocksByTask(taskAttemptId) = ConcurrentHashMultiset.create() + } + /** * Returns the current task's task attempt id (which uniquely identifies the task), or * [[BlockInfo.NON_TASK_WRITER]] if called by a non-task thread. @@ -284,7 +293,7 @@ private[storage] class BlockInfoManager extends Logging { } else { assert(info.readerCount > 0, s"Block $blockId is not locked for reading") info.readerCount -= 1 - val countsForTask = readLocksByTask.get(currentTaskAttemptId) + val countsForTask = readLocksByTask(currentTaskAttemptId) val newPinCountForTask: Int = countsForTask.remove(blockId, 1) - 1 assert(newPinCountForTask >= 0, s"Task $currentTaskAttemptId release lock on block $blockId more times than it acquired it") @@ -325,20 +334,21 @@ private[storage] class BlockInfoManager extends Logging { */ def releaseAllLocksForTask(taskAttemptId: TaskAttemptId): Seq[BlockId] = { val blocksWithReleasedLocks = mutable.ArrayBuffer[BlockId]() - synchronized { - writeLocksByTask.remove(taskAttemptId).foreach { locks => - for (blockId <- locks) { - infos.get(blockId).foreach { info => - assert(info.writerTask == taskAttemptId) - info.writerTask = BlockInfo.NO_WRITER - } - blocksWithReleasedLocks += blockId - } + + val readLocks = synchronized { + readLocksByTask.remove(taskAttemptId).get + } + val writeLocks = synchronized { + writeLocksByTask.remove(taskAttemptId).getOrElse(Seq.empty) + } + + for (blockId <- writeLocks) { + infos.get(blockId).foreach { info => + assert(info.writerTask == taskAttemptId) + info.writerTask = BlockInfo.NO_WRITER } - notifyAll() + blocksWithReleasedLocks += blockId } - val readLocks = readLocksByTask.get(taskAttemptId) - readLocksByTask.invalidate(taskAttemptId) readLocks.entrySet().iterator().asScala.foreach { entry => val blockId = entry.getElement val lockCount = entry.getCount @@ -350,6 +360,7 @@ private[storage] class BlockInfoManager extends Logging { } } } + synchronized { notifyAll() } @@ -369,8 +380,8 @@ private[storage] class BlockInfoManager extends Logging { */ private[storage] def getNumberOfMapEntries: Long = synchronized { size + - readLocksByTask.size() + - readLocksByTask.asMap().asScala.map(_._2.size()).sum + + readLocksByTask.size + + readLocksByTask.map(_._2.size()).sum + writeLocksByTask.size + writeLocksByTask.map(_._2.size).sum } @@ -419,7 +430,7 @@ private[storage] class BlockInfoManager extends Logging { blockInfo.removed = true } infos.clear() - readLocksByTask.invalidateAll() + readLocksByTask.clear() writeLocksByTask.clear() notifyAll() } 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 e4c8a9f38cf24..4f821a98d6121 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -631,6 +631,13 @@ private[spark] class BlockManager( blockInfoManager.unlock(blockId) } + /** + * Registers a task with the BlockManager in order to initialize per-task bookkeeping structures. + */ + def registerTask(taskAttemptId: Long): Unit = { + blockInfoManager.registerTask(taskAttemptId) + } + /** * Release all locks for the given task. * diff --git a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala index 71a6568f3d5bb..662b18f667b0d 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockInfoManagerSuite.scala @@ -34,6 +34,9 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { override protected def beforeEach(): Unit = { super.beforeEach() blockInfoManager = new BlockInfoManager() + for (t <- 0 to 4) { + blockInfoManager.registerTask(t) + } } override protected def afterEach(): Unit = { @@ -62,7 +65,6 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { } test("initial memory usage") { - assert(blockInfoManager.getNumberOfMapEntries === 0) assert(blockInfoManager.size === 0) } @@ -72,7 +74,8 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { assert(blockInfoManager.lockForWriting("non-existent-block").isEmpty) } - test("basic putAndLockForWritingIfAbsent") { + test("basic lockNewBlockForWriting") { + val initialNumMapEntries = blockInfoManager.getNumberOfMapEntries val blockInfo = newBlockInfo() withTaskId(1) { assert(blockInfoManager.lockNewBlockForWriting("block", blockInfo)) @@ -86,7 +89,7 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { assert(blockInfo.writerTask === BlockInfo.NO_WRITER) } assert(blockInfoManager.size === 1) - assert(blockInfoManager.getNumberOfMapEntries === 1) + assert(blockInfoManager.getNumberOfMapEntries === initialNumMapEntries + 1) } test("read locks are reentrant") { @@ -273,11 +276,12 @@ class BlockInfoManagerSuite extends SparkFunSuite with BeforeAndAfterEach { } test("releaseAllLocksForTask releases write locks") { + val initialNumMapEntries = blockInfoManager.getNumberOfMapEntries withTaskId(0) { assert(blockInfoManager.lockNewBlockForWriting("block", newBlockInfo())) } - assert(blockInfoManager.getNumberOfMapEntries === 3) + assert(blockInfoManager.getNumberOfMapEntries === initialNumMapEntries + 3) blockInfoManager.releaseAllLocksForTask(0) - assert(blockInfoManager.getNumberOfMapEntries === 1) + assert(blockInfoManager.getNumberOfMapEntries === initialNumMapEntries) } } 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 071dcbd321031..e4ab9ee0ebb38 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -865,6 +865,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE test("updated block statuses") { store = makeBlockManager(12000) + store.registerTask(0) val list = List.fill(2)(new Array[Byte](2000)) val bigList = List.fill(8)(new Array[Byte](2000)) From 5df7284b4b9fc5de2da773be51f7ae17a3543dcb Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 23 Feb 2016 19:12:07 -0800 Subject: [PATCH 76/81] DeMorgan. --- .../main/scala/org/apache/spark/storage/BlockInfoManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index 5d80dd7802355..c1e9a18a9fcf3 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -86,7 +86,7 @@ private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: Boolea // A block's reader count must be non-negative: assert(_readerCount >= 0) // A block is either locked for reading or for writing, but not for both at the same time: - assert(!(_readerCount != 0 && _writerTask != BlockInfo.NO_WRITER)) + assert(_readerCount == 0 || _writerTask == BlockInfo.NO_WRITER) // If a block is removed then it is not locked: assert(!_removed || (_readerCount == 0 && _writerTask == BlockInfo.NO_WRITER)) } From eab288c1a884966efda962f05298edf127c1b8a9 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 23 Feb 2016 19:12:37 -0800 Subject: [PATCH 77/81] Synchronize BlockInfoManager.registerTask() --- .../main/scala/org/apache/spark/storage/BlockInfoManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index c1e9a18a9fcf3..415dae2497fa9 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -156,7 +156,7 @@ private[storage] class BlockInfoManager extends Logging { * Called at the start of a task in order to register that task with this [[BlockInfoManager]]. * This must be called prior to calling any other BlockInfoManager methods from that task. */ - def registerTask(taskAttemptId: TaskAttemptId): Unit = { + def registerTask(taskAttemptId: TaskAttemptId): Unit = synchronized { require(!readLocksByTask.contains(taskAttemptId), s"Task attempt $taskAttemptId is already registered") readLocksByTask(taskAttemptId) = ConcurrentHashMultiset.create() From 06ebef5ba6a9e625db7a2d3573c113dd2e340456 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 23 Feb 2016 19:20:23 -0800 Subject: [PATCH 78/81] Minor comment fixes. --- .../scala/org/apache/spark/storage/BlockInfoManager.scala | 5 ++++- .../main/scala/org/apache/spark/storage/BlockManager.scala | 1 + 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index 415dae2497fa9..abb6c40048603 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -30,6 +30,9 @@ import org.apache.spark.{Logging, SparkException, TaskContext} /** * Tracks metadata for an individual block. * + * Instances of this class are _not_ thread-safe and are protected by locks in the + * [[BlockInfoManager]]. + * * @param level the block's storage level. This is the requested persistence level, not the * effective storage level of the block (i.e. if this is MEMORY_AND_DISK, then this * does not imply that the block is actually resident in memory). @@ -176,7 +179,7 @@ private[storage] class BlockInfoManager extends Logging { * If another task has already locked this block for reading, then the read lock will be * immediately granted to the calling task and its lock count will be incremented. * - * If another task has locked this block for reading, then this call will block until the write + * If another task has locked this block for writing, then this call will block until the write * lock is released or will return immediately if `blocking = false`. * * A single task can lock a block multiple times for reading, in which case each lock will need 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 4f821a98d6121..29124b368e405 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -1016,6 +1016,7 @@ private[spark] class BlockManager( * If `data` is not put on disk, it won't be created. * * The caller of this method must hold a write lock on the block before calling this method. + * This method does not release the write lock. * * @return the block's new effective StorageLevel. */ From 0628a33b08cb567580aa9c098203f9fafa61b5d2 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 23 Feb 2016 19:21:44 -0800 Subject: [PATCH 79/81] Check lockForReading outcome in downgradeLock() --- .../main/scala/org/apache/spark/storage/BlockInfoManager.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index abb6c40048603..d97ae313b4185 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -279,7 +279,8 @@ private[storage] class BlockInfoManager extends Logging { s"Task $currentTaskAttemptId tried to downgrade a write lock that it does not hold on" + s" block $blockId") unlock(blockId) - lockForReading(blockId, blocking = false) + val lockOutcome = lockForReading(blockId, blocking = false) + assert(lockOutcome.isDefined) } /** From b963178d5521dbced168c802b22e47eebc6acad2 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 23 Feb 2016 19:23:37 -0800 Subject: [PATCH 80/81] More logTrace detail in lockNewBlockForWriting --- .../main/scala/org/apache/spark/storage/BlockInfoManager.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index d97ae313b4185..0eda97e58d451 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -323,8 +323,11 @@ private[storage] class BlockInfoManager extends Logging { infos(blockId) = newBlockInfo newBlockInfo.writerTask = currentTaskAttemptId writeLocksByTask.addBinding(currentTaskAttemptId, blockId) + logTrace(s"Task $currentTaskAttemptId successfully locked new block $blockId") true } else { + logTrace(s"Task $currentTaskAttemptId did not create and lock block $blockId " + + s"because that block already exists") false } } From 9becde3d94ea41bba9e275b9108c61b91074f035 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 23 Feb 2016 23:00:00 -0800 Subject: [PATCH 81/81] Move registration of task with BlockManager into Task.run() --- core/src/main/scala/org/apache/spark/executor/Executor.scala | 2 -- core/src/main/scala/org/apache/spark/scheduler/Task.scala | 1 + core/src/test/scala/org/apache/spark/CacheManagerSuite.scala | 1 + 3 files changed, 2 insertions(+), 2 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 4abb9fb425c45..a602fcac68a6b 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -207,8 +207,6 @@ private[spark] class Executor( logDebug("Task " + taskId + "'s epoch is " + task.epoch) env.mapOutputTracker.updateEpoch(task.epoch) - env.blockManager.registerTask(taskId) - // Run the actual task and measure its runtime. taskStart = System.currentTimeMillis() var threwException = true diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index a49f3716e2702..5c68d001f280f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -64,6 +64,7 @@ private[spark] abstract class Task[T]( taskAttemptId: Long, attemptNumber: Int, metricsSystem: MetricsSystem): T = { + SparkEnv.get.blockManager.registerTask(taskAttemptId) context = new TaskContextImpl( stageId, partitionId, diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index 48a0282b30cf0..ffc02bcb011f3 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -87,6 +87,7 @@ class CacheManagerSuite extends SparkFunSuite with LocalSparkContext with Before val context = TaskContext.empty() try { TaskContext.setTaskContext(context) + sc.env.blockManager.registerTask(0) cacheManager.getOrCompute(rdd3, split, context, StorageLevel.MEMORY_ONLY) assert(context.taskMetrics.updatedBlockStatuses.size === 2) } finally {