-
Notifications
You must be signed in to change notification settings - Fork 28.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-3019] Pluggable block transfer interface (BlockTransferService) #2240
Closed
Closed
Changes from all commits
Commits
Show all changes
13 commits
Select commit
Hold shift + click to select a range
9ef279c
Initial refactoring to move ConnectionManager to use the BlockTransfe…
rxin d8d595c
Merge branch 'master' of github.com:apache/spark into blockTransferSe…
rxin ae05fcd
Updated tests, although DistributedSuite is hanging.
rxin 98c668a
Added failure handling and fixed unit tests.
rxin 07ccf0d
Added init check to CMBlockTransferService.
rxin 2a907e4
Merge branch 'master' into blockTransferService-merge
rxin 2c6b1e1
Removed println in test cases.
rxin 8a1046e
Code review feedback:
rxin e29c721
Updated comment for ShuffleBlockFetcherIterator.
rxin 2960c93
Added ShuffleBlockFetcherIteratorSuite.
rxin 1332156
Fixed style violation from refactoring.
rxin 1dfd3d7
Limit the length of the FileInputStream.
rxin 64cd9d7
Merge branch 'master' into blockTransferService
rxin File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
So, some conf will be add here for specific BlockTransferService selection later?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yes
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Then, how did you plan to handle this securityManager parameter? move it into BlockTransferService interface ( though some implementation might not need it) to make it easy for instantiate works? or some how tweak connectionManager to make it use SecurityManager in lazy way so that it can retrive it from SparkEnv?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We can have it in init or the SparkEnv way (which is not ideal since I want to minimize dependency on SparkEnv). We can worry about that in my next PR.