Skip to content

Commit 1db5314

Browse files
committed
Merge pull request apache#4 from apache/master
merge upstream changes
2 parents 0e0245f + 41e0a21 commit 1db5314

File tree

43 files changed

+1492
-260
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

43 files changed

+1492
-260
lines changed

core/src/main/scala/org/apache/spark/Accumulators.scala

Lines changed: 15 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -36,15 +36,21 @@ import org.apache.spark.serializer.JavaSerializer
3636
*
3737
* @param initialValue initial value of accumulator
3838
* @param param helper object defining how to add elements of type `R` and `T`
39+
* @param name human-readable name for use in Spark's web UI
3940
* @tparam R the full accumulated data (result type)
4041
* @tparam T partial data that can be added in
4142
*/
4243
class Accumulable[R, T] (
4344
@transient initialValue: R,
44-
param: AccumulableParam[R, T])
45+
param: AccumulableParam[R, T],
46+
val name: Option[String])
4547
extends Serializable {
4648

47-
val id = Accumulators.newId
49+
def this(@transient initialValue: R, param: AccumulableParam[R, T]) =
50+
this(initialValue, param, None)
51+
52+
val id: Long = Accumulators.newId
53+
4854
@transient private var value_ = initialValue // Current value on master
4955
val zero = param.zero(initialValue) // Zero value to be passed to workers
5056
private var deserialized = false
@@ -219,8 +225,10 @@ GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Serializa
219225
* @param param helper object defining how to add elements of type `T`
220226
* @tparam T result type
221227
*/
222-
class Accumulator[T](@transient initialValue: T, param: AccumulatorParam[T])
223-
extends Accumulable[T,T](initialValue, param)
228+
class Accumulator[T](@transient initialValue: T, param: AccumulatorParam[T], name: Option[String])
229+
extends Accumulable[T,T](initialValue, param, name) {
230+
def this(initialValue: T, param: AccumulatorParam[T]) = this(initialValue, param, None)
231+
}
224232

225233
/**
226234
* A simpler version of [[org.apache.spark.AccumulableParam]] where the only data type you can add
@@ -281,4 +289,7 @@ private object Accumulators {
281289
}
282290
}
283291
}
292+
293+
def stringifyPartialValue(partialValue: Any) = "%s".format(partialValue)
294+
def stringifyValue(value: Any) = "%s".format(value)
284295
}

core/src/main/scala/org/apache/spark/SecurityManager.scala

Lines changed: 90 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -41,10 +41,19 @@ import org.apache.spark.deploy.SparkHadoopUtil
4141
* secure the UI if it has data that other users should not be allowed to see. The javax
4242
* servlet filter specified by the user can authenticate the user and then once the user
4343
* is logged in, Spark can compare that user versus the view acls to make sure they are
44-
* authorized to view the UI. The configs 'spark.ui.acls.enable' and 'spark.ui.view.acls'
44+
* authorized to view the UI. The configs 'spark.acls.enable' and 'spark.ui.view.acls'
4545
* control the behavior of the acls. Note that the person who started the application
4646
* always has view access to the UI.
4747
*
48+
* Spark has a set of modify acls (`spark.modify.acls`) that controls which users have permission
49+
* to modify a single application. This would include things like killing the application. By
50+
* default the person who started the application has modify access. For modify access through
51+
* the UI, you must have a filter that does authentication in place for the modify acls to work
52+
* properly.
53+
*
54+
* Spark also has a set of admin acls (`spark.admin.acls`) which is a set of users/administrators
55+
* who always have permission to view or modify the Spark application.
56+
*
4857
* Spark does not currently support encryption after authentication.
4958
*
5059
* At this point spark has multiple communication protocols that need to be secured and
@@ -137,18 +146,32 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging {
137146
private val sparkSecretLookupKey = "sparkCookie"
138147

139148
private val authOn = sparkConf.getBoolean("spark.authenticate", false)
140-
private var uiAclsOn = sparkConf.getBoolean("spark.ui.acls.enable", false)
149+
// keep spark.ui.acls.enable for backwards compatibility with 1.0
150+
private var aclsOn = sparkConf.getOption("spark.acls.enable").getOrElse(
151+
sparkConf.get("spark.ui.acls.enable", "false")).toBoolean
152+
153+
// admin acls should be set before view or modify acls
154+
private var adminAcls: Set[String] =
155+
stringToSet(sparkConf.get("spark.admin.acls", ""))
141156

142157
private var viewAcls: Set[String] = _
158+
159+
// list of users who have permission to modify the application. This should
160+
// apply to both UI and CLI for things like killing the application.
161+
private var modifyAcls: Set[String] = _
162+
143163
// always add the current user and SPARK_USER to the viewAcls
144-
private val defaultAclUsers = Seq[String](System.getProperty("user.name", ""),
164+
private val defaultAclUsers = Set[String](System.getProperty("user.name", ""),
145165
Option(System.getenv("SPARK_USER")).getOrElse(""))
166+
146167
setViewAcls(defaultAclUsers, sparkConf.get("spark.ui.view.acls", ""))
168+
setModifyAcls(defaultAclUsers, sparkConf.get("spark.modify.acls", ""))
147169

148170
private val secretKey = generateSecretKey()
149171
logInfo("SecurityManager: authentication " + (if (authOn) "enabled" else "disabled") +
150-
"; ui acls " + (if (uiAclsOn) "enabled" else "disabled") +
151-
"; users with view permissions: " + viewAcls.toString())
172+
"; ui acls " + (if (aclsOn) "enabled" else "disabled") +
173+
"; users with view permissions: " + viewAcls.toString() +
174+
"; users with modify permissions: " + modifyAcls.toString())
152175

153176
// Set our own authenticator to properly negotiate user/password for HTTP connections.
154177
// This is needed by the HTTP client fetching from the HttpServer. Put here so its
@@ -169,18 +192,51 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging {
169192
)
170193
}
171194

172-
private[spark] def setViewAcls(defaultUsers: Seq[String], allowedUsers: String) {
173-
viewAcls = (defaultUsers ++ allowedUsers.split(',')).map(_.trim()).filter(!_.isEmpty).toSet
195+
/**
196+
* Split a comma separated String, filter out any empty items, and return a Set of strings
197+
*/
198+
private def stringToSet(list: String): Set[String] = {
199+
list.split(',').map(_.trim).filter(!_.isEmpty).toSet
200+
}
201+
202+
/**
203+
* Admin acls should be set before the view or modify acls. If you modify the admin
204+
* acls you should also set the view and modify acls again to pick up the changes.
205+
*/
206+
def setViewAcls(defaultUsers: Set[String], allowedUsers: String) {
207+
viewAcls = (adminAcls ++ defaultUsers ++ stringToSet(allowedUsers))
174208
logInfo("Changing view acls to: " + viewAcls.mkString(","))
175209
}
176210

177-
private[spark] def setViewAcls(defaultUser: String, allowedUsers: String) {
178-
setViewAcls(Seq[String](defaultUser), allowedUsers)
211+
def setViewAcls(defaultUser: String, allowedUsers: String) {
212+
setViewAcls(Set[String](defaultUser), allowedUsers)
213+
}
214+
215+
def getViewAcls: String = viewAcls.mkString(",")
216+
217+
/**
218+
* Admin acls should be set before the view or modify acls. If you modify the admin
219+
* acls you should also set the view and modify acls again to pick up the changes.
220+
*/
221+
def setModifyAcls(defaultUsers: Set[String], allowedUsers: String) {
222+
modifyAcls = (adminAcls ++ defaultUsers ++ stringToSet(allowedUsers))
223+
logInfo("Changing modify acls to: " + modifyAcls.mkString(","))
224+
}
225+
226+
def getModifyAcls: String = modifyAcls.mkString(",")
227+
228+
/**
229+
* Admin acls should be set before the view or modify acls. If you modify the admin
230+
* acls you should also set the view and modify acls again to pick up the changes.
231+
*/
232+
def setAdminAcls(adminUsers: String) {
233+
adminAcls = stringToSet(adminUsers)
234+
logInfo("Changing admin acls to: " + adminAcls.mkString(","))
179235
}
180236

181-
private[spark] def setUIAcls(aclSetting: Boolean) {
182-
uiAclsOn = aclSetting
183-
logInfo("Changing acls enabled to: " + uiAclsOn)
237+
def setAcls(aclSetting: Boolean) {
238+
aclsOn = aclSetting
239+
logInfo("Changing acls enabled to: " + aclsOn)
184240
}
185241

186242
/**
@@ -224,22 +280,39 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging {
224280
* Check to see if Acls for the UI are enabled
225281
* @return true if UI authentication is enabled, otherwise false
226282
*/
227-
def uiAclsEnabled(): Boolean = uiAclsOn
283+
def aclsEnabled(): Boolean = aclsOn
228284

229285
/**
230286
* Checks the given user against the view acl list to see if they have
231-
* authorization to view the UI. If the UI acls must are disabled
232-
* via spark.ui.acls.enable, all users have view access.
287+
* authorization to view the UI. If the UI acls are disabled
288+
* via spark.acls.enable, all users have view access. If the user is null
289+
* it is assumed authentication is off and all users have access.
233290
*
234291
* @param user to see if is authorized
235292
* @return true is the user has permission, otherwise false
236293
*/
237294
def checkUIViewPermissions(user: String): Boolean = {
238-
logDebug("user=" + user + " uiAclsEnabled=" + uiAclsEnabled() + " viewAcls=" +
295+
logDebug("user=" + user + " aclsEnabled=" + aclsEnabled() + " viewAcls=" +
239296
viewAcls.mkString(","))
240-
if (uiAclsEnabled() && (user != null) && (!viewAcls.contains(user))) false else true
297+
if (aclsEnabled() && (user != null) && (!viewAcls.contains(user))) false else true
241298
}
242299

300+
/**
301+
* Checks the given user against the modify acl list to see if they have
302+
* authorization to modify the application. If the UI acls are disabled
303+
* via spark.acls.enable, all users have modify access. If the user is null
304+
* it is assumed authentication isn't turned on and all users have access.
305+
*
306+
* @param user to see if is authorized
307+
* @return true is the user has permission, otherwise false
308+
*/
309+
def checkModifyPermissions(user: String): Boolean = {
310+
logDebug("user=" + user + " aclsEnabled=" + aclsEnabled() + " modifyAcls=" +
311+
modifyAcls.mkString(","))
312+
if (aclsEnabled() && (user != null) && (!modifyAcls.contains(user))) false else true
313+
}
314+
315+
243316
/**
244317
* Check to see if authentication for the Spark communication protocols is enabled
245318
* @return true if authentication is enabled, otherwise false

core/src/main/scala/org/apache/spark/SparkConf.scala

Lines changed: 14 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -238,6 +238,20 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging {
238238
}
239239
}
240240

241+
// Validate memory fractions
242+
val memoryKeys = Seq(
243+
"spark.storage.memoryFraction",
244+
"spark.shuffle.memoryFraction",
245+
"spark.shuffle.safetyFraction",
246+
"spark.storage.unrollFraction",
247+
"spark.storage.safetyFraction")
248+
for (key <- memoryKeys) {
249+
val value = getDouble(key, 0.5)
250+
if (value > 1 || value < 0) {
251+
throw new IllegalArgumentException("$key should be between 0 and 1 (was '$value').")
252+
}
253+
}
254+
241255
// Check for legacy configs
242256
sys.env.get("SPARK_JAVA_OPTS").foreach { value =>
243257
val warning =

core/src/main/scala/org/apache/spark/SparkContext.scala

Lines changed: 19 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -760,6 +760,15 @@ class SparkContext(config: SparkConf) extends Logging {
760760
def accumulator[T](initialValue: T)(implicit param: AccumulatorParam[T]) =
761761
new Accumulator(initialValue, param)
762762

763+
/**
764+
* Create an [[org.apache.spark.Accumulator]] variable of a given type, with a name for display
765+
* in the Spark UI. Tasks can "add" values to the accumulator using the `+=` method. Only the
766+
* driver can access the accumulator's `value`.
767+
*/
768+
def accumulator[T](initialValue: T, name: String)(implicit param: AccumulatorParam[T]) = {
769+
new Accumulator(initialValue, param, Some(name))
770+
}
771+
763772
/**
764773
* Create an [[org.apache.spark.Accumulable]] shared variable, to which tasks can add values
765774
* with `+=`. Only the driver can access the accumuable's `value`.
@@ -769,6 +778,16 @@ class SparkContext(config: SparkConf) extends Logging {
769778
def accumulable[T, R](initialValue: T)(implicit param: AccumulableParam[T, R]) =
770779
new Accumulable(initialValue, param)
771780

781+
/**
782+
* Create an [[org.apache.spark.Accumulable]] shared variable, with a name for display in the
783+
* Spark UI. Tasks can add values to the accumuable using the `+=` operator. Only the driver can
784+
* access the accumuable's `value`.
785+
* @tparam T accumulator type
786+
* @tparam R type that can be added to the accumulator
787+
*/
788+
def accumulable[T, R](initialValue: T, name: String)(implicit param: AccumulableParam[T, R]) =
789+
new Accumulable(initialValue, param, Some(name))
790+
772791
/**
773792
* Create an accumulator from a "mutable collection" type.
774793
*

core/src/main/scala/org/apache/spark/SparkEnv.scala

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -35,7 +35,7 @@ import org.apache.spark.metrics.MetricsSystem
3535
import org.apache.spark.network.ConnectionManager
3636
import org.apache.spark.scheduler.LiveListenerBus
3737
import org.apache.spark.serializer.Serializer
38-
import org.apache.spark.shuffle.ShuffleManager
38+
import org.apache.spark.shuffle.{ShuffleMemoryManager, ShuffleManager}
3939
import org.apache.spark.storage._
4040
import org.apache.spark.util.{AkkaUtils, Utils}
4141

@@ -66,12 +66,9 @@ class SparkEnv (
6666
val httpFileServer: HttpFileServer,
6767
val sparkFilesDir: String,
6868
val metricsSystem: MetricsSystem,
69+
val shuffleMemoryManager: ShuffleMemoryManager,
6970
val conf: SparkConf) extends Logging {
7071

71-
// A mapping of thread ID to amount of memory, in bytes, used for shuffle aggregations
72-
// All accesses should be manually synchronized
73-
val shuffleMemoryMap = mutable.HashMap[Long, Long]()
74-
7572
private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]()
7673

7774
// A general, soft-reference map for metadata needed during HadoopRDD split computation
@@ -252,6 +249,8 @@ object SparkEnv extends Logging {
252249
val shuffleManager = instantiateClass[ShuffleManager](
253250
"spark.shuffle.manager", "org.apache.spark.shuffle.hash.HashShuffleManager")
254251

252+
val shuffleMemoryManager = new ShuffleMemoryManager(conf)
253+
255254
// Warn about deprecated spark.cache.class property
256255
if (conf.contains("spark.cache.class")) {
257256
logWarning("The spark.cache.class property is no longer being used! Specify storage " +
@@ -273,6 +272,7 @@ object SparkEnv extends Logging {
273272
httpFileServer,
274273
sparkFilesDir,
275274
metricsSystem,
275+
shuffleMemoryManager,
276276
conf)
277277
}
278278

0 commit comments

Comments
 (0)