Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
76 commits
Select commit Hold shift + click to select a range
711356b
[SPARK-3086] [SPARK-3043] [SPARK-3156] [mllib] DecisionTree aggregat…
jkbradley Sep 8, 2014
e16a8e7
SPARK-3337 Paranoid quoting in shell to allow install dirs with space…
ScrapCodes Sep 8, 2014
16a73c2
SPARK-2978. Transformation with MR shuffle semantics
sryza Sep 8, 2014
386bc24
Provide a default PYSPARK_PYTHON for python/run_tests
Sep 8, 2014
26bc765
[SQL] Minor edits to sql programming guide.
hcook Sep 8, 2014
939a322
[SPARK-3417] Use new-style classes in PySpark
mrocklin Sep 8, 2014
08ce188
[SPARK-3019] Pluggable block transfer interface (BlockTransferService)
rxin Sep 8, 2014
7db5339
[SPARK-3349][SQL] Output partitioning of limit should not be inherite…
Sep 8, 2014
50a4fa7
[SPARK-3443][MLLIB] update default values of tree:
mengxr Sep 9, 2014
ca0348e
SPARK-3423: [SQL] Implement BETWEEN for SQLParser
willb Sep 9, 2014
dc1dbf2
[SPARK-3414][SQL] Stores analyzed logical plan when registering a tem…
liancheng Sep 9, 2014
2b7ab81
[SPARK-3329][SQL] Don't depend on Hive SET pair ordering in tests.
willb Sep 9, 2014
092e2f1
SPARK-2425 Don't kill a still-running Application because of some mis…
markhamstra Sep 9, 2014
ce5cb32
[Build] Removed -Phive-thriftserver since this profile has been removed
liancheng Sep 9, 2014
c419e4f
[Docs] actorStream storageLevel default is MEMORY_AND_DISK_SER_2
melrief Sep 9, 2014
1e03cf7
[SPARK-3455] [SQL] **HOT FIX** Fix the unit test failure
chenghao-intel Sep 9, 2014
88547a0
SPARK-3422. JavaAPISuite.getHadoopInputSplits isn't used anywhere.
sryza Sep 9, 2014
f0f1ba0
SPARK-3404 [BUILD] SparkSubmitSuite fails with "spark-submit exits wi…
srowen Sep 9, 2014
2686233
[SPARK-3193]output errer info when Process exit code is not zero in t…
scwf Sep 9, 2014
02b5ac7
Minor - Fix trivial compilation warnings.
ScrapCodes Sep 9, 2014
07ee4a2
[SPARK-3176] Implement 'ABS and 'LAST' for sql
Sep 9, 2014
c110614
[SPARK-3448][SQL] Check for null in SpecificMutableRow.update
liancheng Sep 10, 2014
25b5b86
[SPARK-3458] enable python "with" statements for SparkContext
Sep 10, 2014
b734ed0
[SPARK-3395] [SQL] DSL sometimes incorrectly reuses attribute ids, br…
Sep 10, 2014
6f7a768
[SPARK-3286] - Cannot view ApplicationMaster UI when Yarn’s url schem…
Sep 10, 2014
a028330
[SPARK-3362][SQL] Fix resolution for casewhen with nulls.
adrian-wang Sep 10, 2014
f0c87dc
[SPARK-3363][SQL] Type Coercion should promote null to all other types.
adrian-wang Sep 10, 2014
26503fd
[HOTFIX] Fix scala style issue introduced by #2276.
JoshRosen Sep 10, 2014
1f4a648
SPARK-1713. Use a thread pool for launching executors.
sryza Sep 10, 2014
e4f4886
[SPARK-2096][SQL] Correctly parse dot notations
cloud-fan Sep 10, 2014
558962a
[SPARK-3411] Improve load-balancing of concurrently-submitted drivers…
WangTaoTheTonic Sep 10, 2014
79cdb9b
[SPARK-2207][SPARK-3272][MLLib]Add minimum information gain and minim…
Sep 10, 2014
84e2c8b
[SQL] Add test case with workaround for reading partitioned Avro files
marmbrus Sep 11, 2014
f92cde2
[SPARK-3447][SQL] Remove explicit conversion with JListWrapper to avo…
marmbrus Sep 11, 2014
c27718f
[SPARK-2781][SQL] Check resolution of LogicalPlans in Analyzer.
staple Sep 11, 2014
ed1980f
[SPARK-2140] Updating heap memory calculation for YARN stable and alpha.
Sep 11, 2014
1ef656e
[SPARK-3047] [PySpark] add an option to use str in textFileRDD
davies Sep 11, 2014
ca83f1e
[SPARK-2917] [SQL] Avoid table creation in logical plan analyzing for…
chenghao-intel Sep 11, 2014
4bc9e04
[SPARK-3390][SQL] sqlContext.jsonRDD fails on a complex structure of …
yhuai Sep 11, 2014
6324eb7
[Spark-3490] Disable SparkUI for tests
andrewor14 Sep 12, 2014
ce59725
[SPARK-3429] Don't include the empty string "" as a defaultAclUser
ash211 Sep 12, 2014
f858f46
SPARK-3462 push down filters and projections into Unions
Sep 12, 2014
33c7a73
SPARK-2482: Resolve sbt warnings during build
witgo Sep 12, 2014
42904b8
[SPARK-3465] fix task metrics aggregation in local mode
davies Sep 12, 2014
b8634df
[SPARK-3160] [SPARK-3494] [mllib] DecisionTree: eliminate pre-alloca…
jkbradley Sep 12, 2014
f116f76
[SPARK-2558][DOCS] Add --queue example to YARN doc
kramimus Sep 12, 2014
5333776
[PySpark] Add blank line so that Python RDD.top() docstring renders c…
rnowling Sep 12, 2014
8194fc6
[SPARK-3481] [SQL] Eliminate the error log in local Hive comparison test
chenghao-intel Sep 12, 2014
eae81b0
MAINTENANCE: Automated closing of pull requests.
pwendell Sep 12, 2014
15a5645
[SPARK-3427] [GraphX] Avoid active vertex tracking in static PageRank
ankurdave Sep 12, 2014
1d76796
SPARK-3014. Log a more informative messages in a couple failure scena…
sryza Sep 12, 2014
af25838
[SPARK-3217] Add Guava to classpath when SPARK_PREPEND_CLASSES is set.
Sep 12, 2014
25311c2
[SPARK-3456] YarnAllocator on alpha can lose container requests to RM
tgravescs Sep 13, 2014
71af030
[SPARK-3094] [PySpark] compatitable with PyPy
davies Sep 13, 2014
885d162
[SPARK-3500] [SQL] use JavaSchemaRDD as SchemaRDD._jschema_rdd
davies Sep 13, 2014
6d887db
[SPARK-3515][SQL] Moves test suite setup code to beforeAll rather tha…
liancheng Sep 13, 2014
2584ea5
[SPARK-3469] Make sure all TaskCompletionListener are called even wit…
rxin Sep 13, 2014
e11eeb7
[SQL][Docs] Update SQL programming guide to show the correct default …
yhuai Sep 13, 2014
feaa370
SPARK-3470 [CORE] [STREAMING] Add Closeable / close() to Java context…
srowen Sep 13, 2014
b4dded4
Proper indent for the previous commit.
rxin Sep 13, 2014
a523cea
[SQL] [Docs] typo fixes
nchammas Sep 13, 2014
184cd51
[SPARK-3481][SQL] Removes the evil MINOR HACK
liancheng Sep 13, 2014
7404924
[SPARK-3294][SQL] Eliminates boxing costs from in-memory columnar sto…
liancheng Sep 13, 2014
0f8c4ed
[SQL] Decrease partitions when testing
marmbrus Sep 13, 2014
2aea0da
[SPARK-3030] [PySpark] Reuse Python worker
davies Sep 13, 2014
4e3fbe8
[SPARK-3463] [PySpark] aggregate and show spilled bytes in Python
davies Sep 14, 2014
c243b21
SPARK-3039: Allow spark to be built using avro-mapred for hadoop2
bbossy Sep 15, 2014
f493f79
[SPARK-3452] Maven build should skip publishing artifacts people shou…
ScrapCodes Sep 15, 2014
cc14644
[SPARK-3410] The priority of shutdownhook for ApplicationMaster shoul…
sarutak Sep 15, 2014
fe2b1d6
[SPARK-3425] do not set MaxPermSize for OpenJDK 1.8
Sep 15, 2014
e59fac1
[SPARK-3518] Remove wasted statement in JsonProtocol
sarutak Sep 15, 2014
37d9252
[SPARK-2714] DAGScheduler logs jobid when runJob finishes
YanTangZhai Sep 15, 2014
3b93128
[SPARK-3396][MLLIB] Use SquaredL2Updater in LogisticRegressionWithSGD
BigCrunsh Sep 16, 2014
983d6a9
[MLlib] Update SVD documentation in IndexedRowMatrix
rezazadeh Sep 16, 2014
fdb302f
[SPARK-3516] [mllib] DecisionTree: Add minInstancesPerNode, minInfoGa…
Sep 16, 2014
da33acb
[SPARK-2951] [PySpark] support unpickle array.array for Python 2.6
davies Sep 16, 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-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 <[email protected]>

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
rxin committed Sep 8, 2014
commit 08ce18881e09c6e91db9c410d1d9ce1e5ae63a62
15 changes: 9 additions & 6 deletions core/src/main/scala/org/apache/spark/SparkEnv.scala
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,8 @@ import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.api.python.PythonWorkerFactory
import org.apache.spark.broadcast.BroadcastManager
import org.apache.spark.metrics.MetricsSystem
import org.apache.spark.network.ConnectionManager
import org.apache.spark.network.BlockTransferService
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}
Expand Down Expand Up @@ -59,8 +60,8 @@ class SparkEnv (
val mapOutputTracker: MapOutputTracker,
val shuffleManager: ShuffleManager,
val broadcastManager: BroadcastManager,
val blockTransferService: BlockTransferService,
val blockManager: BlockManager,
val connectionManager: ConnectionManager,
val securityManager: SecurityManager,
val httpFileServer: HttpFileServer,
val sparkFilesDir: String,
Expand Down Expand Up @@ -88,6 +89,8 @@ class SparkEnv (
// down, but let's call it anyway in case it gets fixed in a later release
// UPDATE: In Akka 2.1.x, this hangs if there are remote actors, so we can't call it.
// actorSystem.awaitTermination()

// Note that blockTransferService is stopped by BlockManager since it is started by it.
}

private[spark]
Expand Down Expand Up @@ -223,14 +226,14 @@ object SparkEnv extends Logging {

val shuffleMemoryManager = new ShuffleMemoryManager(conf)

val blockTransferService = new NioBlockTransferService(conf, securityManager)

val blockManagerMaster = new BlockManagerMaster(registerOrLookup(
"BlockManagerMaster",
new BlockManagerMasterActor(isLocal, conf, listenerBus)), conf, isDriver)

val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster,
serializer, conf, securityManager, mapOutputTracker, shuffleManager)

val connectionManager = blockManager.connectionManager
serializer, conf, mapOutputTracker, shuffleManager, blockTransferService)

