Skip to content
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-3453] Netty-based BlockTransferService #2330

Closed
wants to merge 34 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
34 commits
Select commit Hold shift + click to select a range
5bb88f5
[SPARK-3453] Refactor Netty module to use BlockTransferService.
rxin Sep 9, 2014
9b3b397
Use Epoll.isAvailable in BlockServer as well.
rxin Sep 9, 2014
dd783ff
Added more documentation.
rxin Sep 9, 2014
b5b380e
Reference count buffers and clean them up properly.
rxin Sep 10, 2014
1474824
Fixed ShuffleBlockFetcherIteratorSuite.
rxin Sep 10, 2014
b404da3
Forgot to add TestSerializer to the commit list.
rxin Sep 10, 2014
fbf882d
Shorten NioManagedBuffer and NettyManagedBuffer class names.
rxin Sep 10, 2014
b32c3fe
Added more test cases covering cleanup when fault happens in ShuffleB…
rxin Sep 11, 2014
d135fa3
Fixed style violation.
rxin Sep 11, 2014
1e0d277
Fixed BlockClientHandlerSuite
rxin Sep 11, 2014
6e84cb2
Merge branch 'master' into netty-blockTransferService
rxin Sep 11, 2014
55266d1
Incorporated feedback from Norman:
rxin Sep 12, 2014
f83611e
Added connection pooling.
rxin Sep 12, 2014
6ddaa5d
Removed BlockManager.getLocalShuffleFromDisk.
rxin Sep 12, 2014
8295561
Fixed test hanging.
rxin Sep 12, 2014
d7d0aac
Mark private package visibility and MimaExcludes.
rxin Sep 12, 2014
29fe0cc
Implement java.io.Closeable interface.
rxin Sep 13, 2014
e92dad7
Merge branch 'master' into netty-blockTransferService
rxin Sep 17, 2014
a79a259
Added logging.
rxin Sep 17, 2014
088ed8a
Fixed error message.
rxin Sep 17, 2014
323dfec
Add more debug message.
rxin Sep 29, 2014
5814292
Logging close() in case close() fails.
rxin Sep 29, 2014
f23e682
Merge branch 'master' into netty-blockTransferService
rxin Sep 29, 2014
ba8c441
Fixed tests.
rxin Sep 29, 2014
ca88068
Merge branch 'buffer-debug' into netty-blockTransferService
rxin Sep 29, 2014
dfc2c34
Removed OIO and added num threads settings.
rxin Sep 29, 2014
3fbfd3f
Merge branch 'master' into netty-blockTransferService
rxin Sep 29, 2014
69f5d0a
Copy the buffer in fetchBlockSync.
rxin Sep 29, 2014
bc9ed22
Implemented block uploads.
rxin Sep 30, 2014
a3a09f6
Fix style violation.
rxin Sep 30, 2014
0140d6e
Merge branch 'master' into netty-blockTransferService
rxin Sep 30, 2014
0dae310
Merge with latest master.
rxin Sep 30, 2014
ad09236
Flip buffer.
rxin Sep 30, 2014
bdab2c7
Fixed spark.shuffle.io.receiveBuffer setting.
rxin Sep 30, 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
9 changes: 8 additions & 1 deletion core/src/main/scala/org/apache/spark/SparkEnv.scala
Original file line number Diff line number Diff line change
Expand Up @@ -32,13 +32,15 @@ import org.apache.spark.api.python.PythonWorkerFactory
import org.apache.spark.broadcast.BroadcastManager
import org.apache.spark.metrics.MetricsSystem
import org.apache.spark.network.BlockTransferService
import org.apache.spark.network.netty.NettyBlockTransferService
import org.apache.spark.network.nio.NioBlockTransferService
import org.apache.spark.scheduler.LiveListenerBus
import org.apache.spark.serializer.Serializer
import org.apache.spark.shuffle.{ShuffleMemoryManager, ShuffleManager}
import org.apache.spark.storage._
import org.apache.spark.util.{AkkaUtils, Utils}


