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 f8df5b2a08866b1b0f3e650395152b0908323822..38f8f36a4a4db017aee8b6aa7832cb6dfadc4f1c 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
@@ -437,6 +437,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
    * 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.
+   *
+   * Note: As currently implemented, groupByKey must be able to hold all the key-value pairs for any
+   * key in memory. If a key has too many values, it can result in an [[OutOfMemoryError]].
    */
   def groupByKey(partitioner: Partitioner): RDD[(K, Iterable[V])] = {
     // groupByKey shouldn't use map side combine because map side combine does not
@@ -458,6 +461,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
    * 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.
+   *
+   * Note: As currently implemented, groupByKey must be able to hold all the key-value pairs for any
+   * key in memory. If a key has too many values, it can result in an [[OutOfMemoryError]].
    */
   def groupByKey(numPartitions: Int): RDD[(K, Iterable[V])] = {
     groupByKey(new HashPartitioner(numPartitions))