val broadcastManager = new BroadcastManager(isDriver, conf, securityManager)

Expand Down Expand Up @@ -278,8 +281,8 @@ object SparkEnv extends Logging {
mapOutputTracker,
shuffleManager,
broadcastManager,
blockTransferService,
blockManager,
connectionManager,
securityManager,
httpFileServer,
sparkFilesDir,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,21 +17,20 @@

package org.apache.spark.network

import java.nio.ByteBuffer
import org.apache.spark.{SecurityManager, SparkConf}
import org.apache.spark.storage.StorageLevel

private[spark] object ReceiverTest {
def main(args: Array[String]) {
val conf = new SparkConf
val manager = new ConnectionManager(9999, conf, new SecurityManager(conf))
println("Started connection manager with id = " + manager.id)

manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => {
/* println("Received [" + msg + "] from [" + id + "] at " + System.currentTimeMillis) */
val buffer = ByteBuffer.wrap("response".getBytes("utf-8"))
Some(Message.createBufferMessage(buffer, msg.id))
})
Thread.currentThread.join()
}
}
trait BlockDataManager {

/**
* Interface to get local block data.
*
* @return Some(buffer) if the block exists locally, and None if it doesn't.
*/
def getBlockData(blockId: String): Option[ManagedBuffer]

/**
* Put the block locally, using the given storage level.
*/
def putBlockData(blockId: String, data: ManagedBuffer, level: StorageLevel): Unit
}
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
}
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)
}
}

This file was deleted.

Loading