diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
index fc9beb166befed5d75019eefd405aa1615eb8095..9d62d53fcb23f38db149edd18d284387b6bab45d 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
@@ -353,9 +353,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
    * Group the values for each key in the RDD into a single sequence. Allows controlling the
    * partitioning of the resulting key-value pair RDD by passing a Partitioner.
    *
-   * Note: If you are grouping in order to perform an aggregation (such as a sum or average) over
-   * each key, using [[PairRDDFunctions.reduceByKey]] or [[PairRDDFunctions.combineByKey]]
-   * will provide much better performance.
+   * Note: This operation may be very expensive. If you are grouping in order to perform an
+   * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]]
+   * or [[PairRDDFunctions.reduceByKey]] will provide much better performance.
    */
   def groupByKey(partitioner: Partitioner): RDD[(K, Iterable[V])] = {
     // groupByKey shouldn't use map side combine because map side combine does not
@@ -373,9 +373,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
    * Group the values for each key in the RDD into a single sequence. Hash-partitions the
    * resulting RDD with into `numPartitions` partitions.
    *
-   * Note: If you are grouping in order to perform an aggregation (such as a sum or average) over
-   * each key, using [[PairRDDFunctions.reduceByKey]] or [[PairRDDFunctions.combineByKey]]
-   * will provide much better performance.
+   * Note: This operation may be very expensive. If you are grouping in order to perform an
+   * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]]
+   * or [[PairRDDFunctions.reduceByKey]] will provide much better performance.
    */
   def groupByKey(numPartitions: Int): RDD[(K, Iterable[V])] = {
     groupByKey(new HashPartitioner(numPartitions))
@@ -462,9 +462,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
    * Group the values for each key in the RDD into a single sequence. Hash-partitions the
    * resulting RDD with the existing partitioner/parallelism level.
    *
-   * Note: If you are grouping in order to perform an aggregation (such as a sum or average) over
-   * each key, using [[PairRDDFunctions.reduceByKey]] or [[PairRDDFunctions.combineByKey]]
-   * will provide much better performance,
+   * Note: This operation may be very expensive. If you are grouping in order to perform an
+   * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]]
+   * or [[PairRDDFunctions.reduceByKey]] will provide much better performance.
    */
   def groupByKey(): RDD[(K, Iterable[V])] = {
     groupByKey(defaultPartitioner(self))
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 4e841bc992bff47cad6c57d8c001e7aed138d4d3..a25f263bea5c1ec91c8f548a434210ba81fb7cb0 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -509,6 +509,10 @@ abstract class RDD[T: ClassTag](
   /**
    * Return an RDD of grouped items. Each group consists of a key and a sequence of elements
    * mapping to that key.
+   *
+   * Note: This operation may be very expensive. If you are grouping in order to perform an
+   * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]]
+   * or [[PairRDDFunctions.reduceByKey]] will provide much better performance.
    */
   def groupBy[K](f: T => K)(implicit kt: ClassTag[K]): RDD[(K, Iterable[T])] =
     groupBy[K](f, defaultPartitioner(this))
@@ -516,6 +520,10 @@ abstract class RDD[T: ClassTag](
   /**
    * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements
    * mapping to that key.
+   *
+   * Note: This operation may be very expensive. If you are grouping in order to perform an
+   * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]]
+   * or [[PairRDDFunctions.reduceByKey]] will provide much better performance.
    */
   def groupBy[K](f: T => K, numPartitions: Int)(implicit kt: ClassTag[K]): RDD[(K, Iterable[T])] =
     groupBy(f, new HashPartitioner(numPartitions))
@@ -523,6 +531,10 @@ abstract class RDD[T: ClassTag](
   /**
    * Return an RDD of grouped items. Each group consists of a key and a sequence of elements
    * mapping to that key.
+   *
+   * Note: This operation may be very expensive. If you are grouping in order to perform an
+   * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]]
+   * or [[PairRDDFunctions.reduceByKey]] will provide much better performance.
    */
   def groupBy[K](f: T => K, p: Partitioner)(implicit kt: ClassTag[K], ord: Ordering[K] = null)
       : RDD[(K, Iterable[T])] = {