/**
* :: DeveloperApi ::
* Holds all the runtime environment objects for a running Spark instance (either master or worker),
Expand Down Expand Up @@ -234,7 +236,12 @@ object SparkEnv extends Logging {

val shuffleMemoryManager = new ShuffleMemoryManager(conf)

val blockTransferService = new NioBlockTransferService(conf, securityManager)
// TODO(rxin): Config option based on class name, similar to shuffle mgr and compression codec.
val blockTransferService = if (conf.getBoolean("spark.shuffle.use.netty", false)) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It'd be better if we could go ahead and make the config stable, it's always painful for people to update later. Even if it's just like

val blockTransferService = conf.getString("spark.shuffle.transferService", "NIO") match {
  case "NETTY" => new NettyBlockTransferService(conf)
  case "NIO" => new NioBlockTransferService(conf, securityManager)
  case s => throw new UnsupportedOperationException("Unknown transfer service: " + s)
}

Potentially less pain for when we address the TODO at a later date.

new NettyBlockTransferService(conf)
} else {
new NioBlockTransferService(conf, securityManager)
}

val blockManagerMaster = new BlockManagerMaster(registerOrLookup(
"BlockManagerMaster",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,14 +20,14 @@ package org.apache.spark.network
import org.apache.spark.storage.StorageLevel


private[spark]
trait BlockDataManager {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Did you add this class at some point? Either way, would you mind adding a class comment? It's not clear how it differs from all the other block-related managers.


/**
* 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.
*/
def getBlockData(blockId: String): Option[ManagedBuffer]
def getBlockData(blockId: String): ManagedBuffer
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not sure how I feel about this taking a blockId as a string, especially if it's implemented by the BlockManager itself. What's the reasoning behind not taking a BlockId? Just too many users of the API that only have a String?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

see the pr description on the todos as separate prs


