aboutsummaryrefslogtreecommitdiff
path: root/core/src/main/scala/org/apache/spark/SparkConf.scala
diff options
context:
space:
mode:
Diffstat (limited to 'core/src/main/scala/org/apache/spark/SparkConf.scala')
-rw-r--r--core/src/main/scala/org/apache/spark/SparkConf.scala32
1 files changed, 11 insertions, 21 deletions
diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index 340e1f7824..36e240e618 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -344,17 +344,6 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging {
.map{case (k, v) => (k.substring(prefix.length), v)}
}
- /** Get all akka conf variables set on this SparkConf */
- def getAkkaConf: Seq[(String, String)] =
- /* This is currently undocumented. If we want to make this public we should consider
- * nesting options under the spark namespace to avoid conflicts with user akka options.
- * Otherwise users configuring their own akka code via system properties could mess up
- * spark's akka options.
- *
- * E.g. spark.akka.option.x.y.x = "value"
- */
- getAll.filter { case (k, _) => isAkkaConf(k) }
-
/**
* Returns the Spark application id, valid in the Driver after TaskScheduler registration and
* from the start in the Executor.
@@ -600,7 +589,9 @@ private[spark] object SparkConf extends Logging {
"spark.yarn.max.executor.failures" -> Seq(
AlternateConfig("spark.yarn.max.worker.failures", "1.5")),
"spark.memory.offHeap.enabled" -> Seq(
- AlternateConfig("spark.unsafe.offHeap", "1.6"))
+ AlternateConfig("spark.unsafe.offHeap", "1.6")),
+ "spark.rpc.message.maxSize" -> Seq(
+ AlternateConfig("spark.akka.frameSize", "1.6"))
)
/**
@@ -616,20 +607,12 @@ private[spark] object SparkConf extends Logging {
}
/**
- * Return whether the given config is an akka config (e.g. akka.actor.provider).
- * Note that this does not include spark-specific akka configs (e.g. spark.akka.timeout).
- */
- def isAkkaConf(name: String): Boolean = name.startsWith("akka.")
-
- /**
* Return whether the given config should be passed to an executor on start-up.
*
- * Certain akka and authentication configs are required from the executor when it connects to
+ * Certain authentication configs are required from the executor when it connects to
* the scheduler, while the rest of the spark configs can be inherited from the driver later.
*/
def isExecutorStartupConf(name: String): Boolean = {
- isAkkaConf(name) ||
- name.startsWith("spark.akka") ||
(name.startsWith("spark.auth") && name != SecurityManager.SPARK_AUTH_SECRET_CONF) ||
name.startsWith("spark.ssl") ||
name.startsWith("spark.rpc") ||
@@ -664,12 +647,19 @@ private[spark] object SparkConf extends Logging {
logWarning(
s"The configuration key '$key' has been deprecated as of Spark ${cfg.version} and " +
s"may be removed in the future. ${cfg.deprecationMessage}")
+ return
}
allAlternatives.get(key).foreach { case (newKey, cfg) =>
logWarning(
s"The configuration key '$key' has been deprecated as of Spark ${cfg.version} and " +
s"and may be removed in the future. Please use the new key '$newKey' instead.")
+ return
+ }
+ if (key.startsWith("spark.akka") || key.startsWith("spark.ssl.akka")) {
+ logWarning(
+ s"The configuration key $key is not supported any more " +
+ s"because Spark doesn't use Akka since 2.0")
}
}