Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
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
Original file line number Diff line number Diff line change
Expand Up @@ -908,7 +908,7 @@ private[spark] object MapOutputTracker extends Logging {
if (arr.length >= minBroadcastSize) {
// Use broadcast instead.
// Important arr(0) is the tag == DIRECT, ignore that while deserializing !
val bcast = broadcastManager.newBroadcast(arr, isLocal)
val bcast = broadcastManager.newDriverBroadcast(arr, isLocal)
// toByteArray creates copy, so we can reuse out
out.reset()
out.write(BROADCAST)
Expand Down
29 changes: 26 additions & 3 deletions core/src/main/scala/org/apache/spark/SparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHad
import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat}

import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.broadcast.{Broadcast, BroadcastMode}
import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil}
import org.apache.spark.executor.{ExecutorMetrics, ExecutorMetricsSource}
import org.apache.spark.input.{FixedLengthBinaryInputFormat, PortableDataStream, StreamInputFormat, WholeTextFileInputFormat}
Expand Down Expand Up @@ -1488,13 +1488,36 @@ class SparkContext(config: SparkConf) extends Logging {
assertNotStopped()
require(!classOf[RDD[_]].isAssignableFrom(classTag[T].runtimeClass),
"Can not directly broadcast RDDs; instead, call collect() and broadcast the result.")
val bc = env.broadcastManager.newBroadcast[T](value, isLocal)
val callSite = getCallSite
val bc = env.broadcastManager.newDriverBroadcast[T](value, isLocal)
val callSite = getCallSite()
logInfo("Created broadcast " + bc.id + " from " + callSite.shortForm)
cleaner.foreach(_.registerBroadcastForCleanup(bc))
bc
}

/**
* :: DeveloperApi ::
* Broadcast a read-only variable to the cluster, returning a
* [[org.apache.spark.broadcast.Broadcast]] object for reading it in distributed functions.
* The variable will be sent to each cluster only once.
*
* Notice that the RDD to be broadcasted should be cached and materialized first so we can
* access its data on the executors.
*/
@DeveloperApi
def broadcast[T: ClassTag, U: ClassTag](
rdd: RDD[T], mode: BroadcastMode[T]): Broadcast[U] = {
assertNotStopped()
require(!classOf[RDD[_]].isAssignableFrom(classTag[T].runtimeClass),
"Can not directly broadcast RDDs; instead, call collect() and broadcast the result.")
val bc = env.broadcastManager.newExecutorBroadcast[T, U](rdd, mode, isLocal)
rdd.broadcast(bc)
val callSite = getCallSite()
logInfo("Created executor broadcast " + bc.id + " from " + callSite.shortForm)
cleaner.foreach(_.registerBroadcastForCleanup(bc))
bc
}

/**
* Add a file to be downloaded with this Spark job on every node.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,8 @@ package org.apache.spark.broadcast

import scala.reflect.ClassTag

import org.apache.spark.SecurityManager
import org.apache.spark.SparkConf
import org.apache.spark.{SecurityManager, SparkConf}
import org.apache.spark.rdd.RDD

/**
* An interface for all the broadcast implementations in Spark (to allow
Expand All @@ -38,7 +38,22 @@ private[spark] trait BroadcastFactory {
* @param isLocal whether we are in local mode (single JVM process)
* @param id unique id representing this broadcast variable
*/
def newBroadcast[T: ClassTag](value: T, isLocal: Boolean, id: Long): Broadcast[T]
def newDriverBroadcast[T: ClassTag](value: T, isLocal: Boolean, id: Long): Broadcast[T]

/**
* Creates a new broadcast variable which is broadcasted on executors without collecting first
* to the driver.
*
* @param rdd the RDD to be broadcasted among executors
* @param mode the broadcast mode used to transform the result of RDD to broadcasted object
* @param isLocal whether we are in local mode (single JVM process)
* @param id unique id representing this broadcast variable
*/
def newExecutorBroadcast[T: ClassTag, U: ClassTag](
rdd: RDD[T],
mode: BroadcastMode[T],
isLocal: Boolean,
id: Long): Broadcast[U]

def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean): Unit

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import org.apache.commons.collections.map.{AbstractReferenceMap, ReferenceMap}
import org.apache.spark.{SecurityManager, SparkConf}
import org.apache.spark.api.python.PythonBroadcast
import org.apache.spark.internal.Logging
import org.apache.spark.rdd.RDD

private[spark] class BroadcastManager(
val isDriver: Boolean,
Expand Down Expand Up @@ -62,7 +63,7 @@ private[spark] class BroadcastManager(
.asInstanceOf[java.util.Map[Any, Any]]
)

def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean): Broadcast[T] = {
def newDriverBroadcast[T: ClassTag](value_ : T, isLocal: Boolean): Broadcast[T] = {
val bid = nextBroadcastId.getAndIncrement()
value_ match {
case pb: PythonBroadcast =>
Expand All @@ -74,7 +75,15 @@ private[spark] class BroadcastManager(

case _ => // do nothing
}
broadcastFactory.newBroadcast[T](value_, isLocal, bid)
broadcastFactory.newDriverBroadcast[T](value_, isLocal, bid)
}

def newExecutorBroadcast[T: ClassTag, U: ClassTag](
rdd_ : RDD[T],
mode: BroadcastMode[T],
isLocal: Boolean): Broadcast[U] = {
broadcastFactory.newExecutorBroadcast[T, U](rdd_, mode, isLocal,
nextBroadcastId.getAndIncrement())
}

def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean): Unit = {
Expand Down
30 changes: 30 additions & 0 deletions core/src/main/scala/org/apache/spark/broadcast/BroadcastMode.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
/*
* 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.broadcast

/**
* The trait used in executor side broadcast. The implementation of `transform` identify the shape
* in which the results of a RDD are broadcasted.
*
* @tparam T The type of RDD elements.
*/
trait BroadcastMode[T] extends Serializable {
def transform(rows: Array[T]): Any
def transform(rows: Iterator[T], sizeHint: Option[Long]): Any
def canonicalized: BroadcastMode[T] = this
}
Loading