/**
* Put the block locally, using the given storage level.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import java.util.EventListener
/**
* Listener callback interface for [[BlockTransferService.fetchBlocks]].
*/
private[spark]
trait BlockFetchingListener extends EventListener {

/**
Expand All @@ -31,7 +32,7 @@ trait BlockFetchingListener extends EventListener {
def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit

/**
* Called upon failures. For each failure, this is called only once (i.e. not once per block).
* Called at least once per block upon failures.
*/
def onBlockFetchFailure(exception: Throwable): Unit
def onBlockFetchFailure(blockId: String, exception: Throwable): Unit
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,13 +17,17 @@

package org.apache.spark.network

import java.io.Closeable
import java.nio.ByteBuffer

import scala.concurrent.{Await, Future}
import scala.concurrent.duration.Duration

import org.apache.spark.storage.StorageLevel


abstract class BlockTransferService {
private[spark]
abstract class BlockTransferService extends Closeable {

/**
* Initialize the transfer service by giving it the BlockDataManager that can be used to fetch
Expand All @@ -34,7 +38,7 @@ abstract class BlockTransferService {
/**
* Tear down the transfer service.
*/
def stop(): Unit
def close(): Unit

/**
* Port number the service is listening on, available only after [[init]] is invoked.
Expand All @@ -50,9 +54,6 @@ abstract class BlockTransferService {
* 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.
Expand Down Expand Up @@ -83,15 +84,18 @@ abstract class BlockTransferService {
val lock = new Object
@volatile var result: Either[ManagedBuffer, Throwable] = null
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Out of curiosity, is there an advantage of this over Try[ManagedBuffer]? Note that this would simplify the final statement from a match to result.get (which re-throws exception if there was one)

fetchBlocks(hostName, port, Seq(blockId), new BlockFetchingListener {
override def onBlockFetchFailure(exception: Throwable): Unit = {
override def onBlockFetchFailure(blockId: String, exception: Throwable): Unit = {
lock.synchronized {
result = Right(exception)
lock.notify()
}
}
override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = {
lock.synchronized {
result = Left(data)
val ret = ByteBuffer.allocate(data.size.toInt)
ret.put(data.nioByteBuffer())
ret.flip()
result = Left(new NioManagedBuffer(ret))
lock.notify()
}
}
Expand Down
72 changes: 63 additions & 9 deletions core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,8 @@ import java.nio.channels.FileChannel.MapMode
import scala.util.Try

import com.google.common.io.ByteStreams
import io.netty.buffer.{ByteBufInputStream, ByteBuf}
import io.netty.buffer.{Unpooled, ByteBufInputStream, ByteBuf}
import io.netty.channel.DefaultFileRegion

import org.apache.spark.util.{ByteBufferInputStream, Utils}

Expand All @@ -34,11 +35,17 @@ import org.apache.spark.util.{ByteBufferInputStream, Utils}
* This interface provides an immutable view for data in the form of bytes. The implementation
* should specify how the data is provided:
*
* - FileSegmentManagedBuffer: data backed by part of a file
* - NioByteBufferManagedBuffer: data backed by a NIO ByteBuffer
* - NettyByteBufManagedBuffer: data backed by a Netty ByteBuf
* - [[FileSegmentManagedBuffer]]: data backed by part of a file
* - [[NioManagedBuffer]]: data backed by a NIO ByteBuffer
* - [[NettyManagedBuffer]]: data backed by a Netty ByteBuf
*
* The concrete buffer implementation might be managed outside the JVM garbage collector.
* For example, in the case of [[NettyManagedBuffer]], the buffers are reference counted.
* In that case, if the buffer is going to be passed around to a different thread, retain/release
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The comment here suggests that you should only have to retain if your buffer is a NettyManagedBuffer, which isn't quite true, as you could have an NioManagedBuffer whose ByteBuffer underlies a Netty ByteBuf, in which case it is never safe to pass around.

I'm just a little worried about sanitary buffer usage, as misuse of this API from not copying could lead to nondeterministic data corruption.

* should be called.
*/
sealed abstract class ManagedBuffer {
private[spark]
abstract class ManagedBuffer {
// Note that all the methods are defined with parenthesis because their implementations can
// have side effects (io operations).

Expand All @@ -57,12 +64,29 @@ sealed abstract class ManagedBuffer {
* it does not go over the limit.
*/
def inputStream(): InputStream

/**
* Increment the reference count by one if applicable.
*/
def retain(): this.type

/**
* If applicable, decrement the reference count by one and deallocates the buffer if the
* reference count reaches zero.
*/
def release(): this.type

/**
* Convert the buffer into an Netty object, used to write the data out.
*/
private[network] def convertToNetty(): AnyRef
}


/**
* A [[ManagedBuffer]] backed by a segment in a file
*/
private[spark]
final class FileSegmentManagedBuffer(val file: File, val offset: Long, val length: Long)
extends ManagedBuffer {

Expand Down Expand Up @@ -113,34 +137,64 @@ final class FileSegmentManagedBuffer(val file: File, val offset: Long, val lengt
}
}

private[network] override def convertToNetty(): AnyRef = {
val fileChannel = new FileInputStream(file).getChannel
new DefaultFileRegion(fileChannel, offset, length)
}

// Content of file segments are not in-memory, so no need to reference count.
override def retain(): this.type = this
override def release(): this.type = this

override def toString: String = s"${getClass.getName}($file, $offset, $length)"
}


/**
* A [[ManagedBuffer]] backed by [[java.nio.ByteBuffer]].
*/
final class NioByteBufferManagedBuffer(buf: ByteBuffer) extends ManagedBuffer {
private[spark]
final class NioManagedBuffer(buf: ByteBuffer) extends ManagedBuffer {

override def size: Long = buf.remaining()

override def nioByteBuffer() = buf.duplicate()

override def inputStream() = new ByteBufferInputStream(buf)

private[network] override def convertToNetty(): AnyRef = Unpooled.wrappedBuffer(buf)

// [[ByteBuffer]] is managed by the JVM garbage collector itself.
override def retain(): this.type = this
override def release(): this.type = this

override def toString: String = s"${getClass.getName}($buf)"
}


/**
* A [[ManagedBuffer]] backed by a Netty [[ByteBuf]].
*/
final class NettyByteBufManagedBuffer(buf: ByteBuf) extends ManagedBuffer {
private[spark]
final class NettyManagedBuffer(buf: ByteBuf) extends ManagedBuffer {

override def size: Long = buf.readableBytes()

override def nioByteBuffer() = buf.nioBuffer()

override def inputStream() = new ByteBufInputStream(buf)

// TODO(rxin): Promote this to top level ManagedBuffer interface and add documentation for it.
def release(): Unit = buf.release()
private[network] override def convertToNetty(): AnyRef = buf

override def retain(): this.type = {
buf.retain()
this
}

override def release(): this.type = {
buf.release()
this
}

override def toString: String = s"${getClass.getName}($buf)"
}
Original file line number Diff line number Diff line change
Expand Up @@ -15,15 +15,17 @@
* limitations under the License.
*/

package org.apache.spark.network.netty.client
package org.apache.spark.network

import java.util.EventListener
class BlockFetchFailureException(blockId: String, errorMsg: String, cause: Throwable)
extends Exception(errorMsg, cause) {

def this(blockId: String, errorMsg: String) = this(blockId, errorMsg, null)
}

trait BlockClientListener extends EventListener {

def onFetchSuccess(blockId: String, data: ReferenceCountedBuffer): Unit

def onFetchFailure(blockId: String, errorMsg: String): Unit
class BlockUploadFailureException(blockId: String, cause: Throwable)
extends Exception(s"Failed to fetch block $blockId", cause) {

def this(blockId: String) = this(blockId, null)
}
Loading