diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index 951bfd79d0d6ab98e5239d5bd3392ad35fc48eed..45d19bcbfa6f2e3f891e7802c3279ba37757236b 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -192,7 +192,15 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging {
   }
 
   /** Get all akka conf variables set on this SparkConf */
-  def getAkkaConf: Seq[(String, String)] =  getAll.filter {case (k, v) => k.startsWith("akka.")}
+  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, v) => k.startsWith("akka.")}
 
   /** Does the configuration contain a given parameter? */
   def contains(key: String): Boolean = settings.contains(key)
diff --git a/docs/configuration.md b/docs/configuration.md
index 00864906b3c7b2cf5b4f73bd6ff3b74b0654d8eb..4c2e9cc4798d41ef5225800662524d5cf9058b3c 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -379,13 +379,6 @@ Apart from these, the following properties are also available, and may be useful
     Too large a value decreases parallelism during broadcast (makes it slower); however, if it is too small, <code>BlockManager</code> might take a performance hit.
   </td>
 </tr>
-<tr>
-  <td>akka.x.y....</td>
-  <td>value</td>
-  <td>
-    An arbitrary akka configuration can be set directly on spark conf and it is applied for all the ActorSystems created spark wide for that SparkContext and its assigned executors as well.
-  </td>
-</tr>
 
 <tr>
   <td>spark.shuffle.consolidateFiles</td>