forked from apache/spark
-
Notifications
You must be signed in to change notification settings - Fork 1
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[SPARK-3019] Pluggable block transfer interface (BlockTransferService)
This pull request creates a new BlockTransferService interface for block fetch/upload and refactors the existing ConnectionManager to implement BlockTransferService (NioBlockTransferService). Most of the changes are simply moving code around. The main class to inspect is ShuffleBlockFetcherIterator. Review guide: - Most of the ConnectionManager code is now in network.cm package - ManagedBuffer is a new buffer abstraction backed by several different implementations (file segment, nio ByteBuffer, Netty ByteBuf) - BlockTransferService is the main internal interface introduced in this PR - NioBlockTransferService implements BlockTransferService and replaces the old BlockManagerWorker - ShuffleBlockFetcherIterator replaces the told BlockFetcherIterator to use the new interface TODOs that should be separate PRs: - Implement NettyBlockTransferService - Finalize the API/semantics for ManagedBuffer.release() Author: Reynold Xin <rxin@apache.org> Closes apache#2240 from rxin/blockTransferService and squashes the following commits: 64cd9d7 [Reynold Xin] Merge branch 'master' into blockTransferService 1dfd3d7 [Reynold Xin] Limit the length of the FileInputStream. 1332156 [Reynold Xin] Fixed style violation from refactoring. 2960c93 [Reynold Xin] Added ShuffleBlockFetcherIteratorSuite. e29c721 [Reynold Xin] Updated comment for ShuffleBlockFetcherIterator. 8a1046e [Reynold Xin] Code review feedback: 2c6b1e1 [Reynold Xin] Removed println in test cases. 2a907e4 [Reynold Xin] Merge branch 'master' into blockTransferService-merge 07ccf0d [Reynold Xin] Added init check to CMBlockTransferService. 98c668a [Reynold Xin] Added failure handling and fixed unit tests. ae05fcd [Reynold Xin] Updated tests, although DistributedSuite is hanging. d8d595c [Reynold Xin] Merge branch 'master' of github.com:apache/spark into blockTransferService 9ef279c [Reynold Xin] Initial refactoring to move ConnectionManager to use the BlockTransferService.
- Loading branch information
Showing
38 changed files
with
1,129 additions
and
1,273 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
37 changes: 37 additions & 0 deletions
37
core/src/main/scala/org/apache/spark/network/BlockFetchingListener.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,37 @@ | ||
/* | ||
* 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 | ||
|
||
import java.util.EventListener | ||
|
||
|
||
/** | ||
* Listener callback interface for [[BlockTransferService.fetchBlocks]]. | ||
*/ | ||
trait BlockFetchingListener extends EventListener { | ||
|
||
/** | ||
* Called once per successfully fetched block. | ||
*/ | ||
def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit | ||
|
||
/** | ||
* Called upon failures. For each failure, this is called only once (i.e. not once per block). | ||
*/ | ||
def onBlockFetchFailure(exception: Throwable): Unit | ||
} |
131 changes: 131 additions & 0 deletions
131
core/src/main/scala/org/apache/spark/network/BlockTransferService.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,131 @@ | ||
/* | ||
* 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 | ||
|
||
import scala.concurrent.{Await, Future} | ||
import scala.concurrent.duration.Duration | ||
|
||
import org.apache.spark.storage.StorageLevel | ||
|
||
|
||
abstract class BlockTransferService { | ||
|
||
/** | ||
* Initialize the transfer service by giving it the BlockDataManager that can be used to fetch | ||
* local blocks or put local blocks. | ||
*/ | ||
def init(blockDataManager: BlockDataManager) | ||
|
||
/** | ||
* Tear down the transfer service. | ||
*/ | ||
def stop(): Unit | ||
|
||
/** | ||
* Port number the service is listening on, available only after [[init]] is invoked. | ||
*/ | ||
def port: Int | ||
|
||
/** | ||
* Host name the service is listening on, available only after [[init]] is invoked. | ||
*/ | ||
def hostName: String | ||
|
||
/** | ||
* Fetch a sequence of blocks from a remote node asynchronously, | ||
* available only after [[init]] is invoked. | ||
* | ||
* Note that [[BlockFetchingListener.onBlockFetchSuccess]] is called once per block, | ||
* while [[BlockFetchingListener.onBlockFetchFailure]] is called once per failure (not per block). | ||
* | ||
* Note that this API takes a sequence so the implementation can batch requests, and does not | ||
* return a future so the underlying implementation can invoke onBlockFetchSuccess as soon as | ||
* the data of a block is fetched, rather than waiting for all blocks to be fetched. | ||
*/ | ||
def fetchBlocks( | ||
hostName: String, | ||
port: Int, | ||
blockIds: Seq[String], | ||
listener: BlockFetchingListener): Unit | ||
|
||
/** | ||
* Upload a single block to a remote node, available only after [[init]] is invoked. | ||
*/ | ||
def uploadBlock( | ||
hostname: String, | ||
port: Int, | ||
blockId: String, | ||
blockData: ManagedBuffer, | ||
level: StorageLevel): Future[Unit] | ||
|
||
/** | ||
* A special case of [[fetchBlocks]], as it fetches only one block and is blocking. | ||
* | ||
* It is also only available after [[init]] is invoked. | ||
*/ | ||
def fetchBlockSync(hostName: String, port: Int, blockId: String): ManagedBuffer = { | ||
// A monitor for the thread to wait on. | ||
val lock = new Object | ||
@volatile var result: Either[ManagedBuffer, Throwable] = null | ||
fetchBlocks(hostName, port, Seq(blockId), new BlockFetchingListener { | ||
override def onBlockFetchFailure(exception: Throwable): Unit = { | ||
lock.synchronized { | ||
result = Right(exception) | ||
lock.notify() | ||
} | ||
} | ||
override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = { | ||
lock.synchronized { | ||
result = Left(data) | ||
lock.notify() | ||
} | ||
} | ||
}) | ||
|
||
// Sleep until result is no longer null | ||
lock.synchronized { | ||
while (result == null) { | ||
try { | ||
lock.wait() | ||
} catch { | ||
case e: InterruptedException => | ||
} | ||
} | ||
} | ||
|
||
result match { | ||
case Left(data) => data | ||
case Right(e) => throw e | ||
} | ||
} | ||
|
||
/** | ||
* Upload a single block to a remote node, available only after [[init]] is invoked. | ||
* | ||
* This method is similar to [[uploadBlock]], except this one blocks the thread | ||
* until the upload finishes. | ||
*/ | ||
def uploadBlockSync( | ||
hostname: String, | ||
port: Int, | ||
blockId: String, | ||
blockData: ManagedBuffer, | ||
level: StorageLevel): Unit = { | ||
Await.result(uploadBlock(hostname, port, blockId, blockData, level), Duration.Inf) | ||
} | ||
} |
103 changes: 0 additions & 103 deletions
103
core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala
This file was deleted.
Oops, something went wrong.
Oops, something went wrong.