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
[SPARK-3453] Netty-based BlockTransferService, extracted from Spark core
This PR encapsulates #2330, which is itself a continuation of #2240. The first goal of this
PR is to provide an alternate, simpler implementation of the ConnectionManager which is based on Netty.

In addition to this goal, however, we want to resolve [SPARK-3796](https://issues.apache.org/jira/browse/SPARK-3796), which calls for a
standalone shuffle service which can be integrated into the YARN NodeManager, Standalone Worker, or
on its own. This PR makes the first step in this direction by ensuring that the actual Netty service
is as small as possible and extracted from Spark core. Given this, we should be able to construct
this standalone jar which can be included in other JVMs without incurring significant dependency or
runtime issues. The actual work to ensure that such a standalone shuffle service would work in Spark
will be left for a future PR, however.

In order to minimize dependencies and allow for the service to be long-running (possibly
much longer-running than Spark, and possibly having to support multiple version of Spark
simultaneously), the entire service has been ported to Java, where we have full control
over the binary compatibility of the components and do not depend on the Scala runtime or
version.

These PRs have been addressed by folding in #2330:

SPARK-3453: Refactor Netty module to use BlockTransferService interface
SPARK-3018: Release all buffers upon task completion/failure
SPARK-3002: Create a connection pool and reuse clients across different threads
SPARK-3017: Integration tests and unit tests for connection failures
SPARK-3049: Make sure client doesn't block when server/connection has error(s)
SPARK-3502: SO_RCVBUF and SO_SNDBUF should be bootstrap childOption, not option
SPARK-3503: Disable thread local cache in PooledByteBufAllocator

TODO before mergeable:
[ ] Implement uploadBlock()
[ ] Unit tests for RPC side of code
[ ] Performance testing
[ ] Turn OFF by default (currently on for unit testing)
  • Loading branch information
aarondav committed Oct 10, 2014
commit 29c6dcfaacb2e8b1f0582c6d5e435349c52e29af
5 changes: 5 additions & 0 deletions core/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,11 @@
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>network</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>net.java.dev.jets3t</groupId>
<artifactId>jets3t</artifactId>
Expand Down
17 changes: 9 additions & 8 deletions core/src/main/scala/org/apache/spark/SparkEnv.scala
Original file line number Diff line number Diff line change
Expand Up @@ -32,15 +32,14 @@ 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.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 @@ -233,12 +232,14 @@ object SparkEnv extends Logging {

val shuffleMemoryManager = new ShuffleMemoryManager(conf)

// 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)) {
new NettyBlockTransferService(conf)
} else {
new NioBlockTransferService(conf, securityManager)
}
// TODO: This is only netty by default for initial testing -- it should not be merged as such!!!
val blockTransferService =
conf.get("spark.shuffle.blockTransferService", "netty").toLowerCase match {
case "netty" =>
new NettyBlockTransferService(conf)
case "nio" =>
new NioBlockTransferService(conf, securityManager)
}

val blockManagerMaster = new BlockManagerMaster(registerOrLookup(
"BlockManagerMaster",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,8 @@

package org.apache.spark.network

import org.apache.spark.storage.StorageLevel

import org.apache.spark.network.buffer.ManagedBuffer
import org.apache.spark.storage.{BlockId, StorageLevel}

private[spark]
trait BlockDataManager {
Expand All @@ -27,10 +27,10 @@ trait BlockDataManager {
* Interface to get local block data. Throws an exception if the block cannot be found or
* cannot be read successfully.
*/
def getBlockData(blockId: String): ManagedBuffer
def getBlockData(blockId: BlockId): ManagedBuffer

/**
* Put the block locally, using the given storage level.
*/
def putBlockData(blockId: String, data: ManagedBuffer, level: StorageLevel): Unit
def putBlockData(blockId: BlockId, data: ManagedBuffer, level: StorageLevel): Unit
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@ package org.apache.spark.network

import java.util.EventListener

import org.apache.spark.network.buffer.ManagedBuffer


/**
* Listener callback interface for [[BlockTransferService.fetchBlocks]].
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,16 +18,18 @@
package org.apache.spark.network

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

import org.apache.spark.network.buffer.ManagedBuffer

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

import org.apache.spark.Logging
import org.apache.spark.storage.StorageLevel

import org.apache.spark.util.Utils

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

/**
* Initialize the transfer service by giving it the BlockDataManager that can be used to fetch
Expand Down Expand Up @@ -92,10 +94,7 @@ abstract class BlockTransferService extends Closeable {
}
override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = {
lock.synchronized {
val ret = ByteBuffer.allocate(data.size.toInt)
ret.put(data.nioByteBuffer())
ret.flip()
result = Left(new NioManagedBuffer(ret))
result = Left(data)
lock.notify()
}
}
Expand Down
187 changes: 0 additions & 187 deletions core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala

This file was deleted.

Loading