Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
49 commits
Select commit Hold shift + click to select a range
165eab1
[SPARK-3453] Refactor Netty module to use BlockTransferService.
rxin Sep 9, 2014
1760d32
Use Epoll.isAvailable in BlockServer as well.
rxin Sep 9, 2014
2b44cf1
Added more documentation.
rxin Sep 9, 2014
064747b
Reference count buffers and clean them up properly.
rxin Sep 10, 2014
b5c8d1f
Fixed ShuffleBlockFetcherIteratorSuite.
rxin Sep 10, 2014
108c9ed
Forgot to add TestSerializer to the commit list.
rxin Sep 10, 2014
1be4e8e
Shorten NioManagedBuffer and NettyManagedBuffer class names.
rxin Sep 10, 2014
cb589ec
Added more test cases covering cleanup when fault happens in ShuffleB…
rxin Sep 11, 2014
5cd33d7
Fixed style violation.
rxin Sep 11, 2014
9e0cb87
Fixed BlockClientHandlerSuite
rxin Sep 11, 2014
d23ed7b
Incorporated feedback from Norman:
rxin Sep 12, 2014
b2f3281
Added connection pooling.
rxin Sep 12, 2014
14323a5
Removed BlockManager.getLocalShuffleFromDisk.
rxin Sep 12, 2014
f0a16e9
Fixed test hanging.
rxin Sep 12, 2014
519d64d
Mark private package visibility and MimaExcludes.
rxin Sep 12, 2014
c066309
Implement java.io.Closeable interface.
rxin Sep 13, 2014
6afc435
Added logging.
rxin Sep 17, 2014
f63fb4c
Add more debug message.
rxin Sep 29, 2014
d68f328
Logging close() in case close() fails.
rxin Sep 29, 2014
1bdd7ee
Fixed tests.
rxin Sep 29, 2014
bec4ea2
Removed OIO and added num threads settings.
rxin Sep 29, 2014
4b18db2
Copy the buffer in fetchBlockSync.
rxin Sep 29, 2014
a0518c7
Implemented block uploads.
rxin Sep 30, 2014
407e59a
Fix style violation.
rxin Sep 30, 2014
f6c220d
Merge with latest master.
rxin Sep 30, 2014
5d98ce3
Flip buffer.
rxin Sep 30, 2014
f7e7568
Fixed spark.shuffle.io.receiveBuffer setting.
rxin Sep 30, 2014
29c6dcf
[SPARK-3453] Netty-based BlockTransferService, extracted from Spark core
aarondav Oct 6, 2014
ae4083a
[SPARK-2805] Upgrade Akka to 2.3.4
avati Oct 10, 2014
020691e
[SPARK-3886] [PySpark] use AutoBatchedSerializer by default
davies Oct 10, 2014
2c5d9dc
HOTFIX: Fix build issue with Akka 2.3.4 upgrade.
pwendell Oct 10, 2014
5b5dbe6
[SPARK-2924] Required by scala 2.11, only one fun/ctor amongst overri…
ScrapCodes Oct 11, 2014
8dc1ded
[SPARK-3867][PySpark] ./python/run-tests failed when it run with Pyth…
cocoatomo Oct 11, 2014
aa58f67
[SPARK-3909][PySpark][Doc] A corrupted format in Sphinx documents and…
cocoatomo Oct 11, 2014
939f276
Attempt to make comm. bidirectional
aarondav Oct 12, 2014
dd420fd
Merge branch 'master' of https://github.com/apache/spark into netty-test
aarondav Oct 17, 2014
7b7a26c
Fix Nio compile issue
aarondav Oct 17, 2014
d236dfd
Remove no-op serializer :)
aarondav Oct 17, 2014
9da0bc1
Add RPC unit tests
aarondav Oct 17, 2014
ccd4959
Don't throw exception if client immediately fails
aarondav Oct 17, 2014
e5675a4
Fail outstanding RPCs as well
aarondav Oct 18, 2014
322dfc1
Address Reynold's comments, including major rename
aarondav Oct 27, 2014
8dfcceb
Merge branch 'master' of https://github.com/apache/spark into netty
aarondav Oct 27, 2014
14e37f7
Address Reynold's comments
aarondav Oct 28, 2014
0c5bca2
Merge branch 'master' of https://github.com/apache/spark into netty
aarondav Oct 28, 2014
2b0d1c0
100ch
aarondav Oct 28, 2014
4a204b8
Fail block fetches if client connection fails
aarondav Oct 29, 2014
d7be11b
Turn netty on by default
aarondav Oct 29, 2014
cadfd28
Turn netty off by default
aarondav Oct 29, 2014
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
Fixed ShuffleBlockFetcherIteratorSuite.
  • Loading branch information
