From e807118eef9e0214170ff62c828524d237bd58e3 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Mon, 9 Mar 2020 12:17:59 -0700 Subject: [PATCH] [SPARK-31055][DOCS] Update config docs for shuffle local host reads to have dep on external shuffle service ### What changes were proposed in this pull request? with SPARK-27651 we now support host local reads for shuffle, but only when external shuffle service is enabled. Update the config docs to state that. ### Why are the changes needed? clarify dependency ### Does this PR introduce any user-facing change? no ### How was this patch tested? n/a Closes #27812 from tgravescs/SPARK-27651-follow. Authored-by: Thomas Graves Signed-off-by: Dongjoon Hyun --- .../main/scala/org/apache/spark/internal/config/package.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 d4e15621bbe0e..f3195d978ec6d 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 @@ -1216,7 +1216,8 @@ package object config { private[spark] val SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED = ConfigBuilder("spark.shuffle.readHostLocalDisk") - .doc(s"If enabled (and `${SHUFFLE_USE_OLD_FETCH_PROTOCOL.key}` is disabled), shuffle " + + .doc(s"If enabled (and `${SHUFFLE_USE_OLD_FETCH_PROTOCOL.key}` is disabled and external " + + s"shuffle `${SHUFFLE_SERVICE_ENABLED.key}` is enabled), shuffle " + "blocks requested from those block managers which are running on the same host are read " + "from the disk directly instead of being fetched as remote blocks over the network.") .booleanConf