diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
index e5fdebc65da8a092fa995e6735b08e937d41ca9e..8bf4489e1aca718025477b62d7bbc08fc9ba7c88 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -85,10 +85,14 @@ abstract class RDD[T: ClassTag](
   private def sc: SparkContext = {
     if (_sc == null) {
       throw new SparkException(
-        "RDD transformations and actions can only be invoked by the driver, not inside of other " +
-        "transformations; for example, rdd1.map(x => rdd2.values.count() * x) is invalid because " +
-        "the values transformation and count action cannot be performed inside of the rdd1.map " +
-        "transformation. For more information, see SPARK-5063.")
+        "This RDD lacks a SparkContext. It could happen in the following cases: \n(1) RDD " +
+        "transformations and actions are NOT invoked by the driver, but inside of other " +
+        "transformations; for example, rdd1.map(x => rdd2.values.count() * x) is invalid " +
+        "because the values transformation and count action cannot be performed inside of the " +
+        "rdd1.map transformation. For more information, see SPARK-5063.\n(2) When a Spark " +
+        "Streaming job recovers from checkpoint, this exception will be hit if a reference to " +
+        "an RDD not defined by the streaming job is used in DStream operations. For more " +
+        "information, See SPARK-13758.")
     }
     _sc
   }