rxin authored and aarondav committed Oct 10, 2014
commit b5c8d1fca6d3cf5c2b95395310200c8149a7eb16
Original file line number Diff line number Diff line change
Expand Up @@ -212,9 +212,8 @@ private[spark] class BlockManager(
}

/**
* Interface to get local block data.
*
* @return Some(buffer) if the block exists locally, and None if it doesn't.
* Interface to get local block data. Throws an exception if the block cannot be found or
* cannot be read successfully.
*/
override def getBlockData(blockId: String): ManagedBuffer = {
val bid = BlockId(blockId)
Expand Down
Original file line number Diff line number Diff line change
@@ -1,19 +1,19 @@
/*
* 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.
*/
* 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.netty

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,136 +17,62 @@

package org.apache.spark.storage

import org.apache.spark.TaskContext
import org.apache.spark.network.{BlockFetchingListener, BlockTransferService}

import org.mockito.Mockito._
import org.mockito.Matchers.{any, eq => meq}
import org.mockito.invocation.InvocationOnMock
import org.mockito.stubbing.Answer

import org.scalatest.FunSuite

import org.apache.spark.{SparkConf, TaskContext}
import org.apache.spark.network._
import org.apache.spark.serializer.TestSerializer

class ShuffleBlockFetcherIteratorSuite extends FunSuite {

test("handle local read failures in BlockManager") {
val transfer = mock(classOf[BlockTransferService])
val blockManager = mock(classOf[BlockManager])
doReturn(BlockManagerId("test-client", "test-client", 1)).when(blockManager).blockManagerId

val blIds = Array[BlockId](
ShuffleBlockId(0,0,0),
ShuffleBlockId(0,1,0),
ShuffleBlockId(0,2,0),
ShuffleBlockId(0,3,0),
ShuffleBlockId(0,4,0))

val optItr = mock(classOf[Option[Iterator[Any]]])
val answer = new Answer[Option[Iterator[Any]]] {
override def answer(invocation: InvocationOnMock) = Option[Iterator[Any]] {
throw new Exception
}
}

// 3rd block is going to fail
doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(0)), any())
doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(1)), any())
doAnswer(answer).when(blockManager).getLocalShuffleFromDisk(meq(blIds(2)), any())
doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(3)), any())
doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(4)), any())

val bmId = BlockManagerId("test-client", "test-client", 1)
val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])](
(bmId, blIds.map(blId => (blId, 1.asInstanceOf[Long])).toSeq)
)

val iterator = new ShuffleBlockFetcherIterator(
new TaskContext(0, 0, 0),
transfer,
blockManager,
blocksByAddress,
null,
48 * 1024 * 1024)
class ShuffleBlockFetcherIteratorSuite extends FunSuite {

// Without exhausting the iterator, the iterator should be lazy and not call
// getLocalShuffleFromDisk.
verify(blockManager, times(0)).getLocalShuffleFromDisk(any(), any())

assert(iterator.hasNext, "iterator should have 5 elements but actually has no elements")
// the 2nd element of the tuple returned by iterator.next should be defined when
// fetching successfully
assert(iterator.next()._2.isDefined,
"1st element should be defined but is not actually defined")
verify(blockManager, times(1)).getLocalShuffleFromDisk(any(), any())

assert(iterator.hasNext, "iterator should have 5 elements but actually has 1 element")
assert(iterator.next()._2.isDefined,
"2nd element should be defined but is not actually defined")
verify(blockManager, times(2)).getLocalShuffleFromDisk(any(), any())

assert(iterator.hasNext, "iterator should have 5 elements but actually has 2 elements")
// 3rd fetch should be failed
intercept[Exception] {
iterator.next()
}
verify(blockManager, times(3)).getLocalShuffleFromDisk(any(), any())
}
val conf = new SparkConf

