diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index a54b091a64d5..d55b5592c897 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -429,7 +429,11 @@ package object config { "external shuffle service, this feature can only be worked when external shuffle" + "service is newer than Spark 2.2.") .bytesConf(ByteUnit.BYTE) - .createWithDefault(Long.MaxValue) + // fetch-to-mem is guaranteed to fail if the message is bigger than 2 GB, so we might + // as well use fetch-to-disk in that case. The message includes some metadata in addition + // to the block data itself (in particular UploadBlock has a lot of metadata), so we leave + // extra room. + .createWithDefault(Int.MaxValue - 512) private[spark] val TASK_METRICS_TRACK_UPDATED_BLOCK_STATUSES = ConfigBuilder("spark.taskMetrics.trackUpdatedBlockStatuses") diff --git a/docs/configuration.md b/docs/configuration.md index 64af0e98a82f..1c5c1aeddcf3 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -580,13 +580,15 @@ Apart from these, the following properties are also available, and may be useful
spark.maxRemoteBlockSizeFetchToMem