test("handle local read successes") {
val transfer = mock(classOf[BlockTransferService])
test("handle successful local reads") {
val buf = mock(classOf[ManagedBuffer])
val blockManager = mock(classOf[BlockManager])
doReturn(BlockManagerId("test-client", "test-client", 1)).when(blockManager).blockManagerId

val blIds = Array[BlockId](
ShuffleBlockId(0,0,0),
ShuffleBlockId(0,1,0),
ShuffleBlockId(0,2,0),
ShuffleBlockId(0,3,0),
ShuffleBlockId(0,4,0))

val optItr = mock(classOf[Option[Iterator[Any]]])
val blockIds = Array[BlockId](
ShuffleBlockId(0, 0, 0),
ShuffleBlockId(0, 1, 0),
ShuffleBlockId(0, 2, 0),
ShuffleBlockId(0, 3, 0),
ShuffleBlockId(0, 4, 0))

// All blocks should be fetched successfully
doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(0)), any())
doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(1)), any())
doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(2)), any())
doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(3)), any())
doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(4)), any())
blockIds.foreach { blockId =>
doReturn(buf).when(blockManager).getBlockData(meq(blockId.toString))
}

val bmId = BlockManagerId("test-client", "test-client", 1)
val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])](
(bmId, blIds.map(blId => (blId, 1.asInstanceOf[Long])).toSeq)
(bmId, blockIds.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq)
)

val iterator = new ShuffleBlockFetcherIterator(
new TaskContext(0, 0, 0),
transfer,
mock(classOf[BlockTransferService]),
blockManager,
blocksByAddress,
null,
new TestSerializer,
48 * 1024 * 1024)

// Without exhausting the iterator, the iterator should be lazy and not call getLocalShuffleFromDisk.
verify(blockManager, times(0)).getLocalShuffleFromDisk(any(), any())

assert(iterator.hasNext, "iterator should have 5 elements but actually has no elements")
assert(iterator.next()._2.isDefined,
"All elements should be defined but 1st element is not actually defined")
assert(iterator.hasNext, "iterator should have 5 elements but actually has 1 element")
assert(iterator.next()._2.isDefined,
"All elements should be defined but 2nd element is not actually defined")
assert(iterator.hasNext, "iterator should have 5 elements but actually has 2 elements")
assert(iterator.next()._2.isDefined,
"All elements should be defined but 3rd element is not actually defined")
assert(iterator.hasNext, "iterator should have 5 elements but actually has 3 elements")
assert(iterator.next()._2.isDefined,
"All elements should be defined but 4th element is not actually defined")
assert(iterator.hasNext, "iterator should have 5 elements but actually has 4 elements")
assert(iterator.next()._2.isDefined,
"All elements should be defined but 5th element is not actually defined")

verify(blockManager, times(5)).getLocalShuffleFromDisk(any(), any())
// Local blocks are fetched immediately.
verify(blockManager, times(5)).getBlockData(any())

for (i <- 0 until 5) {
assert(iterator.hasNext, s"iterator should have 5 elements but actually has $i elements")
assert(iterator.next()._2.isDefined,
s"iterator should have 5 elements defined but actually has $i elements")
}
// No more fetching of local blocks.
verify(blockManager, times(5)).getBlockData(any())
}

test("handle remote fetch failures in BlockTransferService") {
Expand All @@ -173,7 +99,7 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite {
transfer,
blockManager,
blocksByAddress,
null,
new TestSerializer,
48 * 1024 * 1024)

iterator.foreach { case (_, iterOption) =>
